You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@flink.apache.org by Jason Brelloch <jb...@gmail.com> on 2017/05/25 14:15:13 UTC

Checkpointing SIGSEGV

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
#

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

Re: Checkpointing SIGSEGV

Posted by Stefan Richter <s....@data-artisans.com>.
FYI, I created this JIRA https://issues.apache.org/jira/browse/FLINK-6761 <https://issues.apache.org/jira/browse/FLINK-6761> to track the problem of large merging state per key. I might also bring this the the RocksDB issue tracker and then figure out how to solve this.

> Am 27.05.2017 um 20:28 schrieb Stefan Richter <s....@data-artisans.com>:
> 
> Hi,
> 
> this is a known and currently „accepted“ problem in Flink which can only happen when a task manager is already going down, e.g. on cancelation. It happens when the RocksDB object was already disposed (as part of the shutdown procedure) but there is still a pending timer firing, and in the process accessing the released native resource. 
> 
> Background why it is like that: waiting for all timer to finish could potentially take time and we want shutdown to be as fast as possible so that we can bring up the task again asap.
> 
> Best,
> Stefan
> 
>> Am 27.05.2017 um 16:19 schrieb Yassine MARZOUGUI <y.marzougui@mindlytix.com <ma...@mindlytix.com>>:
>> 
>> Hi,
>> 
>> This might be related, I'm experiencing a similar SIGSEGV causing the taskmanager to die each time my streaming job fails or even if I cancel it manually.
>> I am using Flink1.4-SNAPSHOT, Commit: 546e2ad.
>> 
>> Here are some examples (see the full dump attached):
>> 
>> On Job failure:
>> #
>> # A fatal error has been detected by the Java Runtime Environment:
>> #
>> #  SIGSEGV (0xb) at pc=0x00007f244f4cf5b3, pid=9532, tid=0x00007f24474fc700
>> #
>> # JRE version: OpenJDK Runtime Environment (8.0_121-b13) (build 1.8.0_121-b13)
>> # Java VM: OpenJDK 64-Bit Server VM (25.121-b13 mixed mode linux-amd64 compressed oops)
>> # Problematic frame:
>> # C  [librocksdbjni-linux64.so+0x1dd5b3]  rocksdb::DBImpl::GetImpl(rocksdb::ReadOptions const&, rocksdb::ColumnFamilyHandle*, rocksdb::Slice const&, std::string*, bool*)+0xe3
>> #
>> # Failed to write core dump. Core dumps have been disabled. To enable core dumping, try "ulimit -c unlimited" before starting Java again
>> #
>> # 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>
>> # The crash happened outside the Java Virtual Machine in native code.
>> # See problematic frame for where to report the bug.
>> #
>> 
>> On job Cancel:
>> #
>> # A fatal error has been detected by the Java Runtime Environment:
>> #
>> #  SIGSEGV (0xb) at pc=0x00007fdd2815c280, pid=4503, tid=0x00007fdc14d48700
>> #
>> # JRE version: OpenJDK Runtime Environment (8.0_121-b13) (build 1.8.0_121-b13)
>> # Java VM: OpenJDK 64-Bit Server VM (25.121-b13 mixed mode linux-amd64 compressed oops)
>> # Problematic frame:
>> # C  0x00007fdd2815c280
>> #
>> # Failed to write core dump. Core dumps have been disabled. To enable core dumping, try "ulimit -c unlimited" before starting Java again
>> #
>> # 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>
>> # The crash happened outside the Java Virtual Machine in native code.
>> # See problematic frame for where to report the bug.
>> #
>> 
>> Side question : Isn't the jobmanager supposed to restart the taskmnanager if it is lost/killed?
>> 
>> Best,
>> Yassine
>> 
>> 
>> 2017-05-27 1:32 GMT+02:00 Stefan Richter <s.richter@data-artisans.com <ma...@data-artisans.com>>:
>> Flink’s version is hosted here: https://github.com/dataArtisans/frocksdb <https://github.com/dataArtisans/frocksdb>
>> 
>>> Am 26.05.2017 um 19:59 schrieb Jason Brelloch <jb.bc.flk@gmail.com <ma...@gmail.com>>:
>>> 
>>> Thanks for looking into this Stefan.  We are moving forward with a different strategy for now.  If I want to take a look at this, where do I go to get the Flink version of RocksDB?
>>> 
>>> On Fri, May 26, 2017 at 1:06 PM, Stefan Richter <s.richter@data-artisans.com <ma...@data-artisans.com>> wrote:
>>> 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.richter@data-artisans.com <ma...@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.flk@gmail.com <ma...@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.richter@data-artisans.com <ma...@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 <rmetzger@apache.org <ma...@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 <ma...@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
>>>> 
>>> 
>>> 
>>> 
>>> 
>>> -- 
>>> 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
>> 
>> 
>> <hs_err_pid4503.log><hs_err_pid9532.log>
> 


Re: Checkpointing SIGSEGV

Posted by Stefan Richter <s....@data-artisans.com>.
Hi,

this is a known and currently „accepted“ problem in Flink which can only happen when a task manager is already going down, e.g. on cancelation. It happens when the RocksDB object was already disposed (as part of the shutdown procedure) but there is still a pending timer firing, and in the process accessing the released native resource. 

Background why it is like that: waiting for all timer to finish could potentially take time and we want shutdown to be as fast as possible so that we can bring up the task again asap.

Best,
Stefan

> Am 27.05.2017 um 16:19 schrieb Yassine MARZOUGUI <y....@mindlytix.com>:
> 
> Hi,
> 
> This might be related, I'm experiencing a similar SIGSEGV causing the taskmanager to die each time my streaming job fails or even if I cancel it manually.
> I am using Flink1.4-SNAPSHOT, Commit: 546e2ad.
> 
> Here are some examples (see the full dump attached):
> 
> On Job failure:
> #
> # A fatal error has been detected by the Java Runtime Environment:
> #
> #  SIGSEGV (0xb) at pc=0x00007f244f4cf5b3, pid=9532, tid=0x00007f24474fc700
> #
> # JRE version: OpenJDK Runtime Environment (8.0_121-b13) (build 1.8.0_121-b13)
> # Java VM: OpenJDK 64-Bit Server VM (25.121-b13 mixed mode linux-amd64 compressed oops)
> # Problematic frame:
> # C  [librocksdbjni-linux64.so+0x1dd5b3]  rocksdb::DBImpl::GetImpl(rocksdb::ReadOptions const&, rocksdb::ColumnFamilyHandle*, rocksdb::Slice const&, std::string*, bool*)+0xe3
> #
> # Failed to write core dump. Core dumps have been disabled. To enable core dumping, try "ulimit -c unlimited" before starting Java again
> #
> # 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>
> # The crash happened outside the Java Virtual Machine in native code.
> # See problematic frame for where to report the bug.
> #
> 
> On job Cancel:
> #
> # A fatal error has been detected by the Java Runtime Environment:
> #
> #  SIGSEGV (0xb) at pc=0x00007fdd2815c280, pid=4503, tid=0x00007fdc14d48700
> #
> # JRE version: OpenJDK Runtime Environment (8.0_121-b13) (build 1.8.0_121-b13)
> # Java VM: OpenJDK 64-Bit Server VM (25.121-b13 mixed mode linux-amd64 compressed oops)
> # Problematic frame:
> # C  0x00007fdd2815c280
> #
> # Failed to write core dump. Core dumps have been disabled. To enable core dumping, try "ulimit -c unlimited" before starting Java again
> #
> # 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>
> # The crash happened outside the Java Virtual Machine in native code.
> # See problematic frame for where to report the bug.
> #
> 
> Side question : Isn't the jobmanager supposed to restart the taskmnanager if it is lost/killed?
> 
> Best,
> Yassine
> 
> 
> 2017-05-27 1:32 GMT+02:00 Stefan Richter <s.richter@data-artisans.com <ma...@data-artisans.com>>:
> Flink’s version is hosted here: https://github.com/dataArtisans/frocksdb <https://github.com/dataArtisans/frocksdb>
> 
>> Am 26.05.2017 um 19:59 schrieb Jason Brelloch <jb.bc.flk@gmail.com <ma...@gmail.com>>:
>> 
>> Thanks for looking into this Stefan.  We are moving forward with a different strategy for now.  If I want to take a look at this, where do I go to get the Flink version of RocksDB?
>> 
>> On Fri, May 26, 2017 at 1:06 PM, Stefan Richter <s.richter@data-artisans.com <ma...@data-artisans.com>> wrote:
>> 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.richter@data-artisans.com <ma...@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.flk@gmail.com <ma...@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.richter@data-artisans.com <ma...@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 <rmetzger@apache.org <ma...@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 <ma...@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
>>> 
>> 
>> 
>> 
>> 
>> -- 
>> 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
> 
> 
> <hs_err_pid4503.log><hs_err_pid9532.log>


Re: Checkpointing SIGSEGV

Posted by Yassine MARZOUGUI <y....@mindlytix.com>.
Hi,

This might be related, I'm experiencing a similar SIGSEGV causing the
taskmanager to die each time my streaming job fails or even if I cancel it
manually.
I am using Flink1.4-SNAPSHOT, Commit: 546e2ad.

Here are some examples (see the full dump attached):

On Job failure:
#
# A fatal error has been detected by the Java Runtime Environment:
#
#  SIGSEGV (0xb) at pc=0x00007f244f4cf5b3, pid=9532, tid=0x00007f24474fc700
#
# JRE version: OpenJDK Runtime Environment (8.0_121-b13) (build
1.8.0_121-b13)
# Java VM: OpenJDK 64-Bit Server VM (25.121-b13 mixed mode linux-amd64
compressed oops)
# Problematic frame:
# C  [librocksdbjni-linux64.so+0x1dd5b3]
 rocksdb::DBImpl::GetImpl(rocksdb::ReadOptions const&,
rocksdb::ColumnFamilyHandle*, rocksdb::Slice const&, std::string*,
bool*)+0xe3
#
# Failed to write core dump. Core dumps have been disabled. To enable core
dumping, try "ulimit -c unlimited" before starting Java again
#
# If you would like to submit a bug report, please visit:
#   http://bugreport.java.com/bugreport/crash.jsp
# The crash happened outside the Java Virtual Machine in native code.
# See problematic frame for where to report the bug.
#

On job Cancel:
#
# A fatal error has been detected by the Java Runtime Environment:
#
#  SIGSEGV (0xb) at pc=0x00007fdd2815c280, pid=4503, tid=0x00007fdc14d48700
#
# JRE version: OpenJDK Runtime Environment (8.0_121-b13) (build
1.8.0_121-b13)
# Java VM: OpenJDK 64-Bit Server VM (25.121-b13 mixed mode linux-amd64
compressed oops)
# Problematic frame:
# C  0x00007fdd2815c280
#
# Failed to write core dump. Core dumps have been disabled. To enable core
dumping, try "ulimit -c unlimited" before starting Java again
#
# If you would like to submit a bug report, please visit:
#   http://bugreport.java.com/bugreport/crash.jsp
# The crash happened outside the Java Virtual Machine in native code.
# See problematic frame for where to report the bug.
#

Side question : Isn't the jobmanager supposed to restart the
taskmnanager if it is lost/killed?

Best,
Yassine


2017-05-27 1:32 GMT+02:00 Stefan Richter <s....@data-artisans.com>:

> Flink’s version is hosted here: https://github.com/dataArtisans/frocksdb
>
> Am 26.05.2017 um 19:59 schrieb Jason Brelloch <jb...@gmail.com>:
>
> Thanks for looking into this Stefan.  We are moving forward with a
> different strategy for now.  If I want to take a look at this, where do I
> go to get the Flink version of RocksDB?
>
> On Fri, May 26, 2017 at 1:06 PM, Stefan Richter <s.richter@data-
> artisans.com> wrote:
>
>> 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.richter@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...@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.richter@data-artisa
>> ns.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 <rm...@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...@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
>>>> #
>>>>
>>>> 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
>>
>>
>>
>>
>
>
> --
> *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
>
>
>

Re: Checkpointing SIGSEGV

Posted by Stefan Richter <s....@data-artisans.com>.
Flink’s version is hosted here: https://github.com/dataArtisans/frocksdb <https://github.com/dataArtisans/frocksdb>

> Am 26.05.2017 um 19:59 schrieb Jason Brelloch <jb...@gmail.com>:
> 
> Thanks for looking into this Stefan.  We are moving forward with a different strategy for now.  If I want to take a look at this, where do I go to get the Flink version of RocksDB?
> 
> On Fri, May 26, 2017 at 1:06 PM, Stefan Richter <s.richter@data-artisans.com <ma...@data-artisans.com>> wrote:
> 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.richter@data-artisans.com <ma...@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.flk@gmail.com <ma...@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.richter@data-artisans.com <ma...@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 <rmetzger@apache.org <ma...@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 <ma...@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
>> 
> 
> 
> 
> 
> -- 
> 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


Re: Checkpointing SIGSEGV

Posted by Jason Brelloch <jb...@gmail.com>.
Thanks for looking into this Stefan.  We are moving forward with a
different strategy for now.  If I want to take a look at this, where do I
go to get the Flink version of RocksDB?

On Fri, May 26, 2017 at 1:06 PM, Stefan Richter <s.richter@data-artisans.com
> wrote:

> 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.richter@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...@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.richter@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 <rm...@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...@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
>>> #
>>>
>>> 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
>
>
>
>


-- 
*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

Re: Checkpointing SIGSEGV

Posted by Stefan Richter <s....@data-artisans.com>.
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....@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.flk@gmail.com <ma...@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.richter@data-artisans.com <ma...@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 <rmetzger@apache.org <ma...@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 <ma...@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
> 


Re: Checkpointing SIGSEGV

Posted by Stefan Richter <s....@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...@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.richter@data-artisans.com <ma...@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 <rmetzger@apache.org <ma...@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 <ma...@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


Re: Checkpointing SIGSEGV

Posted by Jason Brelloch <jb...@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.richter@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 <rm...@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...@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
>> #
>>
>> 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

Re: Checkpointing SIGSEGV

Posted by Stefan Richter <s....@data-artisans.com>.
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 <rm...@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 <ma...@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
> 


Re: Checkpointing SIGSEGV

Posted by Robert Metzger <rm...@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...@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
> #
>
> 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
>