Uploaded image for project: 'Kafka'
  1. Kafka
  2. KAFKA-13684

KStream rebalance can lead to JVM process crash when network issues occure

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Critical
    • Resolution: Not A Bug
    • 2.8.1
    • None
    • streams
    • None

    Description

      Hello,

      Sporadically KStream rebalance leads to segmentation fault

      siginfo: si_signo: 11 (SIGSEGV), si_code: 128 (SI_KERNEL), si_addr: 0x0000000000000000 

      I have spotted it occuring when:

      1) there some intermittent connection issues. I have found org.apache.kafka.common.errors.DisconnectException:  in logs during rebalance

      2) a lot of partitions are shifted due to ks cluster re-balance

       

      crash stack:

      Current thread (0x00007f5bf407a000):  JavaThread "app-blue-v6-StreamThread-2" [_thread_in_native, id=231, stack(0x00007f5bdc2ed000,0x00007f5bdc3ee000)]
      Stack: [0x00007f5bdc2ed000,0x00007f5bdc3ee000],  sp=0x00007f5bdc3ebe30,  free space=1019kNative frames: (J=compiled Java code, A=aot compiled Java code, j=interpreted, Vv=VM code, C=native code)C  [libc.so.6+0x37ab7]  abort+0x297
      
      Java frames: (J=compiled Java code, j=interpreted, Vv=VM code)J 8080  org.rocksdb.WriteBatch.put(J[BI[BIJ)V (0 bytes) @ 0x00007f5c857ca520 [0x00007f5c857ca4a0+0x0000000000000080]J 8835 c2 
      
      org.apache.kafka.streams.state.internals.RocksDBStore$SingleColumnFamilyAccessor.prepareBatchForRestore(Ljava/util/Collection;Lorg/rocksdb/WriteBatch;)V (52 bytes) @ 0x00007f5c858dccb4 [0x00007f5c858dcb60+0x0000000000000154]J 9779 c1 
      org.apache.kafka.streams.state.internals.RocksDBStore$RocksDBBatchingRestoreCallback.restoreAll(Ljava/util/Collection;)V (147 bytes) @ 0x00007f5c7ef7b7e4 [0x00007f5c7ef7b360+0x0000000000000484]J 8857 c2 
      org.apache.kafka.streams.processor.internals.StateRestoreCallbackAdapter.lambda$adapt$0(Lorg/apache/kafka/streams/processor/StateRestoreCallback;Ljava/util/Collection;)V (73 bytes) @ 0x00007f5c858f86dc [0x00007f5c858f8500+0x00000000000001dc]J 9686 c1 
      org.apache.kafka.streams.processor.internals.StateRestoreCallbackAdapter$$Lambda$937.restoreBatch(Ljava/util/Collection;)V (9 bytes) @ 0x00007f5c7dff7bb4 [0x00007f5c7dff7b40+0x0000000000000074]J 9683 c1 
      org.apache.kafka.streams.processor.internals.ProcessorStateManager.restore(Lorg/apache/kafka/streams/processor/internals/ProcessorStateManager$StateStoreMetadata;Ljava/util/List;)V (176 bytes) @ 0x00007f5c7e71af4c [0x00007f5c7e719740+0x000000000000180c]J 8882 c2 
      org.apache.kafka.streams.processor.internals.StoreChangelogReader.restoreChangelog(Lorg/apache/kafka/streams/processor/internals/StoreChangelogReader$ChangelogMetadata;)Z (334 bytes) @ 0x00007f5c859052ec [0x00007f5c85905140+0x00000000000001ac]J 12689 c2 
      org.apache.kafka.streams.processor.internals.StoreChangelogReader.restore(Ljava/util/Map;)V (412 bytes) @ 0x00007f5c85ce98d4 [0x00007f5c85ce8420+0x00000000000014b4]J 12688 c2 
      org.apache.kafka.streams.processor.internals.StreamThread.initializeAndRestorePhase()V (214 bytes) @ 0x00007f5c85ce580c [0x00007f5c85ce5540+0x00000000000002cc]J 17654 c2 
      org.apache.kafka.streams.processor.internals.StreamThread.runOnce()V (725 bytes) @ 0x00007f5c859960e8 [0x00007f5c85995fa0+0x0000000000000148]j  
      org.apache.kafka.streams.processor.internals.StreamThread.runLoop()Z+61j
      org.apache.kafka.streams.processor.internals.StreamThread.run()V+36v  
      
      ~StubRoutines::call_stub 
      
      siginfo: si_signo: 11 (SIGSEGV), si_code: 128 (SI_KERNEL), si_addr: 0x0000000000000000

      I attached whole java cash-dump and digest from our logs. 

      It is executed on azul jdk11

      KS 2.8.1

       

      Attachments

        1. crash-logs.csv
          212 kB
          Peter Cipov
        2. crash-dump.log
          843 kB
          Peter Cipov

        Activity

          People

            Unassigned Unassigned
            petercipov Peter Cipov
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: