52
52
import org .elasticsearch .discovery .zen .fd .MasterFaultDetection ;
53
53
import org .elasticsearch .discovery .zen .fd .NodesFaultDetection ;
54
54
import org .elasticsearch .discovery .zen .membership .MembershipAction ;
55
+ import org .elasticsearch .discovery .zen .ping .PingContextProvider ;
55
56
import org .elasticsearch .discovery .zen .ping .ZenPing ;
56
57
import org .elasticsearch .discovery .zen .ping .ZenPingService ;
57
58
import org .elasticsearch .discovery .zen .publish .PublishClusterStateAction ;
69
70
import java .util .concurrent .CopyOnWriteArrayList ;
70
71
import java .util .concurrent .atomic .AtomicBoolean ;
71
72
import java .util .concurrent .atomic .AtomicInteger ;
73
+ import java .util .concurrent .atomic .AtomicLong ;
72
74
73
75
import static com .google .common .collect .Lists .newArrayList ;
74
76
import static org .elasticsearch .common .unit .TimeValue .timeValueSeconds ;
75
77
76
78
/**
77
79
*
78
80
*/
79
- public class ZenDiscovery extends AbstractLifecycleComponent <Discovery > implements Discovery , DiscoveryNodesProvider {
81
+ public class ZenDiscovery extends AbstractLifecycleComponent <Discovery > implements Discovery , PingContextProvider {
80
82
81
83
public final static String SETTING_REJOIN_ON_MASTER_GONE = "discovery.zen.rejoin_on_master_gone" ;
82
84
public final static String SETTING_PING_TIMEOUT = "discovery.zen.ping.timeout" ;
@@ -139,6 +141,9 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
139
141
140
142
private volatile boolean rejoinOnMasterGone ;
141
143
144
+ /** counts the time this node has joined the cluster or have elected it self as master */
145
+ private final AtomicLong clusterJoinsCounter = new AtomicLong ();
146
+
142
147
@ Nullable
143
148
private NodeService nodeService ;
144
149
@@ -194,7 +199,7 @@ public ZenDiscovery(Settings settings, ClusterName clusterName, ThreadPool threa
194
199
this .nodesFD .addListener (new NodeFaultDetectionListener ());
195
200
196
201
this .publishClusterState = new PublishClusterStateAction (settings , transportService , this , new NewClusterStateListener (), discoverySettings , clusterName );
197
- this .pingService .setNodesProvider (this );
202
+ this .pingService .setPingContextProvider (this );
198
203
this .membership = new MembershipAction (settings , clusterService , transportService , this , new MembershipListener ());
199
204
200
205
transportService .registerHandler (DISCOVERY_REJOIN_ACTION_NAME , new RejoinClusterRequestHandler ());
@@ -290,6 +295,7 @@ public String nodeDescription() {
290
295
return clusterName .value () + "/" + localNode .id ();
291
296
}
292
297
298
+ /** start of {@link org.elasticsearch.discovery.zen.ping.PingContextProvider } implementation */
293
299
@ Override
294
300
public DiscoveryNodes nodes () {
295
301
DiscoveryNodes latestNodes = this .latestDiscoNodes ;
@@ -305,6 +311,14 @@ public NodeService nodeService() {
305
311
return this .nodeService ;
306
312
}
307
313
314
+ @ Override
315
+ public boolean nodeHasJoinedClusterOnce () {
316
+ return clusterJoinsCounter .get () > 0 ;
317
+ }
318
+
319
+ /** end of {@link org.elasticsearch.discovery.zen.ping.PingContextProvider } implementation */
320
+
321
+
308
322
@ Override
309
323
public void publish (ClusterState clusterState , AckListener ackListener ) {
310
324
if (!master ) {
@@ -387,6 +401,8 @@ public void onFailure(String source, Throwable t) {
387
401
@ Override
388
402
public void clusterStateProcessed (String source , ClusterState oldState , ClusterState newState ) {
389
403
sendInitialStateEventIfNeeded ();
404
+ long count = clusterJoinsCounter .incrementAndGet ();
405
+ logger .trace ("cluster joins counter set to [{}] (elected as master)" , count );
390
406
}
391
407
});
392
408
} else {
@@ -404,8 +420,8 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS
404
420
}
405
421
406
422
masterFD .start (masterNode , "initial_join" );
407
- // no need to submit the received cluster state, we will get it from the master when it publishes
408
- // the fact that we joined
423
+ long count = clusterJoinsCounter . incrementAndGet ();
424
+ logger . trace ( "cluster joins counter set to [{}] ( joined master)" , count );
409
425
}
410
426
}
411
427
}
@@ -922,7 +938,7 @@ private DiscoveryNode findMaster() {
922
938
sb .append (" {none}" );
923
939
} else {
924
940
for (ZenPing .PingResponse pingResponse : fullPingResponses ) {
925
- sb .append ("\n \t --> " ).append ("target [" ). append ( pingResponse . target ()). append ( "], master [" ). append ( pingResponse . master ()). append ( "]" );
941
+ sb .append ("\n \t --> " ).append (pingResponse );
926
942
}
927
943
}
928
944
logger .trace (sb .toString ());
@@ -931,7 +947,7 @@ private DiscoveryNode findMaster() {
931
947
// filter responses
932
948
List <ZenPing .PingResponse > pingResponses = Lists .newArrayList ();
933
949
for (ZenPing .PingResponse pingResponse : fullPingResponses ) {
934
- DiscoveryNode node = pingResponse .target ();
950
+ DiscoveryNode node = pingResponse .node ();
935
951
if (masterElectionFilterClientNodes && (node .clientNode () || (!node .masterNode () && !node .dataNode ()))) {
936
952
// filter out the client node, which is a client node, or also one that is not data and not master (effectively, client)
937
953
} else if (masterElectionFilterDataNodes && (!node .masterNode () && node .dataNode ())) {
@@ -947,7 +963,7 @@ private DiscoveryNode findMaster() {
947
963
sb .append (" {none}" );
948
964
} else {
949
965
for (ZenPing .PingResponse pingResponse : pingResponses ) {
950
- sb .append ("\n \t --> " ).append ("target [" ). append ( pingResponse . target ()). append ( "], master [" ). append ( pingResponse . master ()). append ( "]" );
966
+ sb .append ("\n \t --> " ).append (pingResponse );
951
967
}
952
968
}
953
969
logger .debug (sb .toString ());
@@ -963,20 +979,38 @@ private DiscoveryNode findMaster() {
963
979
}
964
980
}
965
981
966
- Set <DiscoveryNode > possibleMasterNodes = Sets .newHashSet ();
982
+ // nodes discovered during pinging
983
+ Set <DiscoveryNode > activeNodes = Sets .newHashSet ();
984
+ // nodes discovered who has previously been part of the cluster and do not ping for the very first time
985
+ Set <DiscoveryNode > joinedOnceActiveNodes = Sets .newHashSet ();
967
986
if (localNode .masterNode ()) {
968
- possibleMasterNodes .add (localNode );
987
+ activeNodes .add (localNode );
988
+ long joinsCounter = clusterJoinsCounter .get ();
989
+ if (joinsCounter > 0 ) {
990
+ logger .trace ("adding local node to the list of active nodes who has previously joined the cluster (joins counter is [{}})" , joinsCounter );
991
+ joinedOnceActiveNodes .add (localNode );
992
+ }
969
993
}
970
994
for (ZenPing .PingResponse pingResponse : pingResponses ) {
971
- possibleMasterNodes .add (pingResponse .target ());
995
+ activeNodes .add (pingResponse .node ());
996
+ if (pingResponse .hasJoinedOnce ()) {
997
+ joinedOnceActiveNodes .add (pingResponse .node ());
998
+ }
972
999
}
973
1000
974
1001
if (pingMasters .isEmpty ()) {
975
- // if we don't have enough master nodes, we bail, because there are not enough master to elect from
976
- if (electMaster .hasEnoughMasterNodes (possibleMasterNodes )) {
977
- return electMaster .electMaster (possibleMasterNodes );
1002
+ if (electMaster .hasEnoughMasterNodes (activeNodes )) {
1003
+ // we give preference to nodes who have previously already joined the cluster. Those will
1004
+ // have a cluster state in memory, including an up to date routing table (which is not persistent to disk
1005
+ // by the gateway)
1006
+ DiscoveryNode master = electMaster .electMaster (joinedOnceActiveNodes );
1007
+ if (master != null ) {
1008
+ return master ;
1009
+ }
1010
+ return electMaster .electMaster (activeNodes );
978
1011
} else {
979
- logger .trace ("not enough master nodes [{}]" , possibleMasterNodes );
1012
+ // if we don't have enough master nodes, we bail, because there are not enough master to elect from
1013
+ logger .trace ("not enough master nodes [{}]" , activeNodes );
980
1014
return null ;
981
1015
}
982
1016
} else {
0 commit comments