flink-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Stefan Richter <s.rich...@data-artisans.com>
Subject Re: Checkpointing SIGSEGV
Date Fri, 26 May 2017 14:20:03 GMT
Hi,

what means „our state“ in this context? The total state in the backend or the state under
one key? If you use, e.g. list state, I could see that the state for one key can grow above
2GB, but once we retrieve the state back from RocksDB as Java arrays (in your stacktrace,
when making a checkpoint), which are bounded in size to a maximum of 2GB (Integer.MAX_VALUE)
and maybe that is what happens in JNI if you try to go beyond that limit. Could that be a
reason for your problem?

> Am 26.05.2017 um 15:50 schrieb Robert Metzger <rmetzger@apache.org>:
> 
> Hi Jason,
> 
> This error is unexpected. I don't think its caused by insufficient memory. I'm including
Stefan into the conversation, he's the RocksDB expert :)
> 
> On Thu, May 25, 2017 at 4:15 PM, Jason Brelloch <jb.bc.flk@gmail.com <mailto:jb.bc.flk@gmail.com>>
wrote:
> Hey guys,
> 
> We are running into a JVM crash on checkpointing when our rocksDB state reaches a certain
size on a taskmanager (about 2GB).  The issue happens with both a hadoop backend and just
writing to a local file.
> 
> We are running on Flink 1.2.1.
> 
> #
> # A fatal error has been detected by the Java Runtime Environment:
> #
> #  SIGSEGV (0xb) at pc=0x00007febf4261b42, pid=1, tid=0x00007fead135f700
> #
> # JRE version: Java(TM) SE Runtime Environment (8.0_131-b11) (build 1.8.0_131-b11)
> # Java VM: Java HotSpot(TM) 64-Bit Server VM (25.131-b11 mixed mode linux-amd64 compressed
oops)
> # Problematic frame:
> # V  [libjvm.so+0x6d1b42]  jni_SetByteArrayRegion+0xc2
> #
> # Core dump written. Default location: //core or core.1
> #
> # An error report file with more information is saved as:
> # /tmp/hs_err_pid1.log
> #
> # If you would like to submit a bug report, please visit:
> #   http://bugreport.java.com/bugreport/crash.jsp <http://bugreport.java.com/bugreport/crash.jsp>
> #
> 
> Is this an issue with not enough memory?  Or maybe not enough allocated to rocksDB?
> 
> I have attached the taskmanager logs, and the core dump.  The jobmanager logs just say
taskmanger lost/killed.
> 
> Thanks!
> 
> -- 
> Jason Brelloch | Product Developer
> 3405 Piedmont Rd. NE, Suite 325, Atlanta, GA 30305 
>  <http://www.bettercloud.com/>
> Subscribe to the BetterCloud Monitor <https://www.bettercloud.com/monitor?utm_source=bettercloud_email&utm_medium=email_signature&utm_campaign=monitor_launch>
- Get IT delivered to your inbox
> 


Mime
View raw message