tillrohrmann commented on a change in pull request #11313: [FLINK-16373] Make 
JobManagerLeaderListener thread safe
URL: https://github.com/apache/flink/pull/11313#discussion_r393706944
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java
 ##########
 @@ -251,90 +254,83 @@ private JobManagerLeaderListener(JobID jobId) {
                }
 
                public void stop() {
-                       stopped = true;
+                       synchronized (lock) {
+                               stopped = true;
 
-                       if (rpcConnection != null) {
-                               rpcConnection.close();
+                               if (rpcConnection != null) {
+                                       rpcConnection.close();
+                               }
                        }
                }
 
                public void reconnect() {
-                       if (stopped) {
-                               LOG.debug("Cannot reconnect because the 
JobManagerLeaderListener has already been stopped.");
-                       } else {
-                               final RegisteredRpcConnection<JobMasterId, 
JobMasterGateway, JMTMRegistrationSuccess> currentRpcConnection = rpcConnection;
-
-                               if (currentRpcConnection != null) {
-                                       if (currentRpcConnection.isConnected()) 
{
-
-                                               if 
(currentRpcConnection.tryReconnect()) {
-                                                       // double check for 
concurrent stop operation
-                                                       if (stopped) {
-                                                               
currentRpcConnection.close();
-                                                       }
-                                               } else {
-                                                       LOG.debug("Could not 
reconnect to the JobMaster {}.", currentRpcConnection.getTargetAddress());
-                                               }
+                       synchronized (lock) {
+                               if (stopped) {
+                                       LOG.debug("Cannot reconnect because the 
JobManagerLeaderListener has already been stopped.");
+                               } else {
+                                       if (rpcConnection != null) {
+                                               Preconditions.checkState(
+                                                       
rpcConnection.tryReconnect(),
+                                                       "Illegal concurrent 
modification of the JobManagerLeaderListener rpc connection.");
                                        } else {
-                                               LOG.debug("Ongoing registration 
to JobMaster {}.", currentRpcConnection.getTargetAddress());
+                                               LOG.debug("Cannot reconnect to 
an unknown JobMaster.");
                                        }
-                               } else {
-                                       LOG.debug("Cannot reconnect to an 
unknown JobMaster.");
                                }
                        }
                }
 
                @Override
-               public void notifyLeaderAddress(final @Nullable String 
leaderAddress, final @Nullable UUID leaderId) {
-                       if (stopped) {
-                               LOG.debug("{}'s leader retrieval listener 
reported a new leader for job {}. " +
-                                       "However, the service is no longer 
running.", JobLeaderService.class.getSimpleName(), jobId);
-                       } else {
-                               final JobMasterId jobMasterId = 
JobMasterId.fromUuidOrNull(leaderId);
-
-                               LOG.debug("New leader information for job {}. 
Address: {}, leader id: {}.",
-                                       jobId, leaderAddress, jobMasterId);
-
-                               if (leaderAddress == null || 
leaderAddress.isEmpty()) {
-                                       // the leader lost leadership but there 
is no other leader yet.
-                                       if (rpcConnection != null) {
-                                               rpcConnection.close();
-                                       }
+               public void notifyLeaderAddress(@Nullable final String 
leaderAddress, @Nullable final UUID leaderId) {
+                       Optional<JobMasterId> jobManagerLostLeadership = 
Optional.empty();
 
-                                       
jobLeaderListener.jobManagerLostLeadership(jobId, currentJobMasterId);
-
-                                       currentJobMasterId = jobMasterId;
+                       synchronized (lock) {
+                               if (stopped) {
+                                       LOG.debug("{}'s leader retrieval 
listener reported a new leader for job {}. " +
+                                               "However, the service is no 
longer running.", JobLeaderService.class.getSimpleName(), jobId);
                                } else {
-                                       currentJobMasterId = jobMasterId;
+                                       final JobMasterId jobMasterId = 
JobMasterId.fromUuidOrNull(leaderId);
 
-                                       if (rpcConnection != null) {
-                                               // check if we are already 
trying to connect to this leader
-                                               if 
(!Objects.equals(jobMasterId, rpcConnection.getTargetLeaderId())) {
+                                       LOG.debug("New leader information for 
job {}. Address: {}, leader id: {}.",
+                                               jobId, leaderAddress, 
jobMasterId);
+
+                                       if (leaderAddress == null || 
leaderAddress.isEmpty()) {
+                                               // the leader lost leadership 
but there is no other leader yet.
+                                               if (rpcConnection != null) {
                                                        rpcConnection.close();
+                                               }
 
+                                               jobManagerLostLeadership = 
Optional.of(currentJobMasterId);
+                                               currentJobMasterId = 
jobMasterId;
+                                       } else {
+                                               currentJobMasterId = 
jobMasterId;
+
+                                               if (rpcConnection != null) {
+                                                       // check if we are 
already trying to connect to this leader
+                                                       if 
(!Objects.equals(jobMasterId, rpcConnection.getTargetLeaderId())) {
+                                                               
rpcConnection.close();
+
+                                                               rpcConnection = 
new JobManagerRegisteredRpcConnection(
+                                                                       LOG,
+                                                                       
leaderAddress,
+                                                                       
jobMasterId,
+                                                                       
rpcService.getExecutor());
+                                                       }
+                                               } else {
                                                        rpcConnection = new 
JobManagerRegisteredRpcConnection(
                                                                LOG,
                                                                leaderAddress,
                                                                jobMasterId,
                                                                
rpcService.getExecutor());
                                                }
-                                       } else {
-                                               rpcConnection = new 
JobManagerRegisteredRpcConnection(
-                                                       LOG,
-                                                       leaderAddress,
-                                                       jobMasterId,
-                                                       
rpcService.getExecutor());
-                                       }
 
-                                       // double check for a concurrent stop 
operation
-                                       if (stopped) {
-                                               rpcConnection.close();
-                                       } else {
                                                LOG.info("Try to register at 
job manager {} with leader id {}.", leaderAddress, leaderId);
                                                rpcConnection.start();
                                        }
                                }
                        }
+
+                       // send callbacks outside of the lock scope
 
 Review comment:
   Exactly. Triggering callbacks outside of the lock scope is preferrable 
because one does not know what the callback will do. It could happen that it 
waits for another lock which could then lead to a deadlock.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

Reply via email to