forked from apache/ratis
/
DataStreamBaseTest.java
402 lines (345 loc) · 12.5 KB
/
DataStreamBaseTest.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
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
/*
* 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.ratis.datastream;
import org.apache.ratis.BaseTest;
import org.apache.ratis.MiniRaftCluster;
import org.apache.ratis.client.impl.DataStreamClientImpl;
import org.apache.ratis.client.impl.DataStreamClientImpl.DataStreamOutputImpl;
import org.apache.ratis.conf.RaftProperties;
import org.apache.ratis.proto.RaftProtos.*;
import org.apache.ratis.protocol.DataStreamReply;
import org.apache.ratis.protocol.GroupInfoReply;
import org.apache.ratis.protocol.GroupInfoRequest;
import org.apache.ratis.protocol.GroupListReply;
import org.apache.ratis.protocol.GroupListRequest;
import org.apache.ratis.protocol.GroupManagementRequest;
import org.apache.ratis.protocol.RaftClientReply;
import org.apache.ratis.protocol.RaftClientRequest;
import org.apache.ratis.protocol.RaftGroup;
import org.apache.ratis.protocol.RaftGroupId;
import org.apache.ratis.protocol.RaftPeer;
import org.apache.ratis.protocol.RaftPeerId;
import org.apache.ratis.proto.RaftProtos.DataStreamPacketHeaderProto.Type;
import org.apache.ratis.protocol.SetConfigurationRequest;
import org.apache.ratis.rpc.RpcType;
import org.apache.ratis.server.DataStreamServerRpc;
import org.apache.ratis.server.RaftServer;
import org.apache.ratis.server.impl.DataStreamServerImpl;
import org.apache.ratis.server.impl.ServerFactory;
import org.apache.ratis.statemachine.StateMachine;
import org.apache.ratis.statemachine.impl.BaseStateMachine;
import org.apache.ratis.statemachine.StateMachine.DataStream;
import org.apache.ratis.util.JavaUtils;
import org.apache.ratis.util.LifeCycle;
import org.apache.ratis.util.NetUtils;
import org.junit.Assert;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.WritableByteChannel;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ThreadLocalRandom;
import java.util.stream.Collectors;
abstract class DataStreamBaseTest extends BaseTest {
static final int MODULUS = 23;
static byte pos2byte(int pos) {
return (byte) ('A' + pos%MODULUS);
}
static class MultiDataStreamStateMachine extends BaseStateMachine {
final ConcurrentMap<Long, SingleDataStream> streams = new ConcurrentHashMap<>();
@Override
public CompletableFuture<DataStream> stream(RaftClientRequest request) {
final SingleDataStream s = new SingleDataStream();
streams.put(request.getCallId(), s);
return s.stream(request);
}
SingleDataStream getSingleDataStream(long callId) {
return streams.get(callId);
}
}
static class SingleDataStream {
private int byteWritten = 0;
private RaftClientRequest writeRequest;
final WritableByteChannel channel = new WritableByteChannel() {
private volatile boolean open = true;
@Override
public int write(ByteBuffer src) {
if (!open) {
throw new IllegalStateException("Already closed");
}
final int remaining = src.remaining();
for(; src.remaining() > 0; ) {
Assert.assertEquals(pos2byte(byteWritten), src.get());
byteWritten += 1;
}
return remaining;
}
@Override
public boolean isOpen() {
return open;
}
@Override
public void close() {
open = false;
}
};
final DataStream stream = new DataStream() {
@Override
public WritableByteChannel getWritableByteChannel() {
return channel;
}
@Override
public CompletableFuture<?> cleanUp() {
try {
channel.close();
} catch (Throwable t) {
return JavaUtils.completeExceptionally(t);
}
return CompletableFuture.completedFuture(null);
}
};
public CompletableFuture<DataStream> stream(RaftClientRequest request) {
writeRequest = request;
return CompletableFuture.completedFuture(stream);
}
public int getByteWritten() {
return byteWritten;
}
public RaftClientRequest getWriteRequest() {
return writeRequest;
}
}
protected RaftProperties properties;
private List<DataStreamServerImpl> servers;
private List<RaftPeer> peers;
private ConcurrentMap<RaftGroupId, MultiDataStreamStateMachine> stateMachines;
protected RaftServer newRaftServer(RaftPeer peer, RaftProperties properties) {
return new RaftServer() {
@Override
public RaftPeerId getId() {
return peer.getId();
}
@Override
public StateMachine getStateMachine(RaftGroupId groupId) {
return stateMachines.computeIfAbsent(groupId, key -> new MultiDataStreamStateMachine());
}
@Override
public RaftProperties getProperties() {
return properties;
}
@Override
public RequestVoteReplyProto requestVote(RequestVoteRequestProto request) {
return null;
}
@Override
public AppendEntriesReplyProto appendEntries(AppendEntriesRequestProto request) {
return null;
}
@Override
public InstallSnapshotReplyProto installSnapshot(InstallSnapshotRequestProto request) {
return null;
}
@Override
public CompletableFuture<AppendEntriesReplyProto> appendEntriesAsync(AppendEntriesRequestProto request) {
return null;
}
@Override
public RpcType getRpcType() {
return null;
}
@Override
public RaftClientReply submitClientRequest(RaftClientRequest request) {
return null;
}
@Override
public RaftClientReply setConfiguration(SetConfigurationRequest request) {
return null;
}
@Override
public CompletableFuture<RaftClientReply> submitClientRequestAsync(RaftClientRequest request) {
return null;
}
@Override
public CompletableFuture<RaftClientReply> setConfigurationAsync(SetConfigurationRequest request) {
return null;
}
@Override
public GroupListReply getGroupList(GroupListRequest request) {
return null;
}
@Override
public GroupInfoReply getGroupInfo(GroupInfoRequest request) {
return null;
}
@Override
public RaftClientReply groupManagement(GroupManagementRequest request) {
return null;
}
@Override
public CompletableFuture<GroupListReply> getGroupListAsync(GroupListRequest request) {
return null;
}
@Override
public CompletableFuture<GroupInfoReply> getGroupInfoAsync(GroupInfoRequest request) {
return null;
}
@Override
public CompletableFuture<RaftClientReply> groupManagementAsync(GroupManagementRequest request) {
return null;
}
@Override
public void close() {
}
@Override
public Iterable<RaftGroupId> getGroupIds() {
return null;
}
@Override
public Iterable<RaftGroup> getGroups() {
return null;
}
@Override
public ServerFactory getFactory() {
return null;
}
@Override
public void start() {
}
@Override
public LifeCycle.State getLifeCycleState() {
return null;
}
};
}
protected void setup(int numServers){
peers = Arrays.stream(MiniRaftCluster.generateIds(numServers, 0))
.map(RaftPeerId::valueOf)
.map(id -> new RaftPeer(id, NetUtils.createLocalServerAddress()))
.collect(Collectors.toList());
servers = new ArrayList<>(peers.size());
stateMachines = new ConcurrentHashMap<>();
// start stream servers on raft peers.
for (int i = 0; i < peers.size(); i++) {
final RaftPeer peer = peers.get(i);
final RaftServer server = newRaftServer(peer, properties);
final DataStreamServerImpl streamServer = new DataStreamServerImpl(server, properties, null);
final DataStreamServerRpc rpc = streamServer.getServerRpc();
if (i == 0) {
// only the first server routes requests to peers.
List<RaftPeer> otherPeers = new ArrayList<>(peers);
otherPeers.remove(peers.get(i));
rpc.addRaftPeers(otherPeers);
}
rpc.start();
servers.add(streamServer);
}
}
DataStreamClientImpl newDataStreamClientImpl() {
return new DataStreamClientImpl(peers.get(0), properties, null);
}
protected void shutdown() throws IOException {
for (DataStreamServerImpl server : servers) {
server.close();
}
}
protected void runTestDataStream(int numServers, int numClients, int numStreams, int bufferSize, int bufferNum)
throws Exception {
try {
setup(numServers);
runTestDataStream(numClients, numStreams, bufferSize, bufferNum);
} finally {
shutdown();
}
}
private void runTestDataStream(int numClients, int numStreams, int bufferSize, int bufferNum) throws Exception {
final List<CompletableFuture<Void>> futures = new ArrayList<>();
final List<DataStreamClientImpl> clients = new ArrayList<>();
try {
for (int j = 0; j < numClients; j++) {
final DataStreamClientImpl client = newDataStreamClientImpl();
clients.add(client);
for (int i = 0; i < numStreams; i++) {
futures.add(CompletableFuture.runAsync(
() -> runTestDataStream((DataStreamOutputImpl) client.stream(), bufferSize, bufferNum)));
}
}
Assert.assertEquals(numClients*numStreams, futures.size());
futures.forEach(CompletableFuture::join);
} finally {
for (int j = 0; j < numClients; j++) {
clients.get(j).close();
}
}
}
private void runTestDataStream(DataStreamOutputImpl out, int bufferSize, int bufferNum) {
final List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
final List<Integer> sizes = new ArrayList<>();
//send data
final int halfBufferSize = bufferSize/2;
int dataSize = 0;
for(int i = 0; i < bufferNum; i++) {
final int size = halfBufferSize + ThreadLocalRandom.current().nextInt(halfBufferSize);
sizes.add(size);
final ByteBuffer bf = initBuffer(dataSize, size);
futures.add(out.writeAsync(bf));
dataSize += size;
}
{ // check header
final DataStreamReply reply = out.getHeaderFuture().join();
Assert.assertTrue(reply.isSuccess());
Assert.assertEquals(0, reply.getBytesWritten());
Assert.assertEquals(reply.getType(), Type.STREAM_HEADER);
}
// check writeAsync requests
for(int i = 0; i < futures.size(); i++) {
final DataStreamReply reply = futures.get(i).join();
Assert.assertTrue(reply.isSuccess());
Assert.assertEquals(sizes.get(i).longValue(), reply.getBytesWritten());
Assert.assertEquals(reply.getType(), Type.STREAM_DATA);
}
final RaftClientRequest header = out.getHeader();
for (MultiDataStreamStateMachine s : stateMachines.values()) {
final SingleDataStream stream = s.getSingleDataStream(header.getCallId());
if (stream == null) {
continue;
}
final RaftClientRequest writeRequest = stream.getWriteRequest();
if (writeRequest.getClientId().equals(header.getClientId())) {
Assert.assertEquals(writeRequest.getCallId(), header.getCallId());
Assert.assertEquals(writeRequest.getRaftGroupId(), header.getRaftGroupId());
Assert.assertEquals(writeRequest.getServerId(), header.getServerId());
}
Assert.assertEquals(dataSize, stream.getByteWritten());
}
}
static ByteBuffer initBuffer(int offset, int size) {
final ByteBuffer buffer = ByteBuffer.allocateDirect(size);
final int length = buffer.capacity();
buffer.position(0).limit(length);
for (int j = 0; j < length; j++) {
buffer.put(pos2byte(offset + j));
}
buffer.flip();
Assert.assertEquals(length, buffer.remaining());
return buffer;
}
}