-
Notifications
You must be signed in to change notification settings - Fork 90
/
RaftGroupService.java
250 lines (228 loc) · 8.4 KB
/
RaftGroupService.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
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF 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.apache.ignite.internal.raft.service;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeoutException;
import org.apache.ignite.internal.raft.Command;
import org.apache.ignite.internal.raft.Peer;
import org.apache.ignite.internal.raft.PeersAndLearners;
import org.apache.ignite.internal.raft.configuration.RaftConfiguration;
import org.apache.ignite.internal.replicator.ReplicationGroupId;
import org.apache.ignite.network.ClusterService;
import org.jetbrains.annotations.Nullable;
import org.jetbrains.annotations.TestOnly;
/**
* A service providing operations on a replication group.
*
* <p>Most of operations require a known group leader. The group leader can be refreshed at any time by calling {@link #refreshLeader()}
* method, otherwise it will happen automatically on a first call.
*
* <p>If a leader has been changed while the operation in progress, the operation will be transparently retried until timeout is reached.
* The current leader will be refreshed automatically (maybe several times) in the process.
*
* <p>Each asynchronous method (returning a future) uses a default timeout to finish, see {@link RaftConfiguration#retryTimeout()}.
* If a result is not available within the timeout, the future will be completed with a {@link TimeoutException}
*
* <p>If an error is occurred during operation execution, the future will be completed with the corresponding IgniteException having an
* error code and a related message.
*
* <p>All async operations provided by the service are not cancellable.
*/
public interface RaftGroupService {
/**
* Returns group id.
*/
ReplicationGroupId groupId();
/**
* Returns current leader id or {@code null} if it has not been yet initialized.
*/
@Nullable Peer leader();
/**
* Returns a list of voting peers or {@code null} if it has not been yet initialized. The order is corresponding to the time of joining
* to the replication group.
*/
@Nullable List<Peer> peers();
/**
* Returns a list of leaners or {@code null} if it has not been yet initialized. The order is corresponding to the time of joining to
* the replication group.
*/
@Nullable List<Peer> learners();
/**
* Refreshes a replication group leader.
*
* <p>After the future completion the method {@link #leader()} can be used to retrieve a current group leader.
*
* <p>This operation is executed on a group leader.
*
* @return A future.
*/
CompletableFuture<Void> refreshLeader();
/**
* Refreshes a replication group leader and returns (leader, term) tuple.
*
* <p>This operation is executed on a group leader.
*
* @return A future, with (leader, term) tuple.
*/
CompletableFuture<LeaderWithTerm> refreshAndGetLeaderWithTerm();
/**
* Refreshes replication group members.
*
* <p>After the future completion methods like {@link #peers()} and {@link #learners()} can be used to retrieve current members of a
* group.
*
* <p>This operation is executed on a group leader.
*
* @param onlyAlive {@code True} to exclude dead nodes.
* @return A future.
*/
CompletableFuture<Void> refreshMembers(boolean onlyAlive);
/**
* Adds a voting peer to the replication group.
*
* <p>After the future completion methods like {@link #peers()} and {@link #learners()} can be used to retrieve current members of a
* group.
*
* <p>This operation is executed on a group leader.
*
* @param peer Peer
* @return A future.
*/
CompletableFuture<Void> addPeer(Peer peer);
/**
* Removes peer from the replication group.
*
* <p>After the future completion methods like {@link #peers()} and {@link #learners()} can be used to retrieve current members of a
* group.
*
* <p>This operation is executed on a group leader.
*
* @param peer Peer.
* @return A future.
*/
CompletableFuture<Void> removePeer(Peer peer);
/**
* Changes peers of a replication group.
*
* <p>After the future completion methods like {@link #peers()} and {@link #learners()} can be used to retrieve current members of a
* group.
*
* <p>This operation is executed on a group leader.
*
* @param peers Peers.
* @return A future.
*/
CompletableFuture<Void> changePeers(Collection<Peer> peers);
/**
* Changes peers and learners of a replication group.
*
* <p>Asynchronous variant of the previous method.
* When the future completed, it just means, that {@code changePeers} process has successfully started.
*
* <p>The results of rebalance itself will be processed by the listener of Raft reconfiguration event
* (from raft/server module).
*
* <p>This operation is executed on a group leader.
*
* @param peersAndLearners New peers and Learners of the Raft group.
* @param term Current known leader term.
* If real raft group term will be different - changePeers will be skipped.
* @return A future.
*/
CompletableFuture<Void> changePeersAsync(PeersAndLearners peersAndLearners, long term);
/**
* Adds learners (non-voting members).
*
* <p>After the future completion methods like {@link #peers()} and {@link #learners()} can be used to retrieve current members of a
* group.
*
* <p>This operation is executed on a group leader.
*
* @param learners Collection of learners.
* @return A future.
*/
CompletableFuture<Void> addLearners(Collection<Peer> learners);
/**
* Removes learners.
*
* <p>After the future completion methods like {@link #peers()} and {@link #learners()} can be used to retrieve current members of a
* group.
*
* <p>This operation is executed on a group leader.
*
* @param learners Collection of learners.
* @return A future.
*/
CompletableFuture<Void> removeLearners(Collection<Peer> learners);
/**
* Set learners of the raft group to needed list of learners.
*
* <p>After the future completion methods like {@link #peers()} and {@link #learners()} can be used to retrieve current members of a
* group.
*
* <p>This operation is executed on a group leader.
*
* @param learners Collection of learners.
* @return A future.
*/
CompletableFuture<Void> resetLearners(Collection<Peer> learners);
/**
* Takes a state machine snapshot on a given group peer.
*
* @param peer Peer.
* @return A future.
*/
CompletableFuture<Void> snapshot(Peer peer);
/**
* Transfers leadership to other peer.
*
* <p>This operation is executed on a group leader.
*
* @param newLeader New leader.
* @return A future.
*/
CompletableFuture<Void> transferLeadership(Peer newLeader);
/**
* Runs a command on a replication group leader.
*
* <p>Read commands always see up to date data.
*
* @param cmd The command.
* @param <R> Execution result type.
* @return A future with the execution result.
*/
<R> CompletableFuture<R> run(Command cmd);
/**
* Shutdown and cleanup resources for this instance.
*/
void shutdown();
/**
* Reads a metadata from leader.
*
* @return Future contains a leader metadata.
*/
CompletableFuture<LeaderMetadata> readLeaderMetadata();
/**
* Returns a cluster service.
*
* @return Cluster service.
*/
@TestOnly
ClusterService clusterService();
}