-
Notifications
You must be signed in to change notification settings - Fork 554
/
RaftPartitionServer.java
360 lines (317 loc) · 12.5 KB
/
RaftPartitionServer.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
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
/*
* Copyright 2016-present Open Networking Foundation
* Copyright © 2020 camunda services GmbH (info@camunda.com)
*
* 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.raft.partition.impl;
import io.atomix.cluster.ClusterMembershipService;
import io.atomix.cluster.MemberId;
import io.atomix.cluster.messaging.ClusterCommunicationService;
import io.atomix.primitive.partition.Partition;
import io.atomix.primitive.partition.PartitionMetadata;
import io.atomix.raft.RaftCommitListener;
import io.atomix.raft.RaftCommittedEntryListener;
import io.atomix.raft.RaftRoleChangeListener;
import io.atomix.raft.RaftServer;
import io.atomix.raft.RaftServer.Role;
import io.atomix.raft.SnapshotReplicationListener;
import io.atomix.raft.metrics.RaftStartupMetrics;
import io.atomix.raft.partition.RaftElectionConfig;
import io.atomix.raft.partition.RaftPartition;
import io.atomix.raft.partition.RaftPartitionGroupConfig;
import io.atomix.raft.partition.RaftStorageConfig;
import io.atomix.raft.roles.RaftRole;
import io.atomix.raft.storage.RaftStorage;
import io.atomix.raft.storage.StorageException;
import io.atomix.raft.storage.log.RaftLogReader;
import io.atomix.raft.zeebe.ZeebeLogAppender;
import io.atomix.utils.Managed;
import io.atomix.utils.concurrent.Futures;
import io.atomix.utils.logging.ContextualLoggerFactory;
import io.atomix.utils.logging.LoggerContext;
import io.atomix.utils.serializer.Serializer;
import io.camunda.zeebe.snapshots.PersistedSnapshotStore;
import io.camunda.zeebe.snapshots.ReceivableSnapshotStore;
import io.camunda.zeebe.util.health.FailureListener;
import io.camunda.zeebe.util.health.HealthMonitorable;
import io.camunda.zeebe.util.health.HealthStatus;
import java.io.IOException;
import java.nio.file.FileVisitResult;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.SimpleFileVisitor;
import java.nio.file.attribute.BasicFileAttributes;
import java.time.Duration;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CopyOnWriteArraySet;
import org.slf4j.Logger;
/** {@link Partition} server. */
public class RaftPartitionServer implements Managed<RaftPartitionServer>, HealthMonitorable {
private final Logger log;
private final MemberId localMemberId;
private final RaftPartition partition;
private final RaftPartitionGroupConfig config;
private final ClusterMembershipService membershipService;
private final ClusterCommunicationService clusterCommunicator;
private final Set<RaftRoleChangeListener> deferredRoleChangeListeners =
new CopyOnWriteArraySet<>();
private final Set<FailureListener> deferredFailureListeners = new CopyOnWriteArraySet<>();
private final PartitionMetadata partitionMetadata;
private final Duration requestTimeout;
private RaftServer server;
private ReceivableSnapshotStore persistedSnapshotStore;
public RaftPartitionServer(
final RaftPartition partition,
final RaftPartitionGroupConfig config,
final MemberId localMemberId,
final ClusterMembershipService membershipService,
final ClusterCommunicationService clusterCommunicator,
final PartitionMetadata partitionMetadata) {
this.partition = partition;
this.config = config;
this.localMemberId = localMemberId;
this.membershipService = membershipService;
this.clusterCommunicator = clusterCommunicator;
log =
ContextualLoggerFactory.getLogger(
getClass(),
LoggerContext.builder(RaftPartitionServer.class).addValue(partition.name()).build());
this.partitionMetadata = partitionMetadata;
requestTimeout = config.getPartitionConfig().getRequestTimeout();
}
@Override
public CompletableFuture<RaftPartitionServer> start() {
final RaftStartupMetrics raftStartupMetrics = new RaftStartupMetrics(partition.name());
final long bootstrapStartTime;
log.info("Starting server for partition {}", partition.id());
final long startTime = System.currentTimeMillis();
final CompletableFuture<RaftServer> serverOpenFuture;
if (partition.members().contains(localMemberId)) {
if (server != null && server.isRunning()) {
return CompletableFuture.completedFuture(null);
}
synchronized (this) {
try {
initServer();
} catch (final StorageException e) {
return Futures.exceptionalFuture(e);
}
}
bootstrapStartTime = System.currentTimeMillis();
serverOpenFuture = server.bootstrap(partition.members());
} else {
bootstrapStartTime = System.currentTimeMillis();
serverOpenFuture = CompletableFuture.completedFuture(null);
}
return serverOpenFuture
.whenComplete(
(r, e) -> {
if (e == null) {
final long endTime = System.currentTimeMillis();
final long startDuration = endTime - startTime;
raftStartupMetrics.observeBootstrapDuration(endTime - bootstrapStartTime);
raftStartupMetrics.observeStartupDuration(startDuration);
log.info(
"Successfully started server for partition {} in {}ms",
partition.id(),
startDuration);
} else {
log.warn("Failed to start server for partition {}", partition.id(), e);
}
})
.thenApply(v -> this);
}
@Override
public boolean isRunning() {
return server.isRunning();
}
@Override
public CompletableFuture<Void> stop() {
return server != null ? server.shutdown() : CompletableFuture.completedFuture(null);
}
private void initServer() {
server = buildServer();
deferredRoleChangeListeners.forEach(server::addRoleChangeListener);
deferredRoleChangeListeners.clear();
deferredFailureListeners.forEach(server::addFailureListener);
deferredFailureListeners.clear();
}
private RaftServer buildServer() {
final var partitionId = partition.id().id();
persistedSnapshotStore =
config
.getStorageConfig()
.getPersistedSnapshotStoreFactory()
.createReceivableSnapshotStore(partition.dataDirectory().toPath(), partitionId);
final var partitionConfig = config.getPartitionConfig();
final var electionConfig =
partitionConfig.isPriorityElectionEnabled()
? RaftElectionConfig.ofPriorityElection(
partitionMetadata.getTargetPriority(), partitionMetadata.getPriority(localMemberId))
: RaftElectionConfig.ofDefaultElection();
return RaftServer.builder(localMemberId)
.withName(partition.name())
.withPartitionId(partitionId)
.withMembershipService(membershipService)
.withProtocol(createServerProtocol())
.withPartitionConfig(partitionConfig)
.withStorage(createRaftStorage())
.withEntryValidator(config.getEntryValidator())
.withElectionConfig(electionConfig)
.build();
}
public CompletableFuture<Void> goInactive() {
return server.goInactive();
}
/**
* Takes a snapshot of the partition server.
*
* @return a future to be completed once the snapshot has been taken
*/
public CompletableFuture<Void> snapshot() {
return server.compact();
}
public void setCompactableIndex(final long index) {
server.getContext().getLogCompactor().setCompactableIndex(index);
}
public RaftLogReader openReader() {
return server.getContext().getLog().openCommittedReader();
}
public void addRoleChangeListener(final RaftRoleChangeListener listener) {
if (server == null) {
deferredRoleChangeListeners.add(listener);
} else {
server.addRoleChangeListener(listener);
}
}
@Override
public HealthStatus getHealthStatus() {
return server.getContext().getHealthStatus();
}
@Override
public void addFailureListener(final FailureListener listener) {
if (server == null) {
deferredFailureListeners.add(listener);
} else {
server.addFailureListener(listener);
}
}
@Override
public void removeFailureListener(final FailureListener listener) {
deferredFailureListeners.remove(listener);
server.removeFailureListener(listener);
}
public void removeRoleChangeListener(final RaftRoleChangeListener listener) {
deferredRoleChangeListeners.remove(listener);
server.removeRoleChangeListener(listener);
}
/** @see io.atomix.raft.impl.RaftContext#addCommitListener(RaftCommitListener) */
public void addCommitListener(final RaftCommitListener commitListener) {
server.getContext().addCommitListener(commitListener);
}
/** @see io.atomix.raft.impl.RaftContext#removeCommitListener(RaftCommitListener) */
public void removeCommitListener(final RaftCommitListener commitListener) {
server.getContext().removeCommitListener(commitListener);
}
/** @see io.atomix.raft.impl.RaftContext#addCommittedEntryListener(RaftCommittedEntryListener) */
public void addCommittedEntryListener(final RaftCommittedEntryListener commitListener) {
server.getContext().addCommittedEntryListener(commitListener);
}
/**
* @see io.atomix.raft.impl.RaftContext#removeCommittedEntryListener(RaftCommittedEntryListener)
*/
public void removeCommittedEntryListener(final RaftCommittedEntryListener commitListener) {
server.getContext().removeCommittedEntryListener(commitListener);
}
/**
* @see
* io.atomix.raft.impl.RaftContext#addSnapshotReplicationListener(SnapshotReplicationListener)
*/
public void addSnapshotReplicationListener(final SnapshotReplicationListener listener) {
server.getContext().addSnapshotReplicationListener(listener);
}
/**
* @see
* io.atomix.raft.impl.RaftContext#removeSnapshotReplicationListener(SnapshotReplicationListener)
*/
public void removeSnapshotReplicationListener(final SnapshotReplicationListener listener) {
server.getContext().removeSnapshotReplicationListener(listener);
}
public PersistedSnapshotStore getPersistedSnapshotStore() {
return persistedSnapshotStore;
}
/** Deletes the server. */
public void delete() {
try {
Files.walkFileTree(
partition.dataDirectory().toPath(),
new SimpleFileVisitor<>() {
@Override
public FileVisitResult visitFile(final Path file, final BasicFileAttributes attrs)
throws IOException {
Files.delete(file);
return FileVisitResult.CONTINUE;
}
@Override
public FileVisitResult postVisitDirectory(final Path dir, final IOException exc)
throws IOException {
Files.delete(dir);
return FileVisitResult.CONTINUE;
}
});
} catch (final IOException e) {
log.error("Failed to delete partition: {}", partition, e);
}
}
public Optional<ZeebeLogAppender> getAppender() {
final RaftRole role = server.getContext().getRaftRole();
if (role instanceof ZeebeLogAppender) {
return Optional.of((ZeebeLogAppender) role);
}
return Optional.empty();
}
public Role getRole() {
return server.getRole();
}
public long getTerm() {
return server.getTerm();
}
private RaftStorage createRaftStorage() {
final RaftStorageConfig storageConfig = config.getStorageConfig();
return RaftStorage.builder()
.withPrefix(partition.name())
.withDirectory(partition.dataDirectory())
.withMaxSegmentSize((int) storageConfig.getSegmentSize().bytes())
.withFlushExplicitly(storageConfig.shouldFlushExplicitly())
.withFreeDiskSpace(storageConfig.getFreeDiskSpace())
.withSnapshotStore(persistedSnapshotStore)
.withJournalIndexDensity(storageConfig.getJournalIndexDensity())
.build();
}
private RaftServerCommunicator createServerProtocol() {
return new RaftServerCommunicator(
partition.name(),
Serializer.using(RaftNamespaces.RAFT_PROTOCOL),
clusterCommunicator,
requestTimeout);
}
public CompletableFuture<Void> stepDown() {
return server.stepDown();
}
public CompletableFuture<RaftServer> promote() {
return server.promote();
}
}