I forgot to mention that you need to run this with Flink’s version of RocksDB, 
as the stock version is already unable to perform the inserts because their 
implementation of merge operator has a performance problem.

Furthermore, I think a higher multiplicator than *2 is required on num (and/or 
a smaller modulo on the key bytes) to trigger the problem; Noticed that I ran 
it multiple times, so it added up to bigger sizes over the runs.

> Am 26.05.2017 um 18:42 schrieb Stefan Richter <s.rich...@data-artisans.com>:
> 
> I played a bit around with your info and this looks now like a general 
> problem in RocksDB to me. Or more specifically, between RocksDB and the JNI 
> bridge. I could reproduce the issue with the following simple test code:
> 
> File rocksDir = new File("/tmp/rocks");
> final Options options = new Options()
>    .setCreateIfMissing(true)
>    .setMergeOperator(new StringAppendOperator())
>    .setCompactionStyle(CompactionStyle.LEVEL)
>    .setLevelCompactionDynamicLevelBytes(true)
>    .setIncreaseParallelism(4)
>    .setUseFsync(false)
>    .setMaxOpenFiles(-1)
>    .setAllowOsBuffer(true)
>    .setDisableDataSync(true);
> 
> final WriteOptions write_options = new WriteOptions()
>    .setSync(false)
>    .setDisableWAL(true);
> 
> try (final RocksDB rocksDB = RocksDB.open(options, 
> rocksDir.getAbsolutePath())) {
>    final String key = "key";
>    final String value = 
> "abcdefghijklmnopqrstuvwxyz0123456789ABCDEFGHIJKLMNOPQRSTUVWXYZ7890654321";
> 
>    byte[] keyBytes = key.getBytes(StandardCharsets.UTF_8);
>    keyBytes = Arrays.copyOf(keyBytes, keyBytes.length + 1);
>    final byte[] valueBytes = value.getBytes(StandardCharsets.UTF_8);
>    final int num = (Integer.MAX_VALUE / valueBytes.length) * 2;
> 
>    System.out.println("begin insert");
> 
>    final long beginInsert = System.nanoTime();
>    for (int i = 0; i < num; i++) {
>       keyBytes[keyBytes.length - 1] = (byte) (i % 9);
>       rocksDB.merge(write_options, keyBytes, valueBytes);
>    }
>    final long endInsert = System.nanoTime();
>    System.out.println("end insert - duration: " + ((endInsert - beginInsert) 
> / 1_000_000) + " ms");
> 
>    final long beginGet = System.nanoTime();
>    try (RocksIterator iterator = rocksDB.newIterator()) {
>       iterator.seekToFirst();
> 
>       while (iterator.isValid()) {
>          iterator.next();
>          byte[] bytes = iterator.value();
>          System.out.println(bytes.length + " " + bytes[bytes.length - 1]);
>       }
>    }
>    final long endGet = System.nanoTime();
> 
>    System.out.println("end get - duration: " + ((endGet - beginGet) / 
> 1_000_000) + " ms");
> }
> 
> Depending on how smooth the 1.3 release is going, maybe I find some time next 
> week to take a closer look into this. If this is urgent, please also feel 
> free to already report this problem to the RocksDB issue tracker.
> 
> Best,
> Stefan
> 
>> Am 26.05.2017 um 16:40 schrieb Jason Brelloch <jb.bc....@gmail.com 
>> <mailto:jb.bc....@gmail.com>>:
>> 
>> ~2 GB was the total state in the backend.  The total number of keys in the 
>> test is 10 with an approximately even distribution of state across keys, and 
>> parallelism of 1 so all keys are on the same taskmanager.  We are using 
>> ListState and the number of elements per list would be about 500000.
>> 
>> On Fri, May 26, 2017 at 10:20 AM, Stefan Richter 
>> <s.rich...@data-artisans.com <mailto:s.rich...@data-artisans.com>> wrote:
>> 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 <rmetz...@apache.org 
>>> <mailto:rmetz...@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....@gmail.com 
>>> <mailto:jb.bc....@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
>>> 
>> 
>> 
>> 
>> 
>> -- 
>> 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
> 

Reply via email to