summaryrefslogtreecommitdiff
path: root/core/src/main/java/org/elasticsearch/cluster/ClusterStateObserver.java
blob: dd30a7116883585e2a5bf31a4047b7875baf0376 (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
/*
 * 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.cluster;

import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.ThreadContext;

import java.util.concurrent.atomic.AtomicReference;

/**
 * A utility class which simplifies interacting with the cluster state in cases where
 * one tries to take action based on the current state but may want to wait for a new state
 * and retry upon failure.
 */
public class ClusterStateObserver {

    protected final ESLogger logger;

    public final ChangePredicate MATCH_ALL_CHANGES_PREDICATE = new EventPredicate() {

        @Override
        public boolean apply(ClusterChangedEvent changedEvent) {
            return changedEvent.previousState().version() != changedEvent.state().version();
        }
    };

    private final  ClusterService clusterService;
    private final ThreadContext contextHolder;
    volatile TimeValue timeOutValue;


    final AtomicReference<ObservedState> lastObservedState;
    final TimeoutClusterStateListener clusterStateListener = new ObserverClusterStateListener();
    // observingContext is not null when waiting on cluster state changes
    final AtomicReference<ObservingContext> observingContext = new AtomicReference<>(null);
    volatile Long startTimeNS;
    volatile boolean timedOut;


    public ClusterStateObserver(ClusterService clusterService, ESLogger logger, ThreadContext contextHolder) {
        this(clusterService, new TimeValue(60000), logger, contextHolder);
    }

    /**
     * @param timeout        a global timeout for this observer. After it has expired the observer
     *                       will fail any existing or new #waitForNextChange calls. Set to null
     *                       to wait indefinitely
     */
    public ClusterStateObserver(ClusterService clusterService, @Nullable TimeValue timeout, ESLogger logger, ThreadContext contextHolder) {
        this.clusterService = clusterService;
        this.lastObservedState = new AtomicReference<>(new ObservedState(clusterService.state()));
        this.timeOutValue = timeout;
        if (timeOutValue != null) {
            this.startTimeNS = System.nanoTime();
        }
        this.logger = logger;
        this.contextHolder = contextHolder;
    }

    /** last cluster state observer by this observer. Note that this may not be the current one */
    public ClusterState observedState() {
        ObservedState state = lastObservedState.get();
        assert state != null;
        return state.clusterState;
    }

    /** indicates whether this observer has timedout */
    public boolean isTimedOut() {
        return timedOut;
    }

    public void waitForNextChange(Listener listener) {
        waitForNextChange(listener, MATCH_ALL_CHANGES_PREDICATE);
    }

    public void waitForNextChange(Listener listener, @Nullable TimeValue timeOutValue) {
        waitForNextChange(listener, MATCH_ALL_CHANGES_PREDICATE, timeOutValue);
    }

    public void waitForNextChange(Listener listener, ChangePredicate changePredicate) {
        waitForNextChange(listener, changePredicate, null);
    }

    /**
     * Wait for the next cluster state which satisfies changePredicate
     *
     * @param listener        callback listener
     * @param changePredicate predicate to check whether cluster state changes are relevant and the callback should be called
     * @param timeOutValue    a timeout for waiting. If null the global observer timeout will be used.
     */
    public void waitForNextChange(Listener listener, ChangePredicate changePredicate, @Nullable TimeValue timeOutValue) {

        if (observingContext.get() != null) {
            throw new ElasticsearchException("already waiting for a cluster state change");
        }

        Long timeoutTimeLeftMS;
        if (timeOutValue == null) {
            timeOutValue = this.timeOutValue;
            if (timeOutValue != null) {
                long timeSinceStartMS = TimeValue.nsecToMSec(System.nanoTime() - startTimeNS);
                timeoutTimeLeftMS = timeOutValue.millis() - timeSinceStartMS;
                if (timeoutTimeLeftMS <= 0L) {
                    // things have timeout while we were busy -> notify
                    logger.trace("observer timed out. notifying listener. timeout setting [{}], time since start [{}]", timeOutValue, new TimeValue(timeSinceStartMS));
                    // update to latest, in case people want to retry
                    timedOut = true;
                    lastObservedState.set(new ObservedState(clusterService.state()));
                    listener.onTimeout(timeOutValue);
                    return;
                }
            } else {
                timeoutTimeLeftMS = null;
            }
        } else {
            this.startTimeNS = System.nanoTime();
            this.timeOutValue = timeOutValue;
            timeoutTimeLeftMS = timeOutValue.millis();
            timedOut = false;
        }

        // sample a new state
        ObservedState newState = new ObservedState(clusterService.state());
        ObservedState lastState = lastObservedState.get();
        if (changePredicate.apply(lastState.clusterState, lastState.status, newState.clusterState, newState.status)) {
            // good enough, let's go.
            logger.trace("observer: sampled state accepted by predicate ({})", newState);
            lastObservedState.set(newState);
            listener.onNewClusterState(newState.clusterState);
        } else {
            logger.trace("observer: sampled state rejected by predicate ({}). adding listener to ClusterService", newState);
            ObservingContext context = new ObservingContext(new ContextPreservingListener(listener, contextHolder.newStoredContext()), changePredicate);
            if (!observingContext.compareAndSet(null, context)) {
                throw new ElasticsearchException("already waiting for a cluster state change");
            }
            clusterService.add(timeoutTimeLeftMS == null ? null : new TimeValue(timeoutTimeLeftMS), clusterStateListener);
        }
    }

    /**
     * reset this observer to the give cluster state. Any pending waits will be canceled.
     */
    public void reset(ClusterState toState) {
        if (observingContext.getAndSet(null) != null) {
            clusterService.remove(clusterStateListener);
        }
        lastObservedState.set(new ObservedState(toState));
    }

    class ObserverClusterStateListener implements TimeoutClusterStateListener {

        @Override
        public void clusterChanged(ClusterChangedEvent event) {
            ObservingContext context = observingContext.get();
            if (context == null) {
                // No need to remove listener as it is the responsibility of the thread that set observingContext to null
                return;
            }
            if (context.changePredicate.apply(event)) {
                if (observingContext.compareAndSet(context, null)) {
                    clusterService.remove(this);
                    ObservedState state = new ObservedState(event.state());
                    logger.trace("observer: accepting cluster state change ({})", state);
                    lastObservedState.set(state);
                    context.listener.onNewClusterState(state.clusterState);
                } else {
                    logger.trace("observer: predicate approved change but observing context has changed - ignoring (new cluster state version [{}])", event.state().version());
                }
            } else {
                logger.trace("observer: predicate rejected change (new cluster state version [{}])", event.state().version());
            }
        }

        @Override
        public void postAdded() {
            ObservingContext context = observingContext.get();
            if (context == null) {
                // No need to remove listener as it is the responsibility of the thread that set observingContext to null
                return;
            }
            ObservedState newState = new ObservedState(clusterService.state());
            ObservedState lastState = lastObservedState.get();
            if (context.changePredicate.apply(lastState.clusterState, lastState.status, newState.clusterState, newState.status)) {
                // double check we're still listening
                if (observingContext.compareAndSet(context, null)) {
                    logger.trace("observer: post adding listener: accepting current cluster state ({})", newState);
                    clusterService.remove(this);
                    lastObservedState.set(newState);
                    context.listener.onNewClusterState(newState.clusterState);
                } else {
                    logger.trace("observer: postAdded - predicate approved state but observing context has changed - ignoring ({})", newState);
                }
            } else {
                logger.trace("observer: postAdded - predicate rejected state ({})", newState);
            }
        }

        @Override
        public void onClose() {
            ObservingContext context = observingContext.getAndSet(null);

            if (context != null) {
                logger.trace("observer: cluster service closed. notifying listener.");
                clusterService.remove(this);
                context.listener.onClusterServiceClose();
            }
        }

        @Override
        public void onTimeout(TimeValue timeout) {
            ObservingContext context = observingContext.getAndSet(null);
            if (context != null) {
                clusterService.remove(this);
                long timeSinceStartMS = TimeValue.nsecToMSec(System.nanoTime() - startTimeNS);
                logger.trace("observer: timeout notification from cluster service. timeout setting [{}], time since start [{}]", timeOutValue, new TimeValue(timeSinceStartMS));
                // update to latest, in case people want to retry
                lastObservedState.set(new ObservedState(clusterService.state()));
                timedOut = true;
                context.listener.onTimeout(timeOutValue);
            }
        }
    }

    public interface Listener {

        /** called when a new state is observed */
        void onNewClusterState(ClusterState state);

        /** called when the cluster service is closed */
        void onClusterServiceClose();

        void onTimeout(TimeValue timeout);
    }

    public interface ChangePredicate {

        /**
         * a rough check used when starting to monitor for a new change. Called infrequently can be less accurate.
         *
         * @return true if newState should be accepted
         */
        boolean apply(ClusterState previousState,
                      ClusterState.ClusterStateStatus previousStatus,
                      ClusterState newState,
                      ClusterState.ClusterStateStatus newStatus);

        /**
         * called to see whether a cluster change should be accepted
         *
         * @return true if changedEvent.state() should be accepted
         */
        boolean apply(ClusterChangedEvent changedEvent);
    }


    public static abstract class ValidationPredicate implements ChangePredicate {

        @Override
        public boolean apply(ClusterState previousState, ClusterState.ClusterStateStatus previousStatus, ClusterState newState, ClusterState.ClusterStateStatus newStatus) {
            return (previousState != newState || previousStatus != newStatus) && validate(newState);
        }

        protected abstract boolean validate(ClusterState newState);

        @Override
        public boolean apply(ClusterChangedEvent changedEvent) {
            return changedEvent.previousState().version() != changedEvent.state().version() && validate(changedEvent.state());
        }
    }

    public static abstract class EventPredicate implements ChangePredicate {
        @Override
        public boolean apply(ClusterState previousState, ClusterState.ClusterStateStatus previousStatus, ClusterState newState, ClusterState.ClusterStateStatus newStatus) {
            return previousState != newState || previousStatus != newStatus;
        }

    }

    static class ObservingContext {
        final public Listener listener;
        final public ChangePredicate changePredicate;

        public ObservingContext(Listener listener, ChangePredicate changePredicate) {
            this.listener = listener;
            this.changePredicate = changePredicate;
        }
    }

    static class ObservedState {
        final public ClusterState clusterState;
        final public ClusterState.ClusterStateStatus status;

        public ObservedState(ClusterState clusterState) {
            this.clusterState = clusterState;
            this.status = clusterState.status();
        }

        @Override
        public String toString() {
            return "version [" + clusterState.version() + "], status [" + status + "]";
        }
    }

    private final static class ContextPreservingListener implements Listener {
        private final Listener delegate;
        private final ThreadContext.StoredContext tempContext;


        private ContextPreservingListener(Listener delegate, ThreadContext.StoredContext storedContext) {
            this.tempContext = storedContext;
            this.delegate = delegate;
        }

        @Override
        public void onNewClusterState(ClusterState state) {
            tempContext.restore();
            delegate.onNewClusterState(state);
        }

        @Override
        public void onClusterServiceClose() {
            tempContext.restore();
            delegate.onClusterServiceClose();
        }

        @Override
        public void onTimeout(TimeValue timeout) {
            tempContext.restore();
            delegate.onTimeout(timeout);
        }
    }
}