zentol commented on code in PR #21019: URL: https://github.com/apache/flink/pull/21019#discussion_r993203411
########## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java: ########## @@ -298,6 +303,27 @@ private JobGraph createJobGraph() { return jobGraph; } + private void waitForSerializationFuturesAndUpdateJobVertices() + throws ExecutionException, InterruptedException { + for (JobVertex jobVertex : jobGraph.getVertices()) { + final List<CompletableFuture<SerializedValue<OperatorCoordinator.Provider>>> + futuresForJobVertex = coordinatorSerializationFutures.remove(jobVertex.getID()); + if (futuresForJobVertex == null) { + LOG.warn( + "No OperatorCoordinator creator serialized for JobVertex {}.", + jobVertex.getID()); + } else { + FutureUtils.combineAll(futuresForJobVertex) + .get() + .forEach(jobVertex::addOperatorCoordinator); + } + } + Preconditions.checkState( + coordinatorSerializationFutures.isEmpty(), + "There are still serialization futures not processed for JobVertex instances: {}", Review Comment: This seems misleading. if this happens then some concurrent operation is going on. ########## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java: ########## @@ -298,6 +303,27 @@ private JobGraph createJobGraph() { return jobGraph; } + private void waitForSerializationFuturesAndUpdateJobVertices() + throws ExecutionException, InterruptedException { + for (JobVertex jobVertex : jobGraph.getVertices()) { + final List<CompletableFuture<SerializedValue<OperatorCoordinator.Provider>>> + futuresForJobVertex = coordinatorSerializationFutures.remove(jobVertex.getID()); + if (futuresForJobVertex == null) { + LOG.warn( + "No OperatorCoordinator creator serialized for JobVertex {}.", + jobVertex.getID()); Review Comment: a) This is an illegal state; throw an exception instead. b) Iterate over the entries in the map instead to avoid the issue. c) Consider not inserting into the map if nothing has to be serialized. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org