-
Notifications
You must be signed in to change notification settings - Fork 383
/
DefaultRaftServer.java
264 lines (231 loc) · 7.63 KB
/
DefaultRaftServer.java
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
/*
* Copyright 2015-present Open Networking Foundation
*
* Licensed 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 io.atomix.protocols.raft.impl;
import io.atomix.protocols.raft.RaftServer;
import io.atomix.protocols.raft.cluster.MemberId;
import io.atomix.protocols.raft.cluster.RaftCluster;
import io.atomix.protocols.raft.cluster.RaftMember;
import io.atomix.protocols.raft.service.RaftService;
import io.atomix.protocols.raft.storage.RaftStorage;
import io.atomix.utils.concurrent.Futures;
import io.atomix.utils.logging.ContextualLoggerFactory;
import io.atomix.utils.logging.LoggerContext;
import org.slf4j.Logger;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.concurrent.CompletableFuture;
import java.util.function.Consumer;
import java.util.function.Supplier;
import static com.google.common.base.MoreObjects.toStringHelper;
import static com.google.common.base.Preconditions.checkNotNull;
/**
* Provides a standalone implementation of the <a href="http://raft.github.io/">Raft consensus algorithm</a>.
*
* @see RaftService
* @see RaftStorage
*/
public class DefaultRaftServer implements RaftServer {
private final Logger log;
protected final RaftContext context;
private volatile CompletableFuture<RaftServer> openFuture;
private volatile CompletableFuture<Void> closeFuture;
private Consumer<RaftMember> electionListener;
private Consumer<RaftContext.State> stateChangeListener;
private volatile boolean started;
public DefaultRaftServer(RaftContext context) {
this.context = checkNotNull(context, "context cannot be null");
this.log = ContextualLoggerFactory.getLogger(getClass(), LoggerContext.builder(RaftServer.class)
.addValue(context.getName())
.build());
}
@Override
public String name() {
return context.getName();
}
@Override
public RaftCluster cluster() {
return context.getCluster();
}
@Override
public Role getRole() {
return context.getRole();
}
@Override
public void addRoleChangeListener(Consumer<Role> listener) {
context.addRoleChangeListener(listener);
}
@Override
public void removeRoleChangeListener(Consumer<Role> listener) {
context.removeRoleChangeListener(listener);
}
@Override
public CompletableFuture<RaftServer> bootstrap() {
return bootstrap(Collections.emptyList());
}
@Override
public CompletableFuture<RaftServer> bootstrap(MemberId... cluster) {
return bootstrap(Arrays.asList(cluster));
}
@Override
public CompletableFuture<RaftServer> bootstrap(Collection<MemberId> cluster) {
return start(() -> cluster().bootstrap(cluster));
}
@Override
public CompletableFuture<RaftServer> join(MemberId... cluster) {
return join(Arrays.asList(cluster));
}
@Override
public CompletableFuture<RaftServer> join(Collection<MemberId> cluster) {
return start(() -> cluster().join(cluster));
}
/**
* Starts the server.
*/
private CompletableFuture<RaftServer> start(Supplier<CompletableFuture<Void>> joiner) {
if (started)
return CompletableFuture.completedFuture(this);
if (openFuture == null) {
synchronized (this) {
if (openFuture == null) {
CompletableFuture<RaftServer> future = new CompletableFuture<>();
openFuture = future;
joiner.get().whenComplete((result, error) -> {
if (error == null) {
context.awaitState(RaftContext.State.READY, state -> {
started = true;
future.complete(null);
});
} else {
future.completeExceptionally(error);
}
});
}
}
}
return openFuture.whenComplete((result, error) -> {
if (error == null) {
log.info("Server started successfully!");
} else {
log.warn("Failed to start server!");
}
});
}
@Override
public CompletableFuture<RaftServer> promote() {
return context.anoint().thenApply(v -> this);
}
/**
* Returns a boolean indicating whether the server is running.
*
* @return Indicates whether the server is running.
*/
public boolean isRunning() {
return started;
}
/**
* Shuts down the server without leaving the Raft cluster.
*
* @return A completable future to be completed once the server has been shutdown.
*/
public CompletableFuture<Void> shutdown() {
if (!started) {
return Futures.exceptionalFuture(new IllegalStateException("context not open"));
}
CompletableFuture<Void> future = new CompletableFuture<>();
context.getThreadContext().execute(() -> {
started = false;
context.transition(Role.INACTIVE);
future.complete(null);
});
return future.whenCompleteAsync((result, error) -> {
context.close();
started = false;
});
}
/**
* Leaves the Raft cluster.
*
* @return A completable future to be completed once the server has left the cluster.
*/
public CompletableFuture<Void> leave() {
if (!started) {
return CompletableFuture.completedFuture(null);
}
if (closeFuture == null) {
synchronized (this) {
if (closeFuture == null) {
closeFuture = new CompletableFuture<>();
if (openFuture == null) {
cluster().leave().whenComplete((leaveResult, leaveError) -> {
shutdown().whenComplete((shutdownResult, shutdownError) -> {
context.delete();
closeFuture.complete(null);
});
});
} else {
openFuture.whenComplete((openResult, openError) -> {
if (openError == null) {
cluster().leave().whenComplete((leaveResult, leaveError) -> {
shutdown().whenComplete((shutdownResult, shutdownError) -> {
context.delete();
closeFuture.complete(null);
});
});
} else {
closeFuture.complete(null);
}
});
}
}
}
}
return closeFuture;
}
@Override
public String toString() {
return toStringHelper(this)
.add("name", name())
.toString();
}
/**
* Default Raft server builder.
*/
public static class Builder extends RaftServer.Builder {
public Builder(MemberId localMemberId) {
super(localMemberId);
}
@Override
public RaftServer build() {
if (serviceRegistry.size() == 0) {
throw new IllegalStateException("No state machines registered");
}
// If the server name is null, set it to the member ID.
if (name == null) {
name = localMemberId.id();
}
// If the storage is not configured, create a new Storage instance with the configured serializer.
if (storage == null) {
storage = RaftStorage.newBuilder().build();
}
RaftContext raft = new RaftContext(name, type, localMemberId, protocol, storage, serviceRegistry, threadModel, threadPoolSize);
raft.setElectionTimeout(electionTimeout);
raft.setHeartbeatInterval(heartbeatInterval);
raft.setSessionTimeout(sessionTimeout);
return new DefaultRaftServer(raft);
}
}
}