You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "Sophie Blee-Goldman (Jira)" <ji...@apache.org> on 2020/06/23 03:33:00 UTC

[jira] [Comment Edited] (KAFKA-4996) Fix findbugs multithreaded correctness warnings for streams

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

Sophie Blee-Goldman edited comment on KAFKA-4996 at 6/23/20, 3:32 AM:
----------------------------------------------------------------------

Took a quick look at removing the spotBugs rule exclusions, and we get failures on
 # InternalTopologyBuilder.applicationId  – this should be fine, as we set this once at the beginning before we start up any other threads
 # StreamsMetadataState.localMetadata  – I think this is actually not threadsafe, but should be? Seems like we synchronize on setting it, but expose it through a public API with no synchronization
 # CachingWindowStore.maxObservedTimestamp – this also seems potentially problematic. We only update it in the synchronized #put method, but it's used during #fetch methods which can be called from IQ threads

Anyways, we had this multithread warning in the excluded rules for pretty much every package in Streams, which is definitely "not good"


was (Author: ableegoldman):
Took a quick look at removing the spotBugs rule exclusions, and we get failures on
 # InternalTopologyBuilder.applicationId  – this should be fine, as we set this once at the beginning before we start up any other threads
 # StreamsMetadataState.localMetadata  – I think this is actually not threadsafe, but should be? Seems like we synchronize on setting it, but expose it through a public API with no synchronization
 # CachingWindowStore.maxObservedTimestamp – this also seems potentially problematic, we only update it in the synchronized #put method but it is used during #fetch methods which can be called from other IQ threads right?

Anyways, we had this multithread warning in the excluded rules for pretty much every package in Streams, which is definitely "not good"

> Fix findbugs multithreaded correctness warnings for streams
> -----------------------------------------------------------
>
>                 Key: KAFKA-4996
>                 URL: https://issues.apache.org/jira/browse/KAFKA-4996
>             Project: Kafka
>          Issue Type: Bug
>          Components: streams
>            Reporter: Colin McCabe
>            Priority: Major
>              Labels: newbie
>
> Fix findbugs multithreaded correctness warnings for streams
> {code}
> Multithreaded correctness Warnings                                                                                                                                                                                                                                                        
>                                                                                                                                                                                                                                                                                           
>    Code Warning                                                                                                                                                                                                                                                                           
>    AT   Sequence of calls to java.util.concurrent.ConcurrentHashMap may not be atomic in org.apache.kafka.streams.state.internals.Segments.getOrCreateSegment(long, ProcessorContext)                                                                                                     
>    IS   Inconsistent synchronization of org.apache.kafka.streams.KafkaStreams.stateListener; locked 66% of time                                                                                                                                                                           
>    IS   Inconsistent synchronization of org.apache.kafka.streams.processor.internals.StreamThread.stateListener; locked 66% of time                                                                                                                                                       
>    IS   Inconsistent synchronization of org.apache.kafka.streams.processor.TopologyBuilder.applicationId; locked 50% of time                                                                                                                                                              
>    IS   Inconsistent synchronization of org.apache.kafka.streams.state.internals.CachingKeyValueStore.context; locked 66% of time                                                                                                                                                         
>    IS   Inconsistent synchronization of org.apache.kafka.streams.state.internals.CachingWindowStore.cache; locked 60% of time                                                                                                                                                             
>    IS   Inconsistent synchronization of org.apache.kafka.streams.state.internals.CachingWindowStore.context; locked 66% of time                                                                                                                                                           
>    IS   Inconsistent synchronization of org.apache.kafka.streams.state.internals.CachingWindowStore.name; locked 60% of time                                                                                                                                                              
>    IS   Inconsistent synchronization of org.apache.kafka.streams.state.internals.CachingWindowStore.serdes; locked 70% of time                                                                                                                                                            
>    IS   Inconsistent synchronization of org.apache.kafka.streams.state.internals.RocksDBStore.db; locked 63% of time                                                                                                                                                                      
>    IS   Inconsistent synchronization of org.apache.kafka.streams.state.internals.RocksDBStore.serdes; locked 76% of time                                                                                                                                                                  
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)