[
https://issues.apache.org/jira/browse/SAMZA-1464?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16211751#comment-16211751
]
ASF GitHub Bot commented on SAMZA-1464:
---------------------------------------
GitHub user prateekm opened a pull request:
https://github.com/apache/samza/pull/334
SAMZA-1464: Flushing a closed RocksDB store causes SIGSEGVs
Made RocksDB operations check if DB is still open to avoid segfaults.
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/prateekm/samza segfault-fix
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/samza/pull/334.patch
To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:
This closes #334
----
commit 56d8aea2ca1df5143d5744783b7dd13032b03c21
Author: Prateek Maheshwari <[email protected]>
Date: 2017-10-19T21:08:53Z
Added concurrent write perf test for KVStore.
commit a504e4b6675ef17a45312d5757c60a55f7fb883e
Author: Prateek Maheshwari <[email protected]>
Date: 2017-10-19T21:12:08Z
Made RocksDB operations check if DB is still open to avoid segfaults.
----
> Flushing a closed RocksDB store causes SIGSEGVs
> -----------------------------------------------
>
> Key: SAMZA-1464
> URL: https://issues.apache.org/jira/browse/SAMZA-1464
> Project: Samza
> Issue Type: Bug
> Reporter: Prateek Maheshwari
> Assignee: Prateek Maheshwari
>
> Flushing a closed RocksDB store causes JVM to exit with a fatal error
> (SIGSEGV):
> {code}
> #
> # A fatal error has been detected by the Java Runtime Environment:
> #
> # SIGSEGV (0xb) at pc=0x0000000000000000, pid=136092, tid=0x00007f5879ad2700
> #
> # JRE version: Java(TM) SE Runtime Environment (8.0_121-b13) (build
> 1.8.0_121-b13)
> # Java VM: Java HotSpot(TM) 64-Bit Server VM (25.121-b13 mixed mode
> linux-amd64 compressed oops)
> # Problematic frame:
> # C 0x0000000000000000
> #
> # Failed to write core dump. Core dumps have been disabled. To enable core
> dumping, try "ulimit -c unlimited" before starting Java again
> ....
> Stack: [0x00007f58799d2000,0x00007f5879ad3000], sp=0x00007f5879ad0ba8, free
> space=1018k
> Java frames: (J=compiled Java code, j=interpreted, Vv=VM code)
> j org.rocksdb.RocksDB.flush(JJ)V+0
> j org.rocksdb.RocksDB.flush(Lorg/rocksdb/FlushOptions;)V+13
> j org.apache.samza.storage.kv.RocksDbKeyValueStore.flush()V+35
> j org.apache.samza.storage.kv.LoggedStore.flush()V+31
> j org.apache.samza.storage.kv.SerializedKeyValueStore.flush()V+31
> j org.apache.samza.storage.kv.CachedStore.flush()V+47
> j org.apache.samza.storage.kv.NullSafeKeyValueStore.flush()V+8
> j
> org.apache.samza.storage.kv.KeyValueStorageEngine$$anonfun$flush$1.apply$mcV$sp()V+40
> j
> org.apache.samza.storage.kv.KeyValueStorageEngine$$anonfun$flush$1.apply()V+5
> j
> org.apache.samza.storage.kv.KeyValueStorageEngine$$anonfun$flush$1.apply()Ljava/lang/Object;+5
> J 17610 C2
> org.apache.samza.util.TimerUtils$class.updateTimer(Lorg/apache/samza/util/TimerUtils;Lorg/apache/samza/metrics/Timer;Lscala/Function0;)Ljava/lang/Object;
> (50 bytes) @ 0x00007f59463cf8d4 [0x00007f59463cf820+0xb4]
> j
> org.apache.samza.storage.kv.KeyValueStorageEngine.updateTimer(Lorg/apache/samza/metrics/Timer;Lscala/Function0;)Ljava/lang/Object;+7
> j org.apache.samza.storage.kv.KeyValueStorageEngine.flush()V+20
> j org.apache.samza.storage.kv.KeyValueStorageEngine.close()V+17
> j org.apache.samza.storage.kv.KeyValueStorageEngine.stop()V+17
> j
> org.apache.samza.storage.TaskStorageManager$$anonfun$stopStores$2.apply(Lorg/apache/samza/storage/StorageEngine;)V+5
> j
> org.apache.samza.storage.TaskStorageManager$$anonfun$stopStores$2.apply(Ljava/lang/Object;)Ljava/lang/Object;+9
> J 7042 C2 scala.collection.AbstractIterator.foreach(Lscala/Function1;)V (15
> bytes) @ 0x00007f5945706220 [0x00007f59457061a0+0x80]
> J 10641 C1
> scala.collection.MapLike$DefaultValuesIterable.foreach(Lscala/Function1;)V
> (24 bytes) @ 0x00007f5945f90854 [0x00007f5945f90580+0x2d4]
> j org.apache.samza.storage.TaskStorageManager.stopStores()V+33
> j org.apache.samza.storage.TaskStorageManager.stop()V+5
> j org.apache.samza.container.TaskInstance.shutdownStores()V+47
> j
> org.apache.samza.container.SamzaContainer$$anonfun$shutdownStores$2.apply(Lorg/apache/samza/container/TaskInstance;)V+5
> j
> org.apache.samza.container.SamzaContainer$$anonfun$shutdownStores$2.apply(Ljava/lang/Object;)Ljava/lang/Object;+9
> J 7042 C2 scala.collection.AbstractIterator.foreach(Lscala/Function1;)V (15
> bytes) @ 0x00007f5945706220 [0x00007f59457061a0+0x80]
> J 10641 C1
> scala.collection.MapLike$DefaultValuesIterable.foreach(Lscala/Function1;)V
> (24 bytes) @ 0x00007f5945f90854 [0x00007f5945f90580+0x2d4]
> j org.apache.samza.container.SamzaContainer.shutdownStores()V+33
> j org.apache.samza.container.SamzaContainer.run()V+281
> j org.apache.samza.job.local.ThreadJob$$anon$1.run()V+11
> v ~StubRoutines::call_stub
> {code}
> This can happen if we close the same store twice. E.g., closing the
> TimelineStore in WindowOperatorImpl and then closing the underlying RocksDB
> store during shutdown:
> {code}
> org.apache.samza.storage.kv.RocksDbKeyValueStore.close(RocksDbKeyValueStore.scala:201)
> at
> org.apache.samza.storage.kv.LoggedStore.close(LoggedStore.scala:114)
> at
> org.apache.samza.storage.kv.SerializedKeyValueStore.close(SerializedKeyValueStore.scala:124)
> at
> org.apache.samza.storage.kv.CachedStore.close(CachedStore.scala:241)
> at
> org.apache.samza.storage.kv.NullSafeKeyValueStore.close(NullSafeKeyValueStore.scala:86)
> at
> org.apache.samza.storage.kv.KeyValueStorageEngine.close(KeyValueStorageEngine.scala:151)
> at
> org.apache.samza.operators.impl.store.TimeSeriesStoreImpl.close(TimeSeriesStoreImpl.java:167)
> at
> org.apache.samza.operators.impl.WindowOperatorImpl.handleClose(WindowOperatorImpl.java:202)
> at
> org.apache.samza.operators.impl.OperatorImpl.close(OperatorImpl.java:380)
> at java.lang.Iterable.forEach(Iterable.java:75)
> at
> org.apache.samza.operators.impl.OperatorImplGraph.close(OperatorImplGraph.java:134)
> at
> org.apache.samza.task.StreamOperatorTask.close(StreamOperatorTask.java:143)
> at
> org.apache.samza.task.AsyncStreamTaskAdapter.close(AsyncStreamTaskAdapter.java:89)
> at
> org.apache.samza.container.TaskInstance.shutdownTask(TaskInstance.scala:212)
> at
> org.apache.samza.container.SamzaContainer$$anonfun$shutdownTask$5.apply(SamzaContainer.scala:997)
> at
> org.apache.samza.container.SamzaContainer$$anonfun$shutdownTask$5.apply(SamzaContainer.scala:997)
> at scala.collection.Iterator$class.foreach(Iterator.scala:893)
> at scala.collection.AbstractIterator.foreach(Iterator.scala:1336)
> at
> scala.collection.MapLike$DefaultValuesIterable.foreach(MapLike.scala:206)
> at
> org.apache.samza.container.SamzaContainer.shutdownTask(SamzaContainer.scala:997)
> at
> org.apache.samza.container.SamzaContainer.run(SamzaContainer.scala:747)
> at
> org.apache.samza.job.local.ThreadJob$$anon$1.run(ThreadJob.scala:38)
> {code}
> Previously also reported in
> https://www.mail-archive.com/[email protected]/msg04374.html.
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)