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

ASF GitHub Bot commented on FLINK-9263:
---------------------------------------

Github user StefanRRichter commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5930#discussion_r184708667
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredBroadcastBackendStateMetaInfo.java
 ---
    @@ -52,6 +52,23 @@ public RegisteredBroadcastBackendStateMetaInfo(
                this.valueSerializer = 
Preconditions.checkNotNull(valueSerializer);
        }
     
    +   public 
RegisteredBroadcastBackendStateMetaInfo(RegisteredBroadcastBackendStateMetaInfo 
copy) {
    +
    +           Preconditions.checkNotNull(copy);
    +
    +           this.name = copy.name;
    +           this.assignmentMode = copy.assignmentMode;
    +           this.keySerializer = copy.keySerializer.duplicate();
    +           this.valueSerializer = copy.valueSerializer.duplicate();
    +   }
    +
    +   /**
    +    * Creates a deep copy of the itself.
    +    */
    +   public RegisteredBroadcastBackendStateMetaInfo deepCopy() {
    --- End diff --
    
    Should not return raw type as well, but 
`RegisteredOperatorBackendStateMetaInfo<S>`.


> Kafka010ITCase failed on travis because of the concurrency problem in 
> DefaultOperateStateBackend
> ------------------------------------------------------------------------------------------------
>
>                 Key: FLINK-9263
>                 URL: https://issues.apache.org/jira/browse/FLINK-9263
>             Project: Flink
>          Issue Type: Bug
>          Components: Tests
>    Affects Versions: 1.5.0
>            Reporter: Sihua Zhou
>            Assignee: Sihua Zhou
>            Priority: Blocker
>             Fix For: 1.5.0
>
>
> instance: https://travis-ci.org/apache/flink/jobs/371952726
> I found this
> {code}
> Caused by: java.util.concurrent.ExecutionException: 
> java.lang.IllegalStateException: Concurrent access to KryoSerializer. Thread 
> 1: pool-26-thread-2 , Thread 2: pool-26-thread-1
>       at java.util.concurrent.FutureTask.report(FutureTask.java:122)
>       at java.util.concurrent.FutureTask.get(FutureTask.java:192)
>       at 
> org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:53)
>       at 
> org.apache.flink.streaming.api.operators.OperatorSnapshotFinalizer.<init>(OperatorSnapshotFinalizer.java:53)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:858)
>       ... 5 more
> Caused by: java.lang.IllegalStateException: Concurrent access to 
> KryoSerializer. Thread 1: pool-26-thread-2 , Thread 2: pool-26-thread-1
>       at 
> org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer.enterExclusiveThread(KryoSerializer.java:622)
>       at 
> org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer.serialize(KryoSerializer.java:254)
>       at 
> org.apache.flink.api.java.typeutils.runtime.TupleSerializer.serialize(TupleSerializer.java:125)
>       at 
> org.apache.flink.api.java.typeutils.runtime.TupleSerializer.serialize(TupleSerializer.java:30)
>       at 
> org.apache.flink.runtime.state.DefaultOperatorStateBackend$PartitionableListState.write(DefaultOperatorStateBackend.java:687)
>       at 
> org.apache.flink.runtime.state.DefaultOperatorStateBackend$1.performOperation(DefaultOperatorStateBackend.java:423)
>       at 
> org.apache.flink.runtime.state.DefaultOperatorStateBackend$1.performOperation(DefaultOperatorStateBackend.java:352)
>       at 
> org.apache.flink.runtime.io.async.AbstractAsyncCallableWithResources.call(AbstractAsyncCallableWithResources.java:75)
>       at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>       at 
> org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:50)
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to