/
DefaultRaftOperationExecutor.java
247 lines (221 loc) · 7.75 KB
/
DefaultRaftOperationExecutor.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
/*
* Copyright 2017-present Open Networking Laboratory
*
* 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.OperationId;
import io.atomix.protocols.raft.OperationType;
import io.atomix.protocols.raft.RaftCommit;
import io.atomix.protocols.raft.RaftOperationExecutor;
import io.atomix.protocols.raft.error.ApplicationException;
import io.atomix.utils.concurrent.Scheduled;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.time.Duration;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Queue;
import java.util.function.Function;
import static com.google.common.base.Preconditions.checkNotNull;
import static com.google.common.base.Preconditions.checkState;
/**
* Default operation executor.
*/
public class DefaultRaftOperationExecutor implements RaftOperationExecutor {
private static final Logger LOGGER = LoggerFactory.getLogger(DefaultRaftOperationExecutor.class);
private final Queue<Runnable> tasks = new LinkedList<>();
private final List<ScheduledTask> scheduledTasks = new ArrayList<>();
private final List<ScheduledTask> complete = new ArrayList<>();
private final Map<OperationId, Function<RaftCommit<byte[]>, byte[]>> operations = new HashMap<>();
private OperationType operationType;
private long timestamp;
/**
* Sets the current operation type.
*
* @param commit the current commit
*/
private void prepareOperation(RaftCommit<byte[]> commit) {
long timestamp = commit.wallClockTime().unixTimestamp();
// Trigger scheduled tasks if this is a command and tasks are waiting to be executed.
if (commit.operation().type() == OperationType.COMMAND && !scheduledTasks.isEmpty()) {
// Iterate through scheduled tasks until we reach a task that has not met its scheduled time.
// The tasks list is sorted by time on insertion.
Iterator<ScheduledTask> iterator = scheduledTasks.iterator();
while (iterator.hasNext()) {
ScheduledTask task = iterator.next();
if (task.isRunnable(timestamp)) {
this.timestamp = task.time;
this.operationType = OperationType.COMMAND;
task.execute();
complete.add(task);
iterator.remove();
} else {
break;
}
}
// Iterate through tasks that were completed and reschedule them.
for (ScheduledTask task : complete) {
task.reschedule(this.timestamp);
}
complete.clear();
}
// Set the current operation type and timestamp.
this.operationType = commit.operation().type();
this.timestamp = timestamp;
}
/**
* Checks that the current operation is of the given type.
*
* @param type the operation type
* @param message the message to print if the current operation does not match the given type
*/
private void checkOperation(OperationType type, String message) {
checkState(operationType == type, message);
}
@Override
public void handle(OperationId operationId, Function<RaftCommit<byte[]>, byte[]> callback) {
checkNotNull(operationId, "operationId cannot be null");
checkNotNull(callback, "callback cannot be null");
operations.put(operationId, callback);
LOGGER.debug("Registered operation callback {}", operationId);
}
@Override
public byte[] apply(RaftCommit<byte[]> commit) {
prepareOperation(commit);
// Look up the registered callback for the operation.
Function<RaftCommit<byte[]>, byte[]> callback = operations.get(commit.operation());
if (callback == null) {
throw new IllegalStateException("Unknown state machine operation: " + commit.operation());
} else {
// Execute the operation. If the operation return value is a Future, await the result,
// otherwise immediately complete the execution future.
try {
return callback.apply(commit);
} catch (Exception e) {
LOGGER.warn("State machine operation failed: {}", e);
throw new ApplicationException(e, "An application error occurred");
} finally {
runTasks();
}
}
}
/**
* Executes tasks after an operation.
*/
private void runTasks() {
// Execute any tasks that were queue during execution of the command.
if (!tasks.isEmpty()) {
for (Runnable callback : tasks) {
callback.run();
}
tasks.clear();
}
}
@Override
public void execute(Runnable callback) {
checkOperation(OperationType.COMMAND, "callbacks can only be scheduled during command execution");
tasks.add(callback);
}
@Override
public Scheduled schedule(Duration delay, Runnable callback) {
checkOperation(OperationType.COMMAND, "callbacks can only be scheduled during command execution");
LOGGER.trace("Scheduled callback {} with delay {}", callback, delay);
return new ScheduledTask(callback, delay.toMillis()).schedule();
}
@Override
public Scheduled schedule(Duration initialDelay, Duration interval, Runnable callback) {
checkOperation(OperationType.COMMAND, "callbacks can only be scheduled during command execution");
LOGGER.trace("Scheduled repeating callback {} with initial delay {} and interval {}", callback, initialDelay, interval);
return new ScheduledTask(callback, initialDelay.toMillis(), interval.toMillis()).schedule();
}
/**
* Scheduled task.
*/
private class ScheduledTask implements Scheduled {
private final long interval;
private final Runnable callback;
private long time;
private ScheduledTask(Runnable callback, long delay) {
this(callback, delay, 0);
}
private ScheduledTask(Runnable callback, long delay, long interval) {
this.interval = interval;
this.callback = callback;
this.time = timestamp + delay;
}
/**
* Schedules the task.
*/
private Scheduled schedule() {
// Perform binary search to insert the task at the appropriate position in the tasks list.
if (scheduledTasks.isEmpty()) {
scheduledTasks.add(this);
} else {
int l = 0;
int u = scheduledTasks.size() - 1;
int i;
while (true) {
i = (u + l) / 2;
long t = scheduledTasks.get(i).time;
if (t == time) {
scheduledTasks.add(i, this);
return this;
} else if (t < time) {
l = i + 1;
if (l > u) {
scheduledTasks.add(i + 1, this);
return this;
}
} else {
u = i - 1;
if (l > u) {
scheduledTasks.add(i, this);
return this;
}
}
}
}
return this;
}
/**
* Reschedules the task.
*/
private void reschedule(long timestamp) {
if (interval > 0) {
time = timestamp + interval;
schedule();
}
}
/**
* Returns a boolean value indicating whether the task delay has been met.
*/
private boolean isRunnable(long timestamp) {
return timestamp > time;
}
/**
* Executes the task.
*/
private synchronized void execute() {
callback.run();
}
@Override
public synchronized void cancel() {
scheduledTasks.remove(this);
}
}
}