summaryrefslogtreecommitdiff
path: root/core/src/main/java/org/elasticsearch/discovery/local/LocalDiscovery.java
blob: d84f471cffeeb44a8ad2c9d7c91673e4cdd83d82 (plain)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
/*
 * Licensed to Elasticsearch under one or more contributor
 * license agreements. See the NOTICE file distributed with
 * this work for additional information regarding copyright
 * ownership. Elasticsearch licenses this file to you under
 * the Apache License, Version 2.0 (the "License"); you may
 * not use this file except in compliance with the License.
 * You may obtain a copy of the License at
 *
 *    http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing,
 * software distributed under the License is distributed on an
 * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
 * KIND, either express or implied.  See the License for the
 * specific language governing permissions and limitations
 * under the License.
 */

package org.elasticsearch.discovery.local;

import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.logging.log4j.util.Supplier;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateUpdateTask;
import org.elasticsearch.cluster.Diff;
import org.elasticsearch.cluster.IncompatibleClusterStateVersionException;
import org.elasticsearch.cluster.block.ClusterBlocks;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.allocation.AllocationService;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.discovery.AckClusterStatePublishResponseHandler;
import org.elasticsearch.discovery.BlockingClusterStatePublishResponseHandler;
import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.discovery.DiscoverySettings;
import org.elasticsearch.discovery.DiscoveryStats;
import org.elasticsearch.discovery.zen.publish.PendingClusterStateStats;

import java.util.HashSet;
import java.util.Optional;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.ConcurrentMap;

import static org.elasticsearch.cluster.ClusterState.Builder;

/**
 *
 */
public class LocalDiscovery extends AbstractLifecycleComponent implements Discovery {

    private static final LocalDiscovery[] NO_MEMBERS = new LocalDiscovery[0];

    private final ClusterService clusterService;
    private AllocationService allocationService;
    private final ClusterName clusterName;

    private final DiscoverySettings discoverySettings;

    private volatile boolean master = false;

    private static final ConcurrentMap<ClusterName, ClusterGroup> clusterGroups = ConcurrentCollections.newConcurrentMap();

    private volatile ClusterState lastProcessedClusterState;

    @Inject
    public LocalDiscovery(Settings settings, ClusterService clusterService, ClusterSettings clusterSettings) {
        super(settings);
        this.clusterName = clusterService.getClusterName();
        this.clusterService = clusterService;
        this.discoverySettings = new DiscoverySettings(settings, clusterSettings);
    }

    @Override
    public void setAllocationService(AllocationService allocationService) {
        this.allocationService = allocationService;
    }

    @Override
    protected void doStart() {

    }

    @Override
    public void startInitialJoin() {
        synchronized (clusterGroups) {
            ClusterGroup clusterGroup = clusterGroups.get(clusterName);
            if (clusterGroup == null) {
                clusterGroup = new ClusterGroup();
                clusterGroups.put(clusterName, clusterGroup);
            }
            logger.debug("Connected to cluster [{}]", clusterName);

            Optional<LocalDiscovery> current = clusterGroup.members().stream().filter(other -> (
                other.localNode().equals(this.localNode()) || other.localNode().getId().equals(this.localNode().getId())
            )).findFirst();
            if (current.isPresent()) {
                throw new IllegalStateException("current cluster group already contains a node with the same id. current "
                    + current.get().localNode() + ", this node " + localNode());
            }

            clusterGroup.members().add(this);

            LocalDiscovery firstMaster = null;
            for (LocalDiscovery localDiscovery : clusterGroup.members()) {
                if (localDiscovery.localNode().isMasterNode()) {
                    firstMaster = localDiscovery;
                    break;
                }
            }

            if (firstMaster != null && firstMaster.equals(this)) {
                // we are the first master (and the master)
                master = true;
                final LocalDiscovery master = firstMaster;
                clusterService.submitStateUpdateTask("local-disco-initial_connect(master)", new ClusterStateUpdateTask() {

                    @Override
                    public boolean runOnlyOnMaster() {
                        return false;
                    }

                    @Override
                    public ClusterState execute(ClusterState currentState) {
                        DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder();
                        for (LocalDiscovery discovery : clusterGroups.get(clusterName).members()) {
                            nodesBuilder.add(discovery.localNode());
                        }
                        nodesBuilder.localNodeId(master.localNode().getId()).masterNodeId(master.localNode().getId());
                        // remove the NO_MASTER block in this case
                        ClusterBlocks.Builder blocks = ClusterBlocks.builder().blocks(currentState.blocks()).removeGlobalBlock(discoverySettings.getNoMasterBlock());
                        return ClusterState.builder(currentState).nodes(nodesBuilder).blocks(blocks).build();
                    }

                    @Override
                    public void onFailure(String source, Exception e) {
                        logger.error((Supplier<?>) () -> new ParameterizedMessage("unexpected failure during [{}]", source), e);
                    }
                });
            } else if (firstMaster != null) {
                // tell the master to send the fact that we are here
                final LocalDiscovery master = firstMaster;
                firstMaster.clusterService.submitStateUpdateTask("local-disco-receive(from node[" + localNode() + "])", new ClusterStateUpdateTask() {
                    @Override
                    public boolean runOnlyOnMaster() {
                        return false;
                    }

                    @Override
                    public ClusterState execute(ClusterState currentState) {
                        DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder();
                        for (LocalDiscovery discovery : clusterGroups.get(clusterName).members()) {
                            nodesBuilder.add(discovery.localNode());
                        }
                        nodesBuilder.localNodeId(master.localNode().getId()).masterNodeId(master.localNode().getId());
                        currentState = ClusterState.builder(currentState).nodes(nodesBuilder).build();
                        RoutingAllocation.Result result =  master.allocationService.reroute(currentState, "node_add");
                        if (result.changed()) {
                            currentState = ClusterState.builder(currentState).routingResult(result).build();
                        }
                        return currentState;
                    }

                    @Override
                    public void onFailure(String source, Exception e) {
                        logger.error((Supplier<?>) () -> new ParameterizedMessage("unexpected failure during [{}]", source), e);
                    }

                });
            }
        } // else, no master node, the next node that will start will fill things in...
    }

    @Override
    protected void doStop() {
        synchronized (clusterGroups) {
            ClusterGroup clusterGroup = clusterGroups.get(clusterName);
            if (clusterGroup == null) {
                logger.warn("Illegal state, should not have an empty cluster group when stopping, I should be there at teh very least...");
                return;
            }
            clusterGroup.members().remove(this);
            if (clusterGroup.members().isEmpty()) {
                // no more members, remove and return
                clusterGroups.remove(clusterName);
                return;
            }

            LocalDiscovery firstMaster = null;
            for (LocalDiscovery localDiscovery : clusterGroup.members()) {
                if (localDiscovery.localNode().isMasterNode()) {
                    firstMaster = localDiscovery;
                    break;
                }
            }

            if (firstMaster != null) {
                // if the removed node is the master, make the next one as the master
                if (master) {
                    firstMaster.master = true;
                }

                final Set<String> newMembers = new HashSet<>();
                for (LocalDiscovery discovery : clusterGroup.members()) {
                    newMembers.add(discovery.localNode().getId());
                }

                final LocalDiscovery master = firstMaster;
                master.clusterService.submitStateUpdateTask("local-disco-update", new ClusterStateUpdateTask() {
                    @Override
                    public boolean runOnlyOnMaster() {
                        return false;
                    }

                    @Override
                    public ClusterState execute(ClusterState currentState) {
                        DiscoveryNodes newNodes = currentState.nodes().removeDeadMembers(newMembers, master.localNode().getId());
                        DiscoveryNodes.Delta delta = newNodes.delta(currentState.nodes());
                        if (delta.added()) {
                            logger.warn("No new nodes should be created when a new discovery view is accepted");
                        }
                        // reroute here, so we eagerly remove dead nodes from the routing
                        ClusterState updatedState = ClusterState.builder(currentState).nodes(newNodes).build();
                        RoutingAllocation.Result routingResult = master.allocationService.deassociateDeadNodes(
                                ClusterState.builder(updatedState).build(), true, "node stopped");
                        return ClusterState.builder(updatedState).routingResult(routingResult).build();
                    }

                    @Override
                    public void onFailure(String source, Exception e) {
                        logger.error((Supplier<?>) () -> new ParameterizedMessage("unexpected failure during [{}]", source), e);
                    }
                });
            }
        }
    }

    @Override
    protected void doClose() {
    }

    @Override
    public DiscoveryNode localNode() {
        return clusterService.localNode();
    }

    @Override
    public String nodeDescription() {
        return clusterName.value() + "/" + localNode().getId();
    }

    @Override
    public void publish(ClusterChangedEvent clusterChangedEvent, final Discovery.AckListener ackListener) {
        if (!master) {
            throw new IllegalStateException("Shouldn't publish state when not master");
        }
        LocalDiscovery[] members = members();
        if (members.length > 0) {
            Set<DiscoveryNode> nodesToPublishTo = new HashSet<>(members.length);
            for (LocalDiscovery localDiscovery : members) {
                if (localDiscovery.master) {
                    continue;
                }
                nodesToPublishTo.add(localDiscovery.localNode());
            }
            publish(members, clusterChangedEvent, new AckClusterStatePublishResponseHandler(nodesToPublishTo, ackListener));
        }
    }

    @Override
    public DiscoveryStats stats() {
        return new DiscoveryStats((PendingClusterStateStats)null);
    }

    @Override
    public DiscoverySettings getDiscoverySettings() {
        return discoverySettings;
    }

    @Override
    public int getMinimumMasterNodes() {
        return -1;
    }

    private LocalDiscovery[] members() {
        ClusterGroup clusterGroup = clusterGroups.get(clusterName);
        if (clusterGroup == null) {
            return NO_MEMBERS;
        }
        Queue<LocalDiscovery> members = clusterGroup.members();
        return members.toArray(new LocalDiscovery[members.size()]);
    }

    private void publish(LocalDiscovery[] members, ClusterChangedEvent clusterChangedEvent, final BlockingClusterStatePublishResponseHandler publishResponseHandler) {

        try {
            // we do the marshaling intentionally, to check it works well...
            byte[] clusterStateBytes = null;
            byte[] clusterStateDiffBytes = null;

            ClusterState clusterState = clusterChangedEvent.state();
            for (final LocalDiscovery discovery : members) {
                if (discovery.master) {
                    continue;
                }
                ClusterState newNodeSpecificClusterState = null;
                synchronized (this) {
                    // we do the marshaling intentionally, to check it works well...
                    // check if we published cluster state at least once and node was in the cluster when we published cluster state the last time
                    if (discovery.lastProcessedClusterState != null && clusterChangedEvent.previousState().nodes().nodeExists(discovery.localNode())) {
                        // both conditions are true - which means we can try sending cluster state as diffs
                        if (clusterStateDiffBytes == null) {
                            Diff diff = clusterState.diff(clusterChangedEvent.previousState());
                            BytesStreamOutput os = new BytesStreamOutput();
                            diff.writeTo(os);
                            clusterStateDiffBytes = BytesReference.toBytes(os.bytes());
                        }
                        try {
                            newNodeSpecificClusterState = discovery.lastProcessedClusterState.readDiffFrom(StreamInput.wrap(clusterStateDiffBytes)).apply(discovery.lastProcessedClusterState);
                            logger.trace("sending diff cluster state version [{}] with size {} to [{}]", clusterState.version(), clusterStateDiffBytes.length, discovery.localNode().getName());
                        } catch (IncompatibleClusterStateVersionException ex) {
                            logger.warn((Supplier<?>) () -> new ParameterizedMessage("incompatible cluster state version [{}] - resending complete cluster state", clusterState.version()), ex);
                        }
                    }
                    if (newNodeSpecificClusterState == null) {
                        if (clusterStateBytes == null) {
                            clusterStateBytes = Builder.toBytes(clusterState);
                        }
                        newNodeSpecificClusterState = ClusterState.Builder.fromBytes(clusterStateBytes, discovery.localNode());
                    }
                    discovery.lastProcessedClusterState = newNodeSpecificClusterState;
                }
                final ClusterState nodeSpecificClusterState = newNodeSpecificClusterState;

                nodeSpecificClusterState.status(ClusterState.ClusterStateStatus.RECEIVED);
                // ignore cluster state messages that do not include "me", not in the game yet...
                if (nodeSpecificClusterState.nodes().getLocalNode() != null) {
                    assert nodeSpecificClusterState.nodes().getMasterNode() != null : "received a cluster state without a master";
                    assert !nodeSpecificClusterState.blocks().hasGlobalBlock(discoverySettings.getNoMasterBlock()) : "received a cluster state with a master block";

                    discovery.clusterService.submitStateUpdateTask("local-disco-receive(from master)", new ClusterStateUpdateTask() {
                        @Override
                        public boolean runOnlyOnMaster() {
                            return false;
                        }

                        @Override
                        public ClusterState execute(ClusterState currentState) {
                            if (currentState.supersedes(nodeSpecificClusterState)) {
                                return currentState;
                            }

                            if (currentState.blocks().hasGlobalBlock(discoverySettings.getNoMasterBlock())) {
                                // its a fresh update from the master as we transition from a start of not having a master to having one
                                logger.debug("got first state from fresh master [{}]", nodeSpecificClusterState.nodes().getMasterNodeId());
                                return nodeSpecificClusterState;
                            }

                            ClusterState.Builder builder = ClusterState.builder(nodeSpecificClusterState);
                            // if the routing table did not change, use the original one
                            if (nodeSpecificClusterState.routingTable().version() == currentState.routingTable().version()) {
                                builder.routingTable(currentState.routingTable());
                            }
                            if (nodeSpecificClusterState.metaData().version() == currentState.metaData().version()) {
                                builder.metaData(currentState.metaData());
                            }

                            return builder.build();
                        }

                        @Override
                        public void onFailure(String source, Exception e) {
                            logger.error((Supplier<?>) () -> new ParameterizedMessage("unexpected failure during [{}]", source), e);
                            publishResponseHandler.onFailure(discovery.localNode(), e);
                        }

                        @Override
                        public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
                            publishResponseHandler.onResponse(discovery.localNode());
                        }
                    });
                } else {
                    publishResponseHandler.onResponse(discovery.localNode());
                }
            }

            TimeValue publishTimeout = discoverySettings.getPublishTimeout();
            if (publishTimeout.millis() > 0) {
                try {
                    boolean awaited = publishResponseHandler.awaitAllNodes(publishTimeout);
                    if (!awaited) {
                        DiscoveryNode[] pendingNodes = publishResponseHandler.pendingNodes();
                        // everyone may have just responded
                        if (pendingNodes.length > 0) {
                            logger.warn("timed out waiting for all nodes to process published state [{}] (timeout [{}], pending nodes: {})", clusterState.version(), publishTimeout, pendingNodes);
                        }
                    }
                } catch (InterruptedException e) {
                    // ignore & restore interrupt
                    Thread.currentThread().interrupt();
                }
            }


        } catch (Exception e) {
            // failure to marshal or un-marshal
            throw new IllegalStateException("Cluster state failed to serialize", e);
        }
    }

    private class ClusterGroup {

        private Queue<LocalDiscovery> members = ConcurrentCollections.newQueue();

        Queue<LocalDiscovery> members() {
            return members;
        }
    }
}