You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@hbase.apache.org by Wayne <wa...@gmail.com> on 2011/01/05 17:10:18 UTC

JVM OOM

I am still struggling with the JVM. We just had a hard OOM crash of a region
server after only running for 36 hours. Any help would be greatly
appreciated. Do we need to restart nodes every 24 hours under load?  GC
Pauses are something we are trying to plan for, but full out OOM crashes are
a new problem.

The message below seems to be where it starts going bad. It is followed by
no less than 63 Concurrent Mode Failure errors over a 16 minute period.

*GC locker: Trying a full collection because scavenge failed*

Lastly here is the end (after the 63 CMF errors).

Heap
 par new generation   total 1887488K, used 303212K [0x00000005fae00000,
0x000000067ae00000, 0x000000067ae00000)
  eden space 1677824K,  18% used [0x00000005fae00000, 0x000000060d61b078,
0x0000000661480000)
  from space 209664K,   0% used [0x000000066e140000, 0x000000066e140000,
0x000000067ae00000)
  to   space 209664K,   0% used [0x0000000661480000, 0x0000000661480000,
0x000000066e140000)
 concurrent mark-sweep generation total 6291456K, used 2440155K
[0x000000067ae00000, 0x00000007fae00000, 0x00000007fae00000)
 concurrent-mark-sweep perm gen total 31704K, used 18999K
[0x00000007fae00000, 0x00000007fccf6000, 0x0000000800000000)

Here again are our custom settings in case there are some suggestions out
there. Are we making it worse with these settings? What should we try next?

        -XX:+UseCMSInitiatingOccupancyOnly
        -XX:CMSInitiatingOccupancyFraction=60
        -XX:+CMSParallelRemarkEnabled
        -XX:SurvivorRatio=8
        -XX:NewRatio=3
        -XX:MaxTenuringThreshold=1


Thanks!

Re: JVM OOM

Posted by Stack <st...@duboce.net>.
Wayne:

In case you didn't see it, one of the lads had a suggestion over in
https://issues.apache.org/jira/browse/HBASE-3421.

St.Ack

On Wed, Jan 5, 2011 at 1:33 PM, Stack <st...@duboce.net> wrote:
> k.  Thanks for stack trace.  Its unrelated to compactions but that
> don't mean compaction not responsible.
>
> I'm filing an issue for wide rows causing OOME (HBASE-3421).  Maybe
> its not the case but we should check it out at least because I know of
> others who have schemas where the row width can grow unbounded.
>
> Thanks,
> St.Ack
>
>
> On Wed, Jan 5, 2011 at 1:08 PM, Wayne <wa...@gmail.com> wrote:
>> Assuming it was big rows I dropped the table and have started loading it
>> again (I could not share the data anyway). We will see it again eventually
>> if it is not the large rows. We have changed our data model to be "taller"
>> so the 30 million column row is now gone for good.
>>
>> Below is the contents of the .out file from the first OOM. I found the hprof
>> files, but I have no public place to put them. At 10GB I can not even get
>> them to my laptop to look at either.
>>
>> Thanks.
>>
>> [Unloading class sun.reflect.GeneratedSerializationConstructorAccessor2]
>> [Unloading class sun.reflect.GeneratedSerializationConstructorAccessor3]
>> [Unloading class sun.reflect.GeneratedSerializationConstructorAccessor1]
>> [Unloading class sun.reflect.GeneratedConstructorAccessor10]
>> [Unloading class sun.reflect.GeneratedConstructorAccessor21]
>> [Unloading class sun.reflect.GeneratedConstructorAccessor23]
>> [Unloading class sun.reflect.GeneratedMethodAccessor17]
>> [Unloading class sun.reflect.GeneratedMethodAccessor1]
>> [Unloading class sun.reflect.GeneratedMethodAccessor19]
>> [Unloading class sun.reflect.GeneratedMethodAccessor18]
>> [Unloading class sun.reflect.GeneratedMethodAccessor16]
>> java.lang.OutOfMemoryError: Java heap space
>> Dumping heap to java_pid17874.hprof ...
>> Exception in thread "pool-1-thread-6" java.lang.OutOfMemoryError: Java
>> heap space
>>        at org.apache.hadoop.hbase.client.Put.readFields(Put.java:500)
>> Heap dump file created [9689854890 bytes in 151.877 secs]
>>        at org.apache.hadoop.hbase.client.MultiPut.readFields(MultiPut.java:111)
>>        at org.apache.hadoop.hbase.io.HbaseObjectWritable.readObject(HbaseObjectWritable.java:461)
>>        at org.apache.hadoop.hbase.ipc.HBaseRPC$Invocation.readFields(HBaseRPC.java:124)
>>        at org.apache.hadoop.hbase.ipc.HBaseServer$Connection.processData(HBaseServer.java:959)
>>        at org.apache.hadoop.hbase.ipc.HBaseServer$Connection.readAndProcess(HBaseServer.java:927)
>>        at org.apache.hadoop.hbase.ipc.HBaseServer$Listener.doRead(HBaseServer.java:503)
>>        at org.apache.hadoop.hbase.ipc.HBaseServer$Listener$Reader.run(HBaseServer.java:297)
>>        at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown Source)
>>        at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
>>        at java.lang.Thread.run(Unknown Source)
>> Exception in thread "ResponseProcessor for block
>> blk_2176268114489978801_654636" java.lang.OutOfMemoryError: Java heap
>> space
>>        at java.util.HashMap.newKeyIterator(Unknown Source)
>>        at java.util.HashMap$KeySet.iterator(Unknown Source)
>>        at java.util.HashSet.iterator(Unknown Source)
>>        at sun.nio.ch.SelectorImpl.processDeregisterQueue(Unknown Source)
>>        at sun.nio.ch.EPollSelectorImpl.doSelect(Unknown Source)
>>        at sun.nio.ch.SelectorImpl.lockAndDoSelect(Unknown Source)
>>        at sun.nio.ch.SelectorImpl.select(Unknown Source)
>>        at org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.select(SocketIOWithTimeout.java:332)
>>        at org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:157)
>>        at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:155)
>>        at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:128)
>>        at java.io.DataInputStream.readFully(Unknown Source)
>>        at java.io.DataInputStream.readLong(Unknown Source)
>>        at org.apache.hadoop.hdfs.protocol.DataTransferProtocol$PipelineAck.readFields(DataTransferProtocol.java:122)
>>        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:2547)
>> Exception in thread "pool-1-thread-7" java.lang.OutOfMemoryError: Java
>> heap space
>>        at org.apache.hadoop.hbase.client.Put.readFields(Put.java:495)
>>        at org.apache.hadoop.hbase.client.MultiPut.readFields(MultiPut.java:111)
>>        at org.apache.hadoop.hbase.io.HbaseObjectWritable.readObject(HbaseObjectWritable.java:461)
>>        at org.apache.hadoop.hbase.ipc.HBaseRPC$Invocation.readFields(HBaseRPC.java:124)
>>        at org.apache.hadoop.hbase.ipc.HBaseServer$Connection.processData(HBaseServer.java:959)
>>        at org.apache.hadoop.hbase.ipc.HBaseServer$Connection.readAndProcess(HBaseServer.java:927)
>>        at org.apache.hadoop.hbase.ipc.HBaseServer$Listener.doRead(HBaseServer.java:503)
>>        at org.apache.hadoop.hbase.ipc.HBaseServer$Listener$Reader.run(HBaseServer.java:297)
>>        at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown Source)
>>        at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
>>        at java.lang.Thread.run(Unknown Source)
>> Exception in thread "pool-1-thread-9" java.lang.OutOfMemoryError: Java
>> heap space
>>
>>
>>
>> On Wed, Jan 5, 2011 at 3:36 PM, Stack <st...@duboce.net> wrote:
>>
>>> That could be it.  Do you know names of participating files?  The
>>> should be printed in the log.  Can you save aside that Store, tar it
>>> up and put it somewhere we can pull (if data is ok for others to see)?
>>>  If we are OOME'ing on big rows, thats something I know some fellas
>>> for sure would be interested in fixing.
>>>
>>> (There is actually a 'limit' on number of kvs to get at a time when
>>> scanning unused by the compaction code -- perhaps thats all it would
>>> take to fix this issue?)
>>>
>>> Its highly unlikely but I'll ask anyways, was the OOME in same
>>> location both times?  (OOME stack trace may be in the .out file rather
>>> than the .log file).
>>>
>>> Location of the hprof is usually where the program was launched from
>>> (check $HBASE_HOME dir).
>>>
>>> St.Ack
>>>
>>>
>>> On Wed, Jan 5, 2011 at 11:24 AM, Wayne <wa...@gmail.com> wrote:
>>> > Pretty sure this is compaction. The same node OOME again along with
>>> another
>>> > node after starting compaction. Like cass* .6 I guess hbase can not
>>> handle a
>>> > row bigger than it can hold in memory. I always read a lot about big
>>> cells
>>> > being a problem, but this problem is big rows.
>>> >
>>> > Thanks.
>>> >
>>> > On Wed, Jan 5, 2011 at 12:13 PM, Wayne <wa...@gmail.com> wrote:
>>> >
>>> >> It was carrying ~9k writes/sec and has been for the last 24+ hours.
>>> There
>>> >> are 500+ regions on that node. I could not find the heap dump
>>> (location?)
>>> >> but we do have some errant big rows that have crashed before. When we
>>> query
>>> >> those big rows thrift has been crashing. Maybe major compaction kicked
>>> in
>>> >> for those rows (see last log entry below)? There are 30 million columns
>>> with
>>> >> all small cell values but the 30 million is definitely too much.
>>> >>
>>> >> Here are some errors from the hadoop log. It looks like it kept getting
>>> >> stuck on something which may point to the data being too big? The error
>>> >> below occured 12 times in a row.
>>> >>
>>> >> org.apache.hadoop.ipc.RemoteException: java.io.IOException:
>>> >> blk_2176268114489978801_654636 is already commited, storedBlock == null.
>>> >>
>>> >> Here is the entry from the HBase log.
>>> >>
>>> >> 15:26:44,946 WARN org.apache.hadoop.hdfs.DFSClient: DataStreamer
>>> Exception:
>>> >> java.io.IOException: Broken pipe
>>> >> 15:26:44,946 FATAL org.apache.hadoop.hbase.regionserver.HRegionServer:
>>> >> Aborting region server serverName=sacdnb08.dataraker.net
>>> ,60020,1294089592450,
>>> >> load=(requests=0, regions=552, usedHeap=7977, maxHeap=7987): Uncaught
>>> >> exception in service thread regionserver60020.compactor
>>> >> java.lang.OutOfMemoryError: Java heap space
>>> >>
>>> >> Thanks.
>>> >>
>>> >>
>>> >> On Wed, Jan 5, 2011 at 11:45 AM, Stack <st...@duboce.net> wrote:
>>> >>
>>> >>> What was the server carrying?  How many regions?  What kinda of
>>> >>> loading was on the cluster?  We should not be OOME'ing.  Do you have
>>> >>> the heap dump lying around (We dump heap on OOME... its named *.hprof
>>> >>> or something.  If you have it, want to put it somewhere for me to pull
>>> >>> it so I can take a look?).  Any chance of a errant big cells?  Lots of
>>> >>> them?  What JVM version?
>>> >>>
>>> >>> St.Ack
>>> >>>
>>> >>> On Wed, Jan 5, 2011 at 8:10 AM, Wayne <wa...@gmail.com> wrote:
>>> >>> > I am still struggling with the JVM. We just had a hard OOM crash of a
>>> >>> region
>>> >>> > server after only running for 36 hours. Any help would be greatly
>>> >>> > appreciated. Do we need to restart nodes every 24 hours under load?
>>>  GC
>>> >>> > Pauses are something we are trying to plan for, but full out OOM
>>> crashes
>>> >>> are
>>> >>> > a new problem.
>>> >>> >
>>> >>> > The message below seems to be where it starts going bad. It is
>>> followed
>>> >>> by
>>> >>> > no less than 63 Concurrent Mode Failure errors over a 16 minute
>>> period.
>>> >>> >
>>> >>> > *GC locker: Trying a full collection because scavenge failed*
>>> >>> >
>>> >>> > Lastly here is the end (after the 63 CMF errors).
>>> >>> >
>>> >>> > Heap
>>> >>> >  par new generation   total 1887488K, used 303212K
>>> [0x00000005fae00000,
>>> >>> > 0x000000067ae00000, 0x000000067ae00000)
>>> >>> >  eden space 1677824K,  18% used [0x00000005fae00000,
>>> 0x000000060d61b078,
>>> >>> > 0x0000000661480000)
>>> >>> >  from space 209664K,   0% used [0x000000066e140000,
>>> 0x000000066e140000,
>>> >>> > 0x000000067ae00000)
>>> >>> >  to   space 209664K,   0% used [0x0000000661480000,
>>> 0x0000000661480000,
>>> >>> > 0x000000066e140000)
>>> >>> >  concurrent mark-sweep generation total 6291456K, used 2440155K
>>> >>> > [0x000000067ae00000, 0x00000007fae00000, 0x00000007fae00000)
>>> >>> >  concurrent-mark-sweep perm gen total 31704K, used 18999K
>>> >>> > [0x00000007fae00000, 0x00000007fccf6000, 0x0000000800000000)
>>> >>> >
>>> >>> > Here again are our custom settings in case there are some suggestions
>>> >>> out
>>> >>> > there. Are we making it worse with these settings? What should we try
>>> >>> next?
>>> >>> >
>>> >>> >        -XX:+UseCMSInitiatingOccupancyOnly
>>> >>> >        -XX:CMSInitiatingOccupancyFraction=60
>>> >>> >        -XX:+CMSParallelRemarkEnabled
>>> >>> >        -XX:SurvivorRatio=8
>>> >>> >        -XX:NewRatio=3
>>> >>> >        -XX:MaxTenuringThreshold=1
>>> >>> >
>>> >>> >
>>> >>> > Thanks!
>>> >>> >
>>> >>>
>>> >>
>>> >>
>>> >
>>>
>>
>

Re: JVM OOM

Posted by Stack <st...@duboce.net>.
k.  Thanks for stack trace.  Its unrelated to compactions but that
don't mean compaction not responsible.

I'm filing an issue for wide rows causing OOME (HBASE-3421).  Maybe
its not the case but we should check it out at least because I know of
others who have schemas where the row width can grow unbounded.

Thanks,
St.Ack


On Wed, Jan 5, 2011 at 1:08 PM, Wayne <wa...@gmail.com> wrote:
> Assuming it was big rows I dropped the table and have started loading it
> again (I could not share the data anyway). We will see it again eventually
> if it is not the large rows. We have changed our data model to be "taller"
> so the 30 million column row is now gone for good.
>
> Below is the contents of the .out file from the first OOM. I found the hprof
> files, but I have no public place to put them. At 10GB I can not even get
> them to my laptop to look at either.
>
> Thanks.
>
> [Unloading class sun.reflect.GeneratedSerializationConstructorAccessor2]
> [Unloading class sun.reflect.GeneratedSerializationConstructorAccessor3]
> [Unloading class sun.reflect.GeneratedSerializationConstructorAccessor1]
> [Unloading class sun.reflect.GeneratedConstructorAccessor10]
> [Unloading class sun.reflect.GeneratedConstructorAccessor21]
> [Unloading class sun.reflect.GeneratedConstructorAccessor23]
> [Unloading class sun.reflect.GeneratedMethodAccessor17]
> [Unloading class sun.reflect.GeneratedMethodAccessor1]
> [Unloading class sun.reflect.GeneratedMethodAccessor19]
> [Unloading class sun.reflect.GeneratedMethodAccessor18]
> [Unloading class sun.reflect.GeneratedMethodAccessor16]
> java.lang.OutOfMemoryError: Java heap space
> Dumping heap to java_pid17874.hprof ...
> Exception in thread "pool-1-thread-6" java.lang.OutOfMemoryError: Java
> heap space
>        at org.apache.hadoop.hbase.client.Put.readFields(Put.java:500)
> Heap dump file created [9689854890 bytes in 151.877 secs]
>        at org.apache.hadoop.hbase.client.MultiPut.readFields(MultiPut.java:111)
>        at org.apache.hadoop.hbase.io.HbaseObjectWritable.readObject(HbaseObjectWritable.java:461)
>        at org.apache.hadoop.hbase.ipc.HBaseRPC$Invocation.readFields(HBaseRPC.java:124)
>        at org.apache.hadoop.hbase.ipc.HBaseServer$Connection.processData(HBaseServer.java:959)
>        at org.apache.hadoop.hbase.ipc.HBaseServer$Connection.readAndProcess(HBaseServer.java:927)
>        at org.apache.hadoop.hbase.ipc.HBaseServer$Listener.doRead(HBaseServer.java:503)
>        at org.apache.hadoop.hbase.ipc.HBaseServer$Listener$Reader.run(HBaseServer.java:297)
>        at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown Source)
>        at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
>        at java.lang.Thread.run(Unknown Source)
> Exception in thread "ResponseProcessor for block
> blk_2176268114489978801_654636" java.lang.OutOfMemoryError: Java heap
> space
>        at java.util.HashMap.newKeyIterator(Unknown Source)
>        at java.util.HashMap$KeySet.iterator(Unknown Source)
>        at java.util.HashSet.iterator(Unknown Source)
>        at sun.nio.ch.SelectorImpl.processDeregisterQueue(Unknown Source)
>        at sun.nio.ch.EPollSelectorImpl.doSelect(Unknown Source)
>        at sun.nio.ch.SelectorImpl.lockAndDoSelect(Unknown Source)
>        at sun.nio.ch.SelectorImpl.select(Unknown Source)
>        at org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.select(SocketIOWithTimeout.java:332)
>        at org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:157)
>        at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:155)
>        at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:128)
>        at java.io.DataInputStream.readFully(Unknown Source)
>        at java.io.DataInputStream.readLong(Unknown Source)
>        at org.apache.hadoop.hdfs.protocol.DataTransferProtocol$PipelineAck.readFields(DataTransferProtocol.java:122)
>        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:2547)
> Exception in thread "pool-1-thread-7" java.lang.OutOfMemoryError: Java
> heap space
>        at org.apache.hadoop.hbase.client.Put.readFields(Put.java:495)
>        at org.apache.hadoop.hbase.client.MultiPut.readFields(MultiPut.java:111)
>        at org.apache.hadoop.hbase.io.HbaseObjectWritable.readObject(HbaseObjectWritable.java:461)
>        at org.apache.hadoop.hbase.ipc.HBaseRPC$Invocation.readFields(HBaseRPC.java:124)
>        at org.apache.hadoop.hbase.ipc.HBaseServer$Connection.processData(HBaseServer.java:959)
>        at org.apache.hadoop.hbase.ipc.HBaseServer$Connection.readAndProcess(HBaseServer.java:927)
>        at org.apache.hadoop.hbase.ipc.HBaseServer$Listener.doRead(HBaseServer.java:503)
>        at org.apache.hadoop.hbase.ipc.HBaseServer$Listener$Reader.run(HBaseServer.java:297)
>        at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown Source)
>        at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
>        at java.lang.Thread.run(Unknown Source)
> Exception in thread "pool-1-thread-9" java.lang.OutOfMemoryError: Java
> heap space
>
>
>
> On Wed, Jan 5, 2011 at 3:36 PM, Stack <st...@duboce.net> wrote:
>
>> That could be it.  Do you know names of participating files?  The
>> should be printed in the log.  Can you save aside that Store, tar it
>> up and put it somewhere we can pull (if data is ok for others to see)?
>>  If we are OOME'ing on big rows, thats something I know some fellas
>> for sure would be interested in fixing.
>>
>> (There is actually a 'limit' on number of kvs to get at a time when
>> scanning unused by the compaction code -- perhaps thats all it would
>> take to fix this issue?)
>>
>> Its highly unlikely but I'll ask anyways, was the OOME in same
>> location both times?  (OOME stack trace may be in the .out file rather
>> than the .log file).
>>
>> Location of the hprof is usually where the program was launched from
>> (check $HBASE_HOME dir).
>>
>> St.Ack
>>
>>
>> On Wed, Jan 5, 2011 at 11:24 AM, Wayne <wa...@gmail.com> wrote:
>> > Pretty sure this is compaction. The same node OOME again along with
>> another
>> > node after starting compaction. Like cass* .6 I guess hbase can not
>> handle a
>> > row bigger than it can hold in memory. I always read a lot about big
>> cells
>> > being a problem, but this problem is big rows.
>> >
>> > Thanks.
>> >
>> > On Wed, Jan 5, 2011 at 12:13 PM, Wayne <wa...@gmail.com> wrote:
>> >
>> >> It was carrying ~9k writes/sec and has been for the last 24+ hours.
>> There
>> >> are 500+ regions on that node. I could not find the heap dump
>> (location?)
>> >> but we do have some errant big rows that have crashed before. When we
>> query
>> >> those big rows thrift has been crashing. Maybe major compaction kicked
>> in
>> >> for those rows (see last log entry below)? There are 30 million columns
>> with
>> >> all small cell values but the 30 million is definitely too much.
>> >>
>> >> Here are some errors from the hadoop log. It looks like it kept getting
>> >> stuck on something which may point to the data being too big? The error
>> >> below occured 12 times in a row.
>> >>
>> >> org.apache.hadoop.ipc.RemoteException: java.io.IOException:
>> >> blk_2176268114489978801_654636 is already commited, storedBlock == null.
>> >>
>> >> Here is the entry from the HBase log.
>> >>
>> >> 15:26:44,946 WARN org.apache.hadoop.hdfs.DFSClient: DataStreamer
>> Exception:
>> >> java.io.IOException: Broken pipe
>> >> 15:26:44,946 FATAL org.apache.hadoop.hbase.regionserver.HRegionServer:
>> >> Aborting region server serverName=sacdnb08.dataraker.net
>> ,60020,1294089592450,
>> >> load=(requests=0, regions=552, usedHeap=7977, maxHeap=7987): Uncaught
>> >> exception in service thread regionserver60020.compactor
>> >> java.lang.OutOfMemoryError: Java heap space
>> >>
>> >> Thanks.
>> >>
>> >>
>> >> On Wed, Jan 5, 2011 at 11:45 AM, Stack <st...@duboce.net> wrote:
>> >>
>> >>> What was the server carrying?  How many regions?  What kinda of
>> >>> loading was on the cluster?  We should not be OOME'ing.  Do you have
>> >>> the heap dump lying around (We dump heap on OOME... its named *.hprof
>> >>> or something.  If you have it, want to put it somewhere for me to pull
>> >>> it so I can take a look?).  Any chance of a errant big cells?  Lots of
>> >>> them?  What JVM version?
>> >>>
>> >>> St.Ack
>> >>>
>> >>> On Wed, Jan 5, 2011 at 8:10 AM, Wayne <wa...@gmail.com> wrote:
>> >>> > I am still struggling with the JVM. We just had a hard OOM crash of a
>> >>> region
>> >>> > server after only running for 36 hours. Any help would be greatly
>> >>> > appreciated. Do we need to restart nodes every 24 hours under load?
>>  GC
>> >>> > Pauses are something we are trying to plan for, but full out OOM
>> crashes
>> >>> are
>> >>> > a new problem.
>> >>> >
>> >>> > The message below seems to be where it starts going bad. It is
>> followed
>> >>> by
>> >>> > no less than 63 Concurrent Mode Failure errors over a 16 minute
>> period.
>> >>> >
>> >>> > *GC locker: Trying a full collection because scavenge failed*
>> >>> >
>> >>> > Lastly here is the end (after the 63 CMF errors).
>> >>> >
>> >>> > Heap
>> >>> >  par new generation   total 1887488K, used 303212K
>> [0x00000005fae00000,
>> >>> > 0x000000067ae00000, 0x000000067ae00000)
>> >>> >  eden space 1677824K,  18% used [0x00000005fae00000,
>> 0x000000060d61b078,
>> >>> > 0x0000000661480000)
>> >>> >  from space 209664K,   0% used [0x000000066e140000,
>> 0x000000066e140000,
>> >>> > 0x000000067ae00000)
>> >>> >  to   space 209664K,   0% used [0x0000000661480000,
>> 0x0000000661480000,
>> >>> > 0x000000066e140000)
>> >>> >  concurrent mark-sweep generation total 6291456K, used 2440155K
>> >>> > [0x000000067ae00000, 0x00000007fae00000, 0x00000007fae00000)
>> >>> >  concurrent-mark-sweep perm gen total 31704K, used 18999K
>> >>> > [0x00000007fae00000, 0x00000007fccf6000, 0x0000000800000000)
>> >>> >
>> >>> > Here again are our custom settings in case there are some suggestions
>> >>> out
>> >>> > there. Are we making it worse with these settings? What should we try
>> >>> next?
>> >>> >
>> >>> >        -XX:+UseCMSInitiatingOccupancyOnly
>> >>> >        -XX:CMSInitiatingOccupancyFraction=60
>> >>> >        -XX:+CMSParallelRemarkEnabled
>> >>> >        -XX:SurvivorRatio=8
>> >>> >        -XX:NewRatio=3
>> >>> >        -XX:MaxTenuringThreshold=1
>> >>> >
>> >>> >
>> >>> > Thanks!
>> >>> >
>> >>>
>> >>
>> >>
>> >
>>
>

Re: JVM OOM

Posted by Wayne <wa...@gmail.com>.
Assuming it was big rows I dropped the table and have started loading it
again (I could not share the data anyway). We will see it again eventually
if it is not the large rows. We have changed our data model to be "taller"
so the 30 million column row is now gone for good.

Below is the contents of the .out file from the first OOM. I found the hprof
files, but I have no public place to put them. At 10GB I can not even get
them to my laptop to look at either.

Thanks.

[Unloading class sun.reflect.GeneratedSerializationConstructorAccessor2]
[Unloading class sun.reflect.GeneratedSerializationConstructorAccessor3]
[Unloading class sun.reflect.GeneratedSerializationConstructorAccessor1]
[Unloading class sun.reflect.GeneratedConstructorAccessor10]
[Unloading class sun.reflect.GeneratedConstructorAccessor21]
[Unloading class sun.reflect.GeneratedConstructorAccessor23]
[Unloading class sun.reflect.GeneratedMethodAccessor17]
[Unloading class sun.reflect.GeneratedMethodAccessor1]
[Unloading class sun.reflect.GeneratedMethodAccessor19]
[Unloading class sun.reflect.GeneratedMethodAccessor18]
[Unloading class sun.reflect.GeneratedMethodAccessor16]
java.lang.OutOfMemoryError: Java heap space
Dumping heap to java_pid17874.hprof ...
Exception in thread "pool-1-thread-6" java.lang.OutOfMemoryError: Java
heap space
	at org.apache.hadoop.hbase.client.Put.readFields(Put.java:500)
Heap dump file created [9689854890 bytes in 151.877 secs]
	at org.apache.hadoop.hbase.client.MultiPut.readFields(MultiPut.java:111)
	at org.apache.hadoop.hbase.io.HbaseObjectWritable.readObject(HbaseObjectWritable.java:461)
	at org.apache.hadoop.hbase.ipc.HBaseRPC$Invocation.readFields(HBaseRPC.java:124)
	at org.apache.hadoop.hbase.ipc.HBaseServer$Connection.processData(HBaseServer.java:959)
	at org.apache.hadoop.hbase.ipc.HBaseServer$Connection.readAndProcess(HBaseServer.java:927)
	at org.apache.hadoop.hbase.ipc.HBaseServer$Listener.doRead(HBaseServer.java:503)
	at org.apache.hadoop.hbase.ipc.HBaseServer$Listener$Reader.run(HBaseServer.java:297)
	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown Source)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
	at java.lang.Thread.run(Unknown Source)
Exception in thread "ResponseProcessor for block
blk_2176268114489978801_654636" java.lang.OutOfMemoryError: Java heap
space
	at java.util.HashMap.newKeyIterator(Unknown Source)
	at java.util.HashMap$KeySet.iterator(Unknown Source)
	at java.util.HashSet.iterator(Unknown Source)
	at sun.nio.ch.SelectorImpl.processDeregisterQueue(Unknown Source)
	at sun.nio.ch.EPollSelectorImpl.doSelect(Unknown Source)
	at sun.nio.ch.SelectorImpl.lockAndDoSelect(Unknown Source)
	at sun.nio.ch.SelectorImpl.select(Unknown Source)
	at org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.select(SocketIOWithTimeout.java:332)
	at org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:157)
	at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:155)
	at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:128)
	at java.io.DataInputStream.readFully(Unknown Source)
	at java.io.DataInputStream.readLong(Unknown Source)
	at org.apache.hadoop.hdfs.protocol.DataTransferProtocol$PipelineAck.readFields(DataTransferProtocol.java:122)
	at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:2547)
Exception in thread "pool-1-thread-7" java.lang.OutOfMemoryError: Java
heap space
	at org.apache.hadoop.hbase.client.Put.readFields(Put.java:495)
	at org.apache.hadoop.hbase.client.MultiPut.readFields(MultiPut.java:111)
	at org.apache.hadoop.hbase.io.HbaseObjectWritable.readObject(HbaseObjectWritable.java:461)
	at org.apache.hadoop.hbase.ipc.HBaseRPC$Invocation.readFields(HBaseRPC.java:124)
	at org.apache.hadoop.hbase.ipc.HBaseServer$Connection.processData(HBaseServer.java:959)
	at org.apache.hadoop.hbase.ipc.HBaseServer$Connection.readAndProcess(HBaseServer.java:927)
	at org.apache.hadoop.hbase.ipc.HBaseServer$Listener.doRead(HBaseServer.java:503)
	at org.apache.hadoop.hbase.ipc.HBaseServer$Listener$Reader.run(HBaseServer.java:297)
	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown Source)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
	at java.lang.Thread.run(Unknown Source)
Exception in thread "pool-1-thread-9" java.lang.OutOfMemoryError: Java
heap space



On Wed, Jan 5, 2011 at 3:36 PM, Stack <st...@duboce.net> wrote:

> That could be it.  Do you know names of participating files?  The
> should be printed in the log.  Can you save aside that Store, tar it
> up and put it somewhere we can pull (if data is ok for others to see)?
>  If we are OOME'ing on big rows, thats something I know some fellas
> for sure would be interested in fixing.
>
> (There is actually a 'limit' on number of kvs to get at a time when
> scanning unused by the compaction code -- perhaps thats all it would
> take to fix this issue?)
>
> Its highly unlikely but I'll ask anyways, was the OOME in same
> location both times?  (OOME stack trace may be in the .out file rather
> than the .log file).
>
> Location of the hprof is usually where the program was launched from
> (check $HBASE_HOME dir).
>
> St.Ack
>
>
> On Wed, Jan 5, 2011 at 11:24 AM, Wayne <wa...@gmail.com> wrote:
> > Pretty sure this is compaction. The same node OOME again along with
> another
> > node after starting compaction. Like cass* .6 I guess hbase can not
> handle a
> > row bigger than it can hold in memory. I always read a lot about big
> cells
> > being a problem, but this problem is big rows.
> >
> > Thanks.
> >
> > On Wed, Jan 5, 2011 at 12:13 PM, Wayne <wa...@gmail.com> wrote:
> >
> >> It was carrying ~9k writes/sec and has been for the last 24+ hours.
> There
> >> are 500+ regions on that node. I could not find the heap dump
> (location?)
> >> but we do have some errant big rows that have crashed before. When we
> query
> >> those big rows thrift has been crashing. Maybe major compaction kicked
> in
> >> for those rows (see last log entry below)? There are 30 million columns
> with
> >> all small cell values but the 30 million is definitely too much.
> >>
> >> Here are some errors from the hadoop log. It looks like it kept getting
> >> stuck on something which may point to the data being too big? The error
> >> below occured 12 times in a row.
> >>
> >> org.apache.hadoop.ipc.RemoteException: java.io.IOException:
> >> blk_2176268114489978801_654636 is already commited, storedBlock == null.
> >>
> >> Here is the entry from the HBase log.
> >>
> >> 15:26:44,946 WARN org.apache.hadoop.hdfs.DFSClient: DataStreamer
> Exception:
> >> java.io.IOException: Broken pipe
> >> 15:26:44,946 FATAL org.apache.hadoop.hbase.regionserver.HRegionServer:
> >> Aborting region server serverName=sacdnb08.dataraker.net
> ,60020,1294089592450,
> >> load=(requests=0, regions=552, usedHeap=7977, maxHeap=7987): Uncaught
> >> exception in service thread regionserver60020.compactor
> >> java.lang.OutOfMemoryError: Java heap space
> >>
> >> Thanks.
> >>
> >>
> >> On Wed, Jan 5, 2011 at 11:45 AM, Stack <st...@duboce.net> wrote:
> >>
> >>> What was the server carrying?  How many regions?  What kinda of
> >>> loading was on the cluster?  We should not be OOME'ing.  Do you have
> >>> the heap dump lying around (We dump heap on OOME... its named *.hprof
> >>> or something.  If you have it, want to put it somewhere for me to pull
> >>> it so I can take a look?).  Any chance of a errant big cells?  Lots of
> >>> them?  What JVM version?
> >>>
> >>> St.Ack
> >>>
> >>> On Wed, Jan 5, 2011 at 8:10 AM, Wayne <wa...@gmail.com> wrote:
> >>> > I am still struggling with the JVM. We just had a hard OOM crash of a
> >>> region
> >>> > server after only running for 36 hours. Any help would be greatly
> >>> > appreciated. Do we need to restart nodes every 24 hours under load?
>  GC
> >>> > Pauses are something we are trying to plan for, but full out OOM
> crashes
> >>> are
> >>> > a new problem.
> >>> >
> >>> > The message below seems to be where it starts going bad. It is
> followed
> >>> by
> >>> > no less than 63 Concurrent Mode Failure errors over a 16 minute
> period.
> >>> >
> >>> > *GC locker: Trying a full collection because scavenge failed*
> >>> >
> >>> > Lastly here is the end (after the 63 CMF errors).
> >>> >
> >>> > Heap
> >>> >  par new generation   total 1887488K, used 303212K
> [0x00000005fae00000,
> >>> > 0x000000067ae00000, 0x000000067ae00000)
> >>> >  eden space 1677824K,  18% used [0x00000005fae00000,
> 0x000000060d61b078,
> >>> > 0x0000000661480000)
> >>> >  from space 209664K,   0% used [0x000000066e140000,
> 0x000000066e140000,
> >>> > 0x000000067ae00000)
> >>> >  to   space 209664K,   0% used [0x0000000661480000,
> 0x0000000661480000,
> >>> > 0x000000066e140000)
> >>> >  concurrent mark-sweep generation total 6291456K, used 2440155K
> >>> > [0x000000067ae00000, 0x00000007fae00000, 0x00000007fae00000)
> >>> >  concurrent-mark-sweep perm gen total 31704K, used 18999K
> >>> > [0x00000007fae00000, 0x00000007fccf6000, 0x0000000800000000)
> >>> >
> >>> > Here again are our custom settings in case there are some suggestions
> >>> out
> >>> > there. Are we making it worse with these settings? What should we try
> >>> next?
> >>> >
> >>> >        -XX:+UseCMSInitiatingOccupancyOnly
> >>> >        -XX:CMSInitiatingOccupancyFraction=60
> >>> >        -XX:+CMSParallelRemarkEnabled
> >>> >        -XX:SurvivorRatio=8
> >>> >        -XX:NewRatio=3
> >>> >        -XX:MaxTenuringThreshold=1
> >>> >
> >>> >
> >>> > Thanks!
> >>> >
> >>>
> >>
> >>
> >
>

Re: JVM OOM

Posted by Stack <st...@duboce.net>.
That could be it.  Do you know names of participating files?  The
should be printed in the log.  Can you save aside that Store, tar it
up and put it somewhere we can pull (if data is ok for others to see)?
 If we are OOME'ing on big rows, thats something I know some fellas
for sure would be interested in fixing.

(There is actually a 'limit' on number of kvs to get at a time when
scanning unused by the compaction code -- perhaps thats all it would
take to fix this issue?)

Its highly unlikely but I'll ask anyways, was the OOME in same
location both times?  (OOME stack trace may be in the .out file rather
than the .log file).

Location of the hprof is usually where the program was launched from
(check $HBASE_HOME dir).

St.Ack


On Wed, Jan 5, 2011 at 11:24 AM, Wayne <wa...@gmail.com> wrote:
> Pretty sure this is compaction. The same node OOME again along with another
> node after starting compaction. Like cass* .6 I guess hbase can not handle a
> row bigger than it can hold in memory. I always read a lot about big cells
> being a problem, but this problem is big rows.
>
> Thanks.
>
> On Wed, Jan 5, 2011 at 12:13 PM, Wayne <wa...@gmail.com> wrote:
>
>> It was carrying ~9k writes/sec and has been for the last 24+ hours. There
>> are 500+ regions on that node. I could not find the heap dump (location?)
>> but we do have some errant big rows that have crashed before. When we query
>> those big rows thrift has been crashing. Maybe major compaction kicked in
>> for those rows (see last log entry below)? There are 30 million columns with
>> all small cell values but the 30 million is definitely too much.
>>
>> Here are some errors from the hadoop log. It looks like it kept getting
>> stuck on something which may point to the data being too big? The error
>> below occured 12 times in a row.
>>
>> org.apache.hadoop.ipc.RemoteException: java.io.IOException:
>> blk_2176268114489978801_654636 is already commited, storedBlock == null.
>>
>> Here is the entry from the HBase log.
>>
>> 15:26:44,946 WARN org.apache.hadoop.hdfs.DFSClient: DataStreamer Exception:
>> java.io.IOException: Broken pipe
>> 15:26:44,946 FATAL org.apache.hadoop.hbase.regionserver.HRegionServer:
>> Aborting region server serverName=sacdnb08.dataraker.net,60020,1294089592450,
>> load=(requests=0, regions=552, usedHeap=7977, maxHeap=7987): Uncaught
>> exception in service thread regionserver60020.compactor
>> java.lang.OutOfMemoryError: Java heap space
>>
>> Thanks.
>>
>>
>> On Wed, Jan 5, 2011 at 11:45 AM, Stack <st...@duboce.net> wrote:
>>
>>> What was the server carrying?  How many regions?  What kinda of
>>> loading was on the cluster?  We should not be OOME'ing.  Do you have
>>> the heap dump lying around (We dump heap on OOME... its named *.hprof
>>> or something.  If you have it, want to put it somewhere for me to pull
>>> it so I can take a look?).  Any chance of a errant big cells?  Lots of
>>> them?  What JVM version?
>>>
>>> St.Ack
>>>
>>> On Wed, Jan 5, 2011 at 8:10 AM, Wayne <wa...@gmail.com> wrote:
>>> > I am still struggling with the JVM. We just had a hard OOM crash of a
>>> region
>>> > server after only running for 36 hours. Any help would be greatly
>>> > appreciated. Do we need to restart nodes every 24 hours under load?  GC
>>> > Pauses are something we are trying to plan for, but full out OOM crashes
>>> are
>>> > a new problem.
>>> >
>>> > The message below seems to be where it starts going bad. It is followed
>>> by
>>> > no less than 63 Concurrent Mode Failure errors over a 16 minute period.
>>> >
>>> > *GC locker: Trying a full collection because scavenge failed*
>>> >
>>> > Lastly here is the end (after the 63 CMF errors).
>>> >
>>> > Heap
>>> >  par new generation   total 1887488K, used 303212K [0x00000005fae00000,
>>> > 0x000000067ae00000, 0x000000067ae00000)
>>> >  eden space 1677824K,  18% used [0x00000005fae00000, 0x000000060d61b078,
>>> > 0x0000000661480000)
>>> >  from space 209664K,   0% used [0x000000066e140000, 0x000000066e140000,
>>> > 0x000000067ae00000)
>>> >  to   space 209664K,   0% used [0x0000000661480000, 0x0000000661480000,
>>> > 0x000000066e140000)
>>> >  concurrent mark-sweep generation total 6291456K, used 2440155K
>>> > [0x000000067ae00000, 0x00000007fae00000, 0x00000007fae00000)
>>> >  concurrent-mark-sweep perm gen total 31704K, used 18999K
>>> > [0x00000007fae00000, 0x00000007fccf6000, 0x0000000800000000)
>>> >
>>> > Here again are our custom settings in case there are some suggestions
>>> out
>>> > there. Are we making it worse with these settings? What should we try
>>> next?
>>> >
>>> >        -XX:+UseCMSInitiatingOccupancyOnly
>>> >        -XX:CMSInitiatingOccupancyFraction=60
>>> >        -XX:+CMSParallelRemarkEnabled
>>> >        -XX:SurvivorRatio=8
>>> >        -XX:NewRatio=3
>>> >        -XX:MaxTenuringThreshold=1
>>> >
>>> >
>>> > Thanks!
>>> >
>>>
>>
>>
>

Re: JVM OOM

Posted by Wayne <wa...@gmail.com>.
Pretty sure this is compaction. The same node OOME again along with another
node after starting compaction. Like cass* .6 I guess hbase can not handle a
row bigger than it can hold in memory. I always read a lot about big cells
being a problem, but this problem is big rows.

Thanks.

On Wed, Jan 5, 2011 at 12:13 PM, Wayne <wa...@gmail.com> wrote:

> It was carrying ~9k writes/sec and has been for the last 24+ hours. There
> are 500+ regions on that node. I could not find the heap dump (location?)
> but we do have some errant big rows that have crashed before. When we query
> those big rows thrift has been crashing. Maybe major compaction kicked in
> for those rows (see last log entry below)? There are 30 million columns with
> all small cell values but the 30 million is definitely too much.
>
> Here are some errors from the hadoop log. It looks like it kept getting
> stuck on something which may point to the data being too big? The error
> below occured 12 times in a row.
>
> org.apache.hadoop.ipc.RemoteException: java.io.IOException:
> blk_2176268114489978801_654636 is already commited, storedBlock == null.
>
> Here is the entry from the HBase log.
>
> 15:26:44,946 WARN org.apache.hadoop.hdfs.DFSClient: DataStreamer Exception:
> java.io.IOException: Broken pipe
> 15:26:44,946 FATAL org.apache.hadoop.hbase.regionserver.HRegionServer:
> Aborting region server serverName=sacdnb08.dataraker.net,60020,1294089592450,
> load=(requests=0, regions=552, usedHeap=7977, maxHeap=7987): Uncaught
> exception in service thread regionserver60020.compactor
> java.lang.OutOfMemoryError: Java heap space
>
> Thanks.
>
>
> On Wed, Jan 5, 2011 at 11:45 AM, Stack <st...@duboce.net> wrote:
>
>> What was the server carrying?  How many regions?  What kinda of
>> loading was on the cluster?  We should not be OOME'ing.  Do you have
>> the heap dump lying around (We dump heap on OOME... its named *.hprof
>> or something.  If you have it, want to put it somewhere for me to pull
>> it so I can take a look?).  Any chance of a errant big cells?  Lots of
>> them?  What JVM version?
>>
>> St.Ack
>>
>> On Wed, Jan 5, 2011 at 8:10 AM, Wayne <wa...@gmail.com> wrote:
>> > I am still struggling with the JVM. We just had a hard OOM crash of a
>> region
>> > server after only running for 36 hours. Any help would be greatly
>> > appreciated. Do we need to restart nodes every 24 hours under load?  GC
>> > Pauses are something we are trying to plan for, but full out OOM crashes
>> are
>> > a new problem.
>> >
>> > The message below seems to be where it starts going bad. It is followed
>> by
>> > no less than 63 Concurrent Mode Failure errors over a 16 minute period.
>> >
>> > *GC locker: Trying a full collection because scavenge failed*
>> >
>> > Lastly here is the end (after the 63 CMF errors).
>> >
>> > Heap
>> >  par new generation   total 1887488K, used 303212K [0x00000005fae00000,
>> > 0x000000067ae00000, 0x000000067ae00000)
>> >  eden space 1677824K,  18% used [0x00000005fae00000, 0x000000060d61b078,
>> > 0x0000000661480000)
>> >  from space 209664K,   0% used [0x000000066e140000, 0x000000066e140000,
>> > 0x000000067ae00000)
>> >  to   space 209664K,   0% used [0x0000000661480000, 0x0000000661480000,
>> > 0x000000066e140000)
>> >  concurrent mark-sweep generation total 6291456K, used 2440155K
>> > [0x000000067ae00000, 0x00000007fae00000, 0x00000007fae00000)
>> >  concurrent-mark-sweep perm gen total 31704K, used 18999K
>> > [0x00000007fae00000, 0x00000007fccf6000, 0x0000000800000000)
>> >
>> > Here again are our custom settings in case there are some suggestions
>> out
>> > there. Are we making it worse with these settings? What should we try
>> next?
>> >
>> >        -XX:+UseCMSInitiatingOccupancyOnly
>> >        -XX:CMSInitiatingOccupancyFraction=60
>> >        -XX:+CMSParallelRemarkEnabled
>> >        -XX:SurvivorRatio=8
>> >        -XX:NewRatio=3
>> >        -XX:MaxTenuringThreshold=1
>> >
>> >
>> > Thanks!
>> >
>>
>
>

Re: JVM OOM

Posted by Wayne <wa...@gmail.com>.
It was carrying ~9k writes/sec and has been for the last 24+ hours. There
are 500+ regions on that node. I could not find the heap dump (location?)
but we do have some errant big rows that have crashed before. When we query
those big rows thrift has been crashing. Maybe major compaction kicked in
for those rows (see last log entry below)? There are 30 million columns with
all small cell values but the 30 million is definitely too much.

Here are some errors from the hadoop log. It looks like it kept getting
stuck on something which may point to the data being too big? The error
below occured 12 times in a row.

org.apache.hadoop.ipc.RemoteException: java.io.IOException:
blk_2176268114489978801_654636 is already commited, storedBlock == null.

Here is the entry from the HBase log.

15:26:44,946 WARN org.apache.hadoop.hdfs.DFSClient: DataStreamer Exception:
java.io.IOException: Broken pipe
15:26:44,946 FATAL org.apache.hadoop.hbase.regionserver.HRegionServer:
Aborting region server serverName=sacdnb08.dataraker.net,60020,1294089592450,
load=(requests=0, regions=552, usedHeap=7977, maxHeap=7987): Uncaught
exception in service thread regionserver60020.compactor
java.lang.OutOfMemoryError: Java heap space

Thanks.

On Wed, Jan 5, 2011 at 11:45 AM, Stack <st...@duboce.net> wrote:

> What was the server carrying?  How many regions?  What kinda of
> loading was on the cluster?  We should not be OOME'ing.  Do you have
> the heap dump lying around (We dump heap on OOME... its named *.hprof
> or something.  If you have it, want to put it somewhere for me to pull
> it so I can take a look?).  Any chance of a errant big cells?  Lots of
> them?  What JVM version?
>
> St.Ack
>
> On Wed, Jan 5, 2011 at 8:10 AM, Wayne <wa...@gmail.com> wrote:
> > I am still struggling with the JVM. We just had a hard OOM crash of a
> region
> > server after only running for 36 hours. Any help would be greatly
> > appreciated. Do we need to restart nodes every 24 hours under load?  GC
> > Pauses are something we are trying to plan for, but full out OOM crashes
> are
> > a new problem.
> >
> > The message below seems to be where it starts going bad. It is followed
> by
> > no less than 63 Concurrent Mode Failure errors over a 16 minute period.
> >
> > *GC locker: Trying a full collection because scavenge failed*
> >
> > Lastly here is the end (after the 63 CMF errors).
> >
> > Heap
> >  par new generation   total 1887488K, used 303212K [0x00000005fae00000,
> > 0x000000067ae00000, 0x000000067ae00000)
> >  eden space 1677824K,  18% used [0x00000005fae00000, 0x000000060d61b078,
> > 0x0000000661480000)
> >  from space 209664K,   0% used [0x000000066e140000, 0x000000066e140000,
> > 0x000000067ae00000)
> >  to   space 209664K,   0% used [0x0000000661480000, 0x0000000661480000,
> > 0x000000066e140000)
> >  concurrent mark-sweep generation total 6291456K, used 2440155K
> > [0x000000067ae00000, 0x00000007fae00000, 0x00000007fae00000)
> >  concurrent-mark-sweep perm gen total 31704K, used 18999K
> > [0x00000007fae00000, 0x00000007fccf6000, 0x0000000800000000)
> >
> > Here again are our custom settings in case there are some suggestions out
> > there. Are we making it worse with these settings? What should we try
> next?
> >
> >        -XX:+UseCMSInitiatingOccupancyOnly
> >        -XX:CMSInitiatingOccupancyFraction=60
> >        -XX:+CMSParallelRemarkEnabled
> >        -XX:SurvivorRatio=8
> >        -XX:NewRatio=3
> >        -XX:MaxTenuringThreshold=1
> >
> >
> > Thanks!
> >
>

Re: JVM OOM

Posted by Stack <st...@duboce.net>.
What was the server carrying?  How many regions?  What kinda of
loading was on the cluster?  We should not be OOME'ing.  Do you have
the heap dump lying around (We dump heap on OOME... its named *.hprof
or something.  If you have it, want to put it somewhere for me to pull
it so I can take a look?).  Any chance of a errant big cells?  Lots of
them?  What JVM version?

St.Ack

On Wed, Jan 5, 2011 at 8:10 AM, Wayne <wa...@gmail.com> wrote:
> I am still struggling with the JVM. We just had a hard OOM crash of a region
> server after only running for 36 hours. Any help would be greatly
> appreciated. Do we need to restart nodes every 24 hours under load?  GC
> Pauses are something we are trying to plan for, but full out OOM crashes are
> a new problem.
>
> The message below seems to be where it starts going bad. It is followed by
> no less than 63 Concurrent Mode Failure errors over a 16 minute period.
>
> *GC locker: Trying a full collection because scavenge failed*
>
> Lastly here is the end (after the 63 CMF errors).
>
> Heap
>  par new generation   total 1887488K, used 303212K [0x00000005fae00000,
> 0x000000067ae00000, 0x000000067ae00000)
>  eden space 1677824K,  18% used [0x00000005fae00000, 0x000000060d61b078,
> 0x0000000661480000)
>  from space 209664K,   0% used [0x000000066e140000, 0x000000066e140000,
> 0x000000067ae00000)
>  to   space 209664K,   0% used [0x0000000661480000, 0x0000000661480000,
> 0x000000066e140000)
>  concurrent mark-sweep generation total 6291456K, used 2440155K
> [0x000000067ae00000, 0x00000007fae00000, 0x00000007fae00000)
>  concurrent-mark-sweep perm gen total 31704K, used 18999K
> [0x00000007fae00000, 0x00000007fccf6000, 0x0000000800000000)
>
> Here again are our custom settings in case there are some suggestions out
> there. Are we making it worse with these settings? What should we try next?
>
>        -XX:+UseCMSInitiatingOccupancyOnly
>        -XX:CMSInitiatingOccupancyFraction=60
>        -XX:+CMSParallelRemarkEnabled
>        -XX:SurvivorRatio=8
>        -XX:NewRatio=3
>        -XX:MaxTenuringThreshold=1
>
>
> Thanks!
>