/
RaftJournalTest.java
611 lines (534 loc) · 25.7 KB
/
RaftJournalTest.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
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
/*
* The Alluxio Open Foundation licenses this work under the Apache License, version 2.0
* (the "License"). You may not use this work except in compliance with the License, which is
* available at www.apache.org/licenses/LICENSE-2.0
*
* This software is distributed on an "AS IS" basis, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND,
* either express or implied, as more fully set forth in the License.
*
* See the NOTICE file distributed with this work for information regarding copyright ownership.
*/
package alluxio.master.journal.raft;
import alluxio.ConfigurationRule;
import alluxio.conf.PropertyKey;
import alluxio.conf.ServerConfiguration;
import alluxio.grpc.QuorumServerInfo;
import alluxio.master.NoopMaster;
import alluxio.master.journal.CatchupFuture;
import alluxio.master.journal.JournalContext;
import alluxio.proto.journal.File;
import alluxio.proto.journal.Journal;
import alluxio.util.CommonUtils;
import alluxio.util.WaitForOptions;
import alluxio.util.network.NetworkAddressUtils;
import com.google.common.annotations.VisibleForTesting;
import org.apache.ratis.server.RaftServer;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import org.junit.rules.TemporaryFolder;
import java.io.Closeable;
import java.lang.reflect.Method;
import java.net.InetSocketAddress;
import java.net.ServerSocket;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ForkJoinPool;
import java.util.stream.Collectors;
public class RaftJournalTest {
@Rule
public TemporaryFolder mFolder = new TemporaryFolder();
@Rule
public ExpectedException mThrown = ExpectedException.none();
private RaftJournalSystem mLeaderJournalSystem;
private RaftJournalSystem mFollowerJournalSystem;
// A 30sec wait-options object for use by the test.
private WaitForOptions mWaitOptions = WaitForOptions.defaults().setTimeoutMs(30000);
@Before
public void before() throws Exception {
// Create and start journal systems.
List<RaftJournalSystem> journalSystems = startJournalCluster(createJournalSystems(2));
// Sleep for 2 leader election cycles for leadership to stabilize.
Thread
.sleep(2 * ServerConfiguration.getMs(PropertyKey.MASTER_EMBEDDED_JOURNAL_ELECTION_TIMEOUT));
// Assign references for leader/follower journal systems.
mLeaderJournalSystem = journalSystems.get(0);
mFollowerJournalSystem = journalSystems.get(1);
CommonUtils.waitFor("a leader is elected",
() -> mFollowerJournalSystem.isLeader() || mLeaderJournalSystem.isLeader(), mWaitOptions);
if (journalSystems.get(1).isLeader()) {
mLeaderJournalSystem = journalSystems.get(1);
mFollowerJournalSystem = journalSystems.get(0);
}
// Transition primary journal to primacy state.
mLeaderJournalSystem.gainPrimacy();
}
@After
public void after() throws Exception {
mLeaderJournalSystem.stop();
mFollowerJournalSystem.stop();
}
@Test
public void writeJournal() throws Exception {
// Create a counting master implementation that counts how many journal entries it processed.
CountingDummyFileSystemMaster countingMaster = new CountingDummyFileSystemMaster();
mFollowerJournalSystem.createJournal(countingMaster);
// Create entries on the leader journal context.
// These will be replicated to follower journal context.
final int entryCount = 10;
try (JournalContext journalContext =
mLeaderJournalSystem.createJournal(new NoopMaster()).createJournalContext()) {
for (int i = 0; i < entryCount; i++) {
journalContext.append(
alluxio.proto.journal.Journal.JournalEntry.newBuilder().setInodeLastModificationTime(
File.InodeLastModificationTimeEntry.newBuilder().setId(i).build()).build());
}
}
// Wait for sequences to be caught up.
CommonUtils.waitFor("full state acquired", () -> countingMaster.getApplyCount() == entryCount,
mWaitOptions);
}
@Test
public void joinCluster() throws Exception {
// Create entries on the leader journal context.
// These will be replicated to follower journal context.
final int entryCount = 10;
try (JournalContext journalContext =
mLeaderJournalSystem.createJournal(new NoopMaster()).createJournalContext()) {
for (int i = 0; i < entryCount; i++) {
journalContext.append(
alluxio.proto.journal.Journal.JournalEntry.newBuilder().setInodeLastModificationTime(
File.InodeLastModificationTimeEntry.newBuilder().setId(i).build()).build());
}
}
RaftJournalSystem newJs = createNewJournalSystem(mLeaderJournalSystem);
// Create a counting master implementation that counts how many journal entries it processed.
CountingDummyFileSystemMaster countingMaster = new CountingDummyFileSystemMaster();
newJs.createJournal(countingMaster);
newJs.start();
// Write more entries and validate they are replicated to follower.
try (JournalContext journalContext =
mLeaderJournalSystem.createJournal(new NoopMaster()).createJournalContext()) {
journalContext
.append(alluxio.proto.journal.Journal.JournalEntry.newBuilder()
.setInodeLastModificationTime(
File.InodeLastModificationTimeEntry.newBuilder().setId(entryCount).build())
.build());
}
CommonUtils.waitFor("follower catches up on all changes",
() -> countingMaster.getApplyCount() == entryCount + 1, mWaitOptions);
}
@Test
public void suspendCatchupResume() throws Exception {
// Create a counting master implementation that counts how many journal entries it processed.
CountingDummyFileSystemMaster countingMaster = new CountingDummyFileSystemMaster();
mFollowerJournalSystem.createJournal(countingMaster);
// Suspend follower journal system.
mFollowerJournalSystem.suspend(null);
try {
mFollowerJournalSystem.suspend(null);
Assert.fail("Suspend succeeded for already suspended journal.");
} catch (Exception e) {
// Expected to fail when suspending a suspended journal.
}
// Catch up follower journal system to target-index:5.
final long catchupIndex = 5;
Map<String, Long> backupSequences = new HashMap<>();
backupSequences.put("FileSystemMaster", catchupIndex);
CatchupFuture catchupFuture = mFollowerJournalSystem.catchup(backupSequences);
// Create entries on the leader journal context.
// These will be replicated to follower journal context.
final int entryCount = 10;
try (JournalContext journalContext =
mLeaderJournalSystem.createJournal(new NoopMaster()).createJournalContext()) {
for (int i = 0; i < entryCount; i++) {
journalContext.append(
alluxio.proto.journal.Journal.JournalEntry.newBuilder().setInodeLastModificationTime(
File.InodeLastModificationTimeEntry.newBuilder().setId(i).build()).build());
}
}
// Wait for sequences to be caught up.
catchupFuture.waitTermination();
Assert.assertEquals(catchupIndex + 1, countingMaster.getApplyCount());
// Wait for 2 heart-beat period and verify follower master state hasn't changed.
Thread.sleep(
2 * ServerConfiguration.getMs(PropertyKey.MASTER_EMBEDDED_JOURNAL_HEARTBEAT_INTERVAL));
Assert.assertEquals(catchupIndex + 1, countingMaster.getApplyCount());
// Exit backup mode and wait until follower master acquires the current knowledge.
mFollowerJournalSystem.resume();
CommonUtils.waitFor("full state acquired", () -> countingMaster.getApplyCount() == entryCount,
mWaitOptions);
// Write more entries and validate they are replicated to follower.
try (JournalContext journalContext =
mLeaderJournalSystem.createJournal(new NoopMaster()).createJournalContext()) {
journalContext
.append(alluxio.proto.journal.Journal.JournalEntry.newBuilder()
.setInodeLastModificationTime(
File.InodeLastModificationTimeEntry.newBuilder().setId(entryCount).build())
.build());
}
CommonUtils.waitFor("full state acquired after resume",
() -> countingMaster.getApplyCount() == entryCount + 1, mWaitOptions);
}
@Test
public void suspendSnapshotRestart() throws Exception {
// Create a counting master implementation that counts how many journal entries it processed.
CountingDummyFileSystemMaster countingMaster = new CountingDummyFileSystemMaster();
mFollowerJournalSystem.createJournal(countingMaster);
final int entryCount = 10;
try (JournalContext journalContext =
mLeaderJournalSystem.createJournal(new NoopMaster()).createJournalContext()) {
for (int i = 0; i < entryCount; i++) {
journalContext.append(
alluxio.proto.journal.Journal.JournalEntry.newBuilder().setInodeLastModificationTime(
File.InodeLastModificationTimeEntry.newBuilder().setId(i).build()).build());
}
}
// Suspend follower journal system.
mFollowerJournalSystem.suspend(null);
// Write more entries which are not applied due to suspension.
try (JournalContext journalContext =
mLeaderJournalSystem.createJournal(new NoopMaster()).createJournalContext()) {
journalContext
.append(alluxio.proto.journal.Journal.JournalEntry.newBuilder()
.setInodeLastModificationTime(
File.InodeLastModificationTimeEntry.newBuilder().setId(entryCount).build())
.build());
}
// Ask the follower to do a snapshot.
mFollowerJournalSystem.checkpoint();
// Restart the follower.
mFollowerJournalSystem.stop();
mFollowerJournalSystem.start();
Thread.sleep(
2 * ServerConfiguration.getMs(PropertyKey.MASTER_EMBEDDED_JOURNAL_HEARTBEAT_INTERVAL));
// Verify that all entries are replayed despite the snapshot was requested while some entries
// are queued up during suspension.
CommonUtils.waitFor("full state acquired after restart",
() -> countingMaster.getApplyCount() == entryCount + 1, mWaitOptions);
}
// Raft journal receives leader knowledge in chunks.
// So advancing should take into account seeing partial knowledge.
@Test
public void catchUpInSteps() throws Exception {
// Create a counting master implementation that counts how many journal entries it processed.
CountingDummyFileSystemMaster countingMaster = new CountingDummyFileSystemMaster();
mFollowerJournalSystem.createJournal(countingMaster);
// Suspend follower journal system.
mFollowerJournalSystem.suspend(null);
final int entryBatchCount = 5;
// Create batch of entries on the leader journal context.
try (JournalContext journalContext =
mLeaderJournalSystem.createJournal(new NoopMaster()).createJournalContext()) {
for (int i = 0; i < entryBatchCount; i++) {
journalContext.append(
alluxio.proto.journal.Journal.JournalEntry.newBuilder().setInodeLastModificationTime(
File.InodeLastModificationTimeEntry.newBuilder().setId(i).build()).build());
}
}
// Catch up follower journal system to target-index:(fileCount * 2) - 1.
Map<String, Long> backupSequences = new HashMap<>();
backupSequences.put("FileSystemMaster", (long) (entryBatchCount * 2) - 1);
CatchupFuture catchupFuture = mFollowerJournalSystem.catchup(backupSequences);
// Create next batch of entries on the leader journal context.
try (JournalContext journalContext =
mLeaderJournalSystem.createJournal(new NoopMaster()).createJournalContext()) {
for (int i = 0; i < entryBatchCount; i++) {
journalContext.append(
alluxio.proto.journal.Journal.JournalEntry.newBuilder().setInodeLastModificationTime(
File.InodeLastModificationTimeEntry.newBuilder().setId(i).build()).build());
}
}
// Wait for sequence to be caught up.
catchupFuture.waitTermination();
Assert.assertEquals(entryBatchCount * 2, countingMaster.getApplyCount());
// Catchup on the already met sequence.
mFollowerJournalSystem.catchup(backupSequences);
Assert.assertEquals(entryBatchCount * 2, countingMaster.getApplyCount());
}
@Test
public void subsequentCatchups() throws Exception {
// Create a counting master implementation that counts how many journal entries it processed.
CountingDummyFileSystemMaster countingMaster = new CountingDummyFileSystemMaster();
mFollowerJournalSystem.createJournal(countingMaster);
// Suspend follower journal system.
mFollowerJournalSystem.suspend(null);
final int entryBatchCount = 5;
// Create 2 batches of entries on the leader journal context.
try (JournalContext journalContext =
mLeaderJournalSystem.createJournal(new NoopMaster()).createJournalContext()) {
for (int i = 0; i < entryBatchCount * 2; i++) {
journalContext.append(
alluxio.proto.journal.Journal.JournalEntry.newBuilder().setInodeLastModificationTime(
File.InodeLastModificationTimeEntry.newBuilder().setId(i).build()).build());
}
}
Map<String, Long> backupSequences = new HashMap<>();
// Catch up follower journal system to first batch of entries.
backupSequences.put("FileSystemMaster", (long) entryBatchCount - 1);
mFollowerJournalSystem.catchup(backupSequences).waitTermination();
// Catch up follower journal system to second batch of entries.
backupSequences.put("FileSystemMaster", (long) (2 * entryBatchCount) - 1);
mFollowerJournalSystem.catchup(backupSequences).waitTermination();
// Verify master has caught up after advancing.
Assert.assertEquals(entryBatchCount * 2, countingMaster.getApplyCount());
}
@Test
public void gainPrimacyAfterSuspend() throws Exception {
// Create a counting master implementation that counts how many journal entries it processed.
CountingDummyFileSystemMaster countingMaster = new CountingDummyFileSystemMaster();
mFollowerJournalSystem.createJournal(countingMaster);
// Suspend follower journal system.
mFollowerJournalSystem.suspend(null);
// Create entries on the leader journal context.
// These will be replicated to follower journal context.
final int entryCount = 10;
try (JournalContext journalContext =
mLeaderJournalSystem.createJournal(new NoopMaster()).createJournalContext()) {
for (int i = 0; i < entryCount; i++) {
journalContext.append(
alluxio.proto.journal.Journal.JournalEntry.newBuilder().setInodeLastModificationTime(
File.InodeLastModificationTimeEntry.newBuilder().setId(i).build()).build());
}
}
// Assert that no entries applied by suspended journal system.
Assert.assertEquals(0, countingMaster.getApplyCount());
// Gain primacy in follower journal and validate it catches up.
promoteFollower();
CommonUtils.waitFor(
"full state acquired after resume", () -> mFollowerJournalSystem.getCurrentSequenceNumbers()
.values().stream().distinct().collect(Collectors.toList()).get(0) == entryCount - 1,
mWaitOptions);
// Follower should no longer be suspended after becoming primary.
Assert.assertFalse(mFollowerJournalSystem.isSuspended());
}
@Test
public void gainPrimacyAfterCatchup() throws Exception {
// Create a counting master implementation that counts how many journal entries it processed.
CountingDummyFileSystemMaster countingMaster = new CountingDummyFileSystemMaster();
mFollowerJournalSystem.createJournal(countingMaster);
// Suspend follower journal system.
mFollowerJournalSystem.suspend(null);
// Catch up follower journal system to target-index:5.
final long catchupIndex = 5;
Map<String, Long> backupSequences = new HashMap<>();
backupSequences.put("FileSystemMaster", catchupIndex);
CatchupFuture catchupFuture = mFollowerJournalSystem.catchup(backupSequences);
// Create entries on the leader journal context.
// These will be replicated to follower journal context.
final int entryCount = 10;
try (JournalContext journalContext =
mLeaderJournalSystem.createJournal(new NoopMaster()).createJournalContext()) {
for (int i = 0; i < entryCount; i++) {
journalContext.append(
alluxio.proto.journal.Journal.JournalEntry.newBuilder().setInodeLastModificationTime(
File.InodeLastModificationTimeEntry.newBuilder().setId(i).build()).build());
}
}
// Wait until caught up.
catchupFuture.waitTermination();
Assert.assertEquals(catchupIndex + 1, countingMaster.getApplyCount());
// Gain primacy in follower journal and validate it catches up.
promoteFollower();
CommonUtils.waitFor("full state acquired after resume",
() -> countingMaster.getApplyCount() == entryCount, mWaitOptions);
// Follower should no longer be suspended after becoming primary.
Assert.assertFalse(mFollowerJournalSystem.isSuspended());
}
private void promoteFollower() throws Exception {
System.out.printf("Leader is leader? %s", mLeaderJournalSystem.isLeader());
changeToFollower(mLeaderJournalSystem);
System.out.printf("Follower is leader? %s", mFollowerJournalSystem.isLeader());
changeToFollower(mLeaderJournalSystem);
changeToCandidate(mFollowerJournalSystem);
CommonUtils.waitFor("follower becomes leader",
() -> mFollowerJournalSystem.isLeader(), mWaitOptions);
mFollowerJournalSystem.gainPrimacy();
}
@Test
public void gainPrimacyDuringCatchup() throws Exception {
// TODO(feng): remove this test when remote journal write is deprecated
after();
try (Closeable r = new ConfigurationRule(
PropertyKey.MASTER_EMBEDDED_JOURNAL_WRITE_REMOTE_ENABLED, "true",
ServerConfiguration.global()).toResource()) {
before();
// Create a counting master implementation that counts how many journal entries it processed.
CountingDummyFileSystemMaster countingMaster = new CountingDummyFileSystemMaster();
mFollowerJournalSystem.createJournal(countingMaster);
// Using a large entry count for catching transition while in-progress.
final int entryCount = 100000;
// Suspend follower journal system.
mFollowerJournalSystem.suspend(null);
// Catch up follower journal to a large index to be able to transition while in progress.
final long catchupIndex = entryCount - 5;
Map<String, Long> backupSequences = new HashMap<>();
backupSequences.put("FileSystemMaster", catchupIndex);
CatchupFuture catchupFuture = mFollowerJournalSystem.catchup(backupSequences);
// Create entries in parallel on the leader journal context.
// These will be replicated to follower journal context.
ForkJoinPool.commonPool().submit(() -> {
try (JournalContext journalContext =
mLeaderJournalSystem.createJournal(new NoopMaster()).createJournalContext()) {
for (int i = 0; i < entryCount; i++) {
journalContext
.append(
alluxio.proto.journal.Journal.JournalEntry.newBuilder()
.setInodeLastModificationTime(
File.InodeLastModificationTimeEntry.newBuilder().setId(i).build())
.build());
}
} catch (Exception e) {
Assert.fail(String.format("Failed while writing entries: %s", e.toString()));
}
});
// Wait until advancing starts.
CommonUtils.waitFor("Advancing to start.", () -> countingMaster.getApplyCount() > 0,
mWaitOptions);
// Gain primacy in follower journal and validate it catches up.
mLeaderJournalSystem.notifyLeadershipStateChanged(false);
mFollowerJournalSystem.notifyLeadershipStateChanged(true);
mFollowerJournalSystem.gainPrimacy();
// Can't use countingMaster because Raft stops applying entries for primary journals.
// Using JournalSystem#getCurrentSequences() API instead.
CommonUtils.waitFor(
"full state acquired after resume",
() -> mFollowerJournalSystem.getCurrentSequenceNumbers()
.values().stream().distinct().collect(Collectors.toList()).get(0) == entryCount - 1,
mWaitOptions);
// Follower should no longer be suspended after becoming primary.
Assert.assertFalse(mFollowerJournalSystem.isSuspended());
}
}
/**
* Creates list of raft journal systems in a clustered mode.
*/
private List<RaftJournalSystem> createJournalSystems(int journalSystemCount) throws Exception {
// Override defaults for faster quorum formation.
ServerConfiguration.set(PropertyKey.MASTER_EMBEDDED_JOURNAL_ELECTION_TIMEOUT, 550);
ServerConfiguration.set(PropertyKey.MASTER_EMBEDDED_JOURNAL_HEARTBEAT_INTERVAL, 250);
List<InetSocketAddress> clusterAddresses = new ArrayList<>(journalSystemCount);
List<Integer> freePorts = getFreePorts(journalSystemCount);
for (int i = 0; i < journalSystemCount; i++) {
clusterAddresses.add(InetSocketAddress.createUnresolved("localhost", freePorts.get(i)));
}
List<RaftJournalSystem> journalSystems = new ArrayList<>(journalSystemCount);
for (int i = 0; i < journalSystemCount; i++) {
journalSystems.add(RaftJournalSystem.create(RaftJournalConfiguration
.defaults(NetworkAddressUtils.ServiceType.MASTER_RAFT).setPath(mFolder.newFolder())
.setClusterAddresses(clusterAddresses).setLocalAddress(clusterAddresses.get(i))));
}
return journalSystems;
}
/**
* Creates list of raft journal systems in a clustered mode.
*/
private RaftJournalSystem createNewJournalSystem(RaftJournalSystem seed) throws Exception {
List<InetSocketAddress> clusterAddresses = seed.getQuorumServerInfoList().stream()
.map(QuorumServerInfo::getServerAddress)
.map(address -> InetSocketAddress.createUnresolved(address.getHost(), address.getRpcPort()))
.collect(Collectors.toList());
List<Integer> freePorts = getFreePorts(1);
InetSocketAddress joinAddr = InetSocketAddress.createUnresolved("localhost", freePorts.get(0));
clusterAddresses.add(joinAddr);
return RaftJournalSystem.create(RaftJournalConfiguration
.defaults(NetworkAddressUtils.ServiceType.MASTER_RAFT).setPath(mFolder.newFolder())
.setClusterAddresses(clusterAddresses).setLocalAddress(joinAddr));
}
/**
* Starts given journal systems asynchronously and waits until complete.
*/
private List<RaftJournalSystem> startJournalCluster(List<RaftJournalSystem> journalSystems)
throws Exception {
List<CompletableFuture<?>> futures = new LinkedList<>();
for (RaftJournalSystem js : journalSystems) {
futures.add(CompletableFuture.runAsync(() -> {
try {
js.start();
} catch (Exception e) {
throw new RuntimeException("Failed to start journal system.", e);
}
}));
}
CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])).get();
return journalSystems;
}
@VisibleForTesting
void changeToCandidate(RaftJournalSystem journalSystem) throws Exception {
RaftServer.Division serverImpl = journalSystem.getRaftServer()
.getDivision(RaftJournalSystem.RAFT_GROUP_ID);
Class<?> raftServerImpl = (Class.forName("org.apache.ratis.server.impl.RaftServerImpl"));
Method method = raftServerImpl.getDeclaredMethod("changeToCandidate", boolean.class);
method.setAccessible(true);
method.invoke(serverImpl, false);
}
@VisibleForTesting
void changeToFollower(RaftJournalSystem journalSystem) throws Exception {
RaftServer.Division serverImplObj = journalSystem.getRaftServer()
.getDivision(RaftJournalSystem.RAFT_GROUP_ID);
Class<?> raftServerImplClass = Class.forName("org.apache.ratis.server.impl.RaftServerImpl");
Method getStateMethod = raftServerImplClass.getDeclaredMethod("getState");
getStateMethod.setAccessible(true);
Object serverStateObj = getStateMethod.invoke(serverImplObj);
Class<?> serverStateClass = Class.forName("org.apache.ratis.server.impl.ServerState");
Method getCurrentTermMethod = serverStateClass.getDeclaredMethod("getCurrentTerm");
getCurrentTermMethod.setAccessible(true);
long currentTermObj = (long) getCurrentTermMethod.invoke(serverStateObj);
Method changeToFollowerMethod = raftServerImplClass.getDeclaredMethod("changeToFollower",
long.class, boolean.class, Object.class);
changeToFollowerMethod.setAccessible(true);
changeToFollowerMethod.invoke(serverImplObj, currentTermObj, true, "test");
}
/**
* @return a list of free ports
*/
private List<Integer> getFreePorts(int portCount) throws Exception {
List<ServerSocket> sockets = new ArrayList<>(portCount);
for (int i = 0; i < portCount; i++) {
sockets.add(new ServerSocket(0));
}
List<Integer> ports = new ArrayList<>(portCount);
for (ServerSocket socket : sockets) {
ports.add(socket.getLocalPort());
socket.close();
}
return ports;
}
/**
* Used to validate journal apply counts to master.
*/
class CountingDummyFileSystemMaster extends NoopMaster {
/** Tracks how many entries have been applied to master. */
private long mApplyCount = 0;
@Override
public boolean processJournalEntry(Journal.JournalEntry entry) {
mApplyCount++;
return true;
}
@Override
public void resetState() {
mApplyCount = 0;
}
/**
* @return how many entries are applied
*/
public long getApplyCount() {
return mApplyCount;
}
@Override
public String getName() {
/**
* RaftJournalWriter doesn't accept empty journal entries. FileSystemMaster is returned here
* according to injected entry type during the test.
*/
return "FileSystemMaster";
}
}
}