samza-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Tommy Becker <tobec...@tivo.com>
Subject SIGSEGV in RocksDB when killing jobs
Date Wed, 14 Sep 2016 14:56:35 GMT
While testing with Samza 0.10.1 I noticed the following crash whenever I would kill a job that
uses a RocksDB store:


# A fatal error has been detected by the Java Runtime Environment:
#
#  SIGSEGV (0xb) at pc=0x00007eff66b6c27e, pid=20315, tid=139636974364416
#
# JRE version: Java(TM) SE Runtime Environment (8.0_51-b16) (build 1.8.0_51-b16)
# Java VM: Java HotSpot(TM) 64-Bit Server VM (25.51-b03 mixed mode linux-amd64 compressed
oops)
# Problematic frame:
# C  [librocksdbjni2253915919401340417..so+0x11427e]  rocksdb_flush_helper(JNIEnv_*, rocksdb::DB*,
rocksdb::FlushOptions const&, rocksdb::ColumnFamilyHandle*)+0x1e
#
# Failed to write core dump. Core dumps have been disabled. To enable core dumping, try "ulimit
-c unlimited" before starting Java again
#
# An error report file with more information is saved as:
# /home/tommy/projects/ffs/ffs-stream-jobs/target/ffs-stream-jobs-8.1.4.0-SNAPSHOT-dist/ffs-stream-jobs/hs_err_pid20315.log
#
# 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.

I was able to tie this back to the RocksDB store being closed by both our StreamTask and the
SamzaContainer. We always close stores via CloseableTask#close just for housekeeping purposes.
Prior to this issue I was not aware that this also happens automatically in SamzaContainer#shutdownStores.
 When closed, KeyValueStorageEngine first calls flush() on the underlying store and there
is no guard to ensure that close has not already been called. The flush() call on a closed
DB is what seems to cause the crash. Obviously RocksDB should handle this more gracefully,
but I wonder if a patch is warranted for Samza also. Thoughts?

--
Tommy Becker
Senior Software Engineer

Digitalsmiths
A TiVo Company

www.digitalsmiths.com<http://www.digitalsmiths.com>
tobecker@tivo.com<mailto:tobecker@tivo.com>

________________________________

This email and any attachments may contain confidential and privileged material for the sole
use of the intended recipient. Any review, copying, or distribution of this email (or any
attachments) by others is prohibited. If you are not the intended recipient, please contact
the sender immediately and permanently delete this email and any attachments. No employee
or agent of TiVo Inc. is authorized to conclude any binding agreement on behalf of TiVo Inc.
by email. Binding agreements with TiVo Inc. may only be made by a signed written agreement.

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message