Stephan,

I ported the fix for the concurrency issue from the Flink commit so now
that should be fine. I ran some fail/restore tests and that specific issue
hasn't appeared again.

However I now get many segfaults in the initializeForJob method where the
RocksDb instance is opened. Just for the record this is the same exact code
as we have in Flink now.:

#
# A fatal error has been detected by the Java Runtime Environment:
#
#  SIGSEGV (0xb) at pc=0x00007f12b018f51f, pid=12576, tid=139668190197504
#
# JRE version: Java(TM) SE Runtime Environment (8.0_60-b27) (build
1.8.0_60-b27)
# Java VM: Java HotSpot(TM) 64-Bit Server VM (25.60-b23 mixed mode
linux-amd64 )
# Problematic frame:
# C  [libc.so.6+0x7b51f]
...
Stack: [0x00007f0708ccf000,0x00007f0708dd0000],  sp=0x00007f0708dccd20,
 free space=1015k
Native frames: (J=compiled Java code, j=interpreted, Vv=VM code, C=native
code)
C  [libc.so.6+0x7b51f]

Java frames: (J=compiled Java code, j=interpreted, Vv=VM code)
j
 
org.rocksdb.RocksDB.open(JLjava/lang/String;Ljava/util/List;I)Ljava/util/List;+0
j
 
org.rocksdb.RocksDB.open(Lorg/rocksdb/DBOptions;Ljava/lang/String;Ljava/util/List;Ljava/util/List;)Lorg/rocksdb/RocksDB;+23
j
 com.king.rbea.backend.state.rocksdb.RocksDBStateBackend.initializeForJob...

And this happens fairly frequently when the jobs are restarting after
failure.

Cheers,
Gyula

Gyula Fóra <gyula.f...@gmail.com> ezt írta (időpont: 2016. aug. 25., Cs,
19:07):

> Yes seems like that, I remember the fix in Flink. I apparently made a
> mistake somewhere in our code :)
>
> Thanks,
> Gyula
>
> On Thu, Aug 25, 2016, 18:59 Stephan Ewen <se...@apache.org> wrote:
>
>> We saw some crashes in earlier versions when native handles in RocksDB
>> (even for config option objects) were manually and too eagerly released.
>>
>> Maybe you have a similar issue here?
>>
>> On Thu, Aug 25, 2016 at 6:27 PM, Gyula Fóra <gyula.f...@gmail.com> wrote:
>>
>> > Hi,
>> > This seems to be a sneaky concurrency issue in our custom statebackend
>> > implementation.
>> >
>> > I made some changes, will keep you posted.
>> >
>> > Cheers,
>> > Gyula
>> >
>> > On Thu, Aug 25, 2016, 10:54 Gyula Fóra <gyula.f...@gmail.com> wrote:
>> >
>> > > Hi,
>> > >
>> > > Sure I am sending the TM logs in priv.
>> > >
>> > > Currently what I did was to bump the Rocks version to 4.9.0 let's see
>> if
>> > > that helps.
>> > >
>> > > Cheers,
>> > > Gyula
>> > >
>> > > Till Rohrmann <trohrm...@apache.org> ezt írta (időpont: 2016. aug.
>> 25.,
>> > > Cs, 10:35):
>> > >
>> > >> Hi Gyula,
>> > >>
>> > >> I haven't seen this problem before. Do you have the logs of the
>> failed
>> > TMs
>> > >> so that we have some more context what was going on?
>> > >>
>> > >> Cheers,
>> > >> Till
>> > >>
>> > >> On Thu, Aug 25, 2016 at 9:40 AM, Gyula Fóra <gyf...@apache.org>
>> wrote:
>> > >>
>> > >> > Hi guys,
>> > >> >
>> > >> > For quite some time now we fairly frequently experience a task
>> manager
>> > >> > crashes around the time new streaming jobs are deployed. We use
>> > RocksDB
>> > >> > backend so this might be related.
>> > >> >
>> > >> > We tried changing the GC from G1 to CMS that didnt help.
>> > >> >
>> > >> > Yesterday for instance 6 task managers crashed one ofter the other
>> > with
>> > >> > similar errors:
>> > >> >
>> > >> > *** Error in `java': double free or corruption (!prev):
>> > >> 0x00007fac0414d760
>> > >> > ***
>> > >> > *** Error in `java': free(): invalid pointer: 0x00007f8dcc0026c0
>> ***
>> > >> > *** Error in `java': double free or corruption (!prev):
>> > >> 0x00007f15247f9a90
>> > >> > ***
>> > >> > ...
>> > >> >
>> > >> > Does anyone have any clue what might cause this or how to debug?
>> > >> > This is very a critical issue :(
>> > >> >
>> > >> > Cheers,
>> > >> > Gyula
>> > >> >
>> > >>
>> > >
>> >
>>
>

Reply via email to