2424import java .io .DataInputStream ;
2525import java .io .DataOutputStream ;
2626import java .io .IOException ;
27- import java .net .BindException ;
2827import java .net .InetSocketAddress ;
2928import java .net .ServerSocket ;
3029import java .net .Socket ;
4039import java .util .List ;
4140import java .util .Map ;
4241import java .util .Objects ;
42+ import java .util .Optional ;
4343import java .util .Set ;
4444import java .util .concurrent .ConcurrentHashMap ;
4545import java .util .concurrent .ConcurrentLinkedQueue ;
@@ -290,15 +290,20 @@ public boolean isQuorumSynced(QuorumVerifier qv) {
290290 addresses = self .getQuorumAddress ().getAllAddresses ();
291291 }
292292
293- for (InetSocketAddress address : addresses ) {
294- serverSockets .add (createServerSocket (address , self .shouldUsePortUnification (), self .isSslQuorum ()));
293+ addresses .stream ()
294+ .map (address -> createServerSocket (address , self .shouldUsePortUnification (), self .isSslQuorum ()))
295+ .filter (Optional ::isPresent )
296+ .map (Optional ::get )
297+ .forEach (serverSockets ::add );
298+
299+ if (serverSockets .isEmpty ()) {
300+ throw new IOException ("Leader failed to initialize any of the following sockets: " + addresses );
295301 }
296302
297303 this .zk = zk ;
298304 }
299305
300- ServerSocket createServerSocket (InetSocketAddress address , boolean portUnification , boolean sslQuorum )
301- throws IOException {
306+ Optional <ServerSocket > createServerSocket (InetSocketAddress address , boolean portUnification , boolean sslQuorum ) {
302307 ServerSocket serverSocket ;
303308 try {
304309 if (portUnification || sslQuorum ) {
@@ -308,11 +313,11 @@ ServerSocket createServerSocket(InetSocketAddress address, boolean portUnificati
308313 }
309314 serverSocket .setReuseAddress (true );
310315 serverSocket .bind (address );
311- return serverSocket ;
312- } catch (BindException e ) {
316+ return Optional . of ( serverSocket ) ;
317+ } catch (IOException e ) {
313318 LOG .error ("Couldn't bind to " + address .toString (), e );
314- throw e ;
315319 }
320+ return Optional .empty ();
316321 }
317322
318323 /**
0 commit comments