[ 
https://issues.apache.org/jira/browse/KAFKA-3805?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15321500#comment-15321500
 ] 

Guozhang Wang commented on KAFKA-3805:
--------------------------------------

Thanks for reporting this [~agomez].

[~enothereska] Since different tasks have their own sub-directories under the 
application-id directory, I think ideally users should be able to use the same 
{{state.dir}}. Originally we use {{maybeCleanup}} instead of cleaning up the 
state directory right away during rebalance is to optimize the case where 
migrated-out tasks were migrated back shortly after (think of rolling bounce 
the instances for upgrade), and we only delete the task directory ONLY IF the 
thread can successfully grab the file lock. So as long as the other Kafka 
Streams instance's threads hold the file locks, they should not be deleted by 
other threads. Is there anything I missed?


> Running multiple instances of a Streams app on the same machine results in 
> Java_org_rocksdb_RocksDB_write error
> ---------------------------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-3805
>                 URL: https://issues.apache.org/jira/browse/KAFKA-3805
>             Project: Kafka
>          Issue Type: Bug
>          Components: streams
>    Affects Versions: 0.10.0.0
>         Environment: * Ubuntu Server 16.04 LTS
>             - Java: OpenJDK Runtime Environment (build 
> 1.8.0_91-8u91-b14-0ubuntu4~16.04.1-b14)
> * OSX 10.11.5
>            - Java: Java(TM) SE Runtime Environment (build 1.8.0_74-b02)
>            Reporter: Andres Gomez Ferrer
>            Assignee: Eno Thereska
>            Priority: Critical
>         Attachments: hs_err_pid23047.log
>
>
> I reproduce the error working with simple two instances of  my kafka streams 
> application reading from one topic writing on other and using a RocksDB store.
> I reproduce it starting one instance, sending some messages on the input 
> topic and later starting another Kafka streams instance. Then, the first 
> instance died and drop a core dump.
> ------------------------------------------------------------------------------------------------------------------
>  A fatal error has been detected by the Java Runtime Environment:
>   SIGSEGV (0xb) at pc=0x00007f9f488fd8e8, pid=23047, tid=140322171361024
>  JRE version: OpenJDK Runtime Environment (8.0_91-b14) (build 
> 1.8.0_91-8u91-b14-0ubuntu4~16.04.1-b14)
>  Java VM: OpenJDK 64-Bit Server VM (25.91-b14 mixed mode linux-amd64 
> compressed oops)
>  Problematic frame:
>  C  [librocksdbjni712288404493406713..so+0x15b8e8]  
> Java_org_rocksdb_RocksDB_write0+0x48
>  Core dump written. Default location: /root/core or core.23047
>  An error report file with more information is saved as:
>  /root/hs_err_pid23047.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.
> ------------------------------------------------------------------------------------------------------------------
> The core dump is there: 
> https://s3-eu-west-1.amazonaws.com/arodriguezg-test/core
> Note: The core dump is collected at Ubuntu Server.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to