45d625768940780624e476643c1f3faed141b609
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / AbstractShardTest.java
1 /*
2  * Copyright (c) 2015 Brocade Communications Systems, Inc. and others.  All rights reserved.
3  *
4  * This program and the accompanying materials are made available under the
5  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6  * and is available at http://www.eclipse.org/legal/epl-v10.html
7  */
8 package org.opendaylight.controller.cluster.datastore;
9
10 import static org.junit.Assert.assertEquals;
11 import static org.junit.Assert.assertNotNull;
12 import static org.junit.Assert.assertTrue;
13 import static org.junit.Assert.fail;
14 import static org.mockito.Matchers.any;
15 import static org.mockito.Mockito.doAnswer;
16 import static org.mockito.Mockito.doReturn;
17 import static org.mockito.Mockito.mock;
18 import static org.opendaylight.controller.cluster.datastore.DataStoreVersions.CURRENT_VERSION;
19 import static org.opendaylight.controller.cluster.datastore.ShardDataTreeMocking.successfulCanCommit;
20 import static org.opendaylight.controller.cluster.datastore.ShardDataTreeMocking.successfulCommit;
21 import static org.opendaylight.controller.cluster.datastore.ShardDataTreeMocking.successfulPreCommit;
22
23 import akka.actor.ActorRef;
24 import akka.actor.PoisonPill;
25 import akka.actor.Props;
26 import akka.dispatch.Dispatchers;
27 import akka.japi.Creator;
28 import akka.pattern.Patterns;
29 import akka.testkit.TestActorRef;
30 import akka.util.Timeout;
31 import com.google.common.primitives.UnsignedLong;
32 import com.google.common.util.concurrent.FutureCallback;
33 import com.google.common.util.concurrent.Uninterruptibles;
34 import java.io.IOException;
35 import java.util.Collections;
36 import java.util.HashMap;
37 import java.util.Map;
38 import java.util.Optional;
39 import java.util.Set;
40 import java.util.SortedSet;
41 import java.util.concurrent.CountDownLatch;
42 import java.util.concurrent.ExecutionException;
43 import java.util.concurrent.TimeUnit;
44 import java.util.concurrent.TimeoutException;
45 import java.util.concurrent.atomic.AtomicInteger;
46 import org.junit.After;
47 import org.junit.Assert;
48 import org.junit.Before;
49 import org.opendaylight.controller.cluster.access.concepts.MemberName;
50 import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
51 import org.opendaylight.controller.cluster.datastore.DatastoreContext.Builder;
52 import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier;
53 import org.opendaylight.controller.cluster.datastore.messages.BatchedModifications;
54 import org.opendaylight.controller.cluster.datastore.messages.ForwardedReadyTransaction;
55 import org.opendaylight.controller.cluster.datastore.modification.MergeModification;
56 import org.opendaylight.controller.cluster.datastore.modification.MutableCompositeModification;
57 import org.opendaylight.controller.cluster.datastore.modification.WriteModification;
58 import org.opendaylight.controller.cluster.datastore.persisted.CommitTransactionPayload;
59 import org.opendaylight.controller.cluster.datastore.persisted.MetadataShardDataTreeSnapshot;
60 import org.opendaylight.controller.cluster.datastore.persisted.ShardSnapshotState;
61 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
62 import org.opendaylight.controller.cluster.raft.TestActorFactory;
63 import org.opendaylight.controller.cluster.raft.persisted.Snapshot;
64 import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal;
65 import org.opendaylight.controller.cluster.raft.utils.InMemorySnapshotStore;
66 import org.opendaylight.controller.md.cluster.datastore.model.CarsModel;
67 import org.opendaylight.controller.md.cluster.datastore.model.TestModel;
68 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
69 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.PathArgument;
70 import org.opendaylight.yangtools.yang.data.api.schema.DataContainerChild;
71 import org.opendaylight.yangtools.yang.data.api.schema.MapEntryNode;
72 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
73 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTree;
74 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidate;
75 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidateNode;
76 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidateTip;
77 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeConfiguration;
78 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeModification;
79 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataValidationFailedException;
80 import org.opendaylight.yangtools.yang.data.api.schema.tree.ModificationType;
81 import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
82 import org.opendaylight.yangtools.yang.data.impl.schema.tree.InMemoryDataTreeFactory;
83 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
84 import scala.concurrent.Await;
85 import scala.concurrent.Future;
86 import scala.concurrent.duration.Duration;
87
88 /**
89  * Abstract base for shard unit tests.
90  *
91  * @author Thomas Pantelis
92  */
93 public abstract class AbstractShardTest extends AbstractActorTest {
94     protected static final SchemaContext SCHEMA_CONTEXT = TestModel.createTestContext();
95
96     private static final AtomicInteger NEXT_SHARD_NUM = new AtomicInteger();
97
98     protected static final int HEARTBEAT_MILLIS = 100;
99
100     protected final ShardIdentifier shardID = ShardIdentifier.create("inventory", MemberName.forName("member-1"),
101         "config" + NEXT_SHARD_NUM.getAndIncrement());
102
103     protected final Builder dataStoreContextBuilder = DatastoreContext.newBuilder()
104             .shardJournalRecoveryLogBatchSize(3).shardSnapshotBatchCount(5000)
105             .shardHeartbeatIntervalInMillis(HEARTBEAT_MILLIS);
106
107     protected final TestActorFactory actorFactory = new TestActorFactory(getSystem());
108
109     @Before
110     public void setUp() {
111         InMemorySnapshotStore.clear();
112         InMemoryJournal.clear();
113     }
114
115     @After
116     public void tearDown() {
117         InMemorySnapshotStore.clear();
118         InMemoryJournal.clear();
119         actorFactory.close();
120     }
121
122     protected DatastoreContext newDatastoreContext() {
123         return dataStoreContextBuilder.build();
124     }
125
126     protected Props newShardProps() {
127         return newShardBuilder().props();
128     }
129
130     protected Shard.Builder newShardBuilder() {
131         return Shard.builder().id(shardID).datastoreContext(newDatastoreContext())
132             .schemaContextProvider(() -> SCHEMA_CONTEXT);
133     }
134
135     protected void testRecovery(final Set<Integer> listEntryKeys) throws Exception {
136         // Create the actor and wait for recovery complete.
137
138         final int nListEntries = listEntryKeys.size();
139
140         final CountDownLatch recoveryComplete = new CountDownLatch(1);
141
142         final Creator<Shard> creator = () -> new Shard(newShardBuilder()) {
143             @Override
144             protected void onRecoveryComplete() {
145                 try {
146                     super.onRecoveryComplete();
147                 } finally {
148                     recoveryComplete.countDown();
149                 }
150             }
151         };
152
153         final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
154                 Props.create(new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()),
155                 "testRecovery");
156
157         assertEquals("Recovery complete", true, recoveryComplete.await(5, TimeUnit.SECONDS));
158
159         // Verify data in the data store.
160
161         final NormalizedNode<?, ?> outerList = readStore(shard, TestModel.OUTER_LIST_PATH);
162         assertNotNull(TestModel.OUTER_LIST_QNAME.getLocalName() + " not found", outerList);
163         assertTrue(TestModel.OUTER_LIST_QNAME.getLocalName() + " value is not Iterable",
164                 outerList.getValue() instanceof Iterable);
165         for (final Object entry: (Iterable<?>) outerList.getValue()) {
166             assertTrue(TestModel.OUTER_LIST_QNAME.getLocalName() + " entry is not MapEntryNode",
167                     entry instanceof MapEntryNode);
168             final MapEntryNode mapEntry = (MapEntryNode)entry;
169             final Optional<DataContainerChild<? extends PathArgument, ?>> idLeaf =
170                     mapEntry.getChild(new YangInstanceIdentifier.NodeIdentifier(TestModel.ID_QNAME));
171             assertTrue("Missing leaf " + TestModel.ID_QNAME.getLocalName(), idLeaf.isPresent());
172             final Object value = idLeaf.get().getValue();
173             assertTrue("Unexpected value for leaf " + TestModel.ID_QNAME.getLocalName() + ": " + value,
174                     listEntryKeys.remove(value));
175         }
176
177         if (!listEntryKeys.isEmpty()) {
178             fail("Missing " + TestModel.OUTER_LIST_QNAME.getLocalName() + " entries with keys: " + listEntryKeys);
179         }
180
181         assertEquals("Last log index", nListEntries,
182                 shard.underlyingActor().getShardMBean().getLastLogIndex());
183         assertEquals("Commit index", nListEntries,
184                 shard.underlyingActor().getShardMBean().getCommitIndex());
185         assertEquals("Last applied", nListEntries,
186                 shard.underlyingActor().getShardMBean().getLastApplied());
187
188         shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
189     }
190
191     protected void verifyLastApplied(final TestActorRef<Shard> shard, final long expectedValue) {
192         long lastApplied = -1;
193         for (int i = 0; i < 20 * 5; i++) {
194             lastApplied = shard.underlyingActor().getShardMBean().getLastApplied();
195             if (lastApplied == expectedValue) {
196                 return;
197             }
198             Uninterruptibles.sleepUninterruptibly(50, TimeUnit.MILLISECONDS);
199         }
200
201         Assert.fail(String.format("Expected last applied: %d, Actual: %d", expectedValue, lastApplied));
202     }
203
204     protected DataTree createDelegatingMockDataTree() throws Exception {
205         final DataTree actual = new InMemoryDataTreeFactory().create(DataTreeConfiguration.DEFAULT_CONFIGURATION);
206         final DataTree mock = mock(DataTree.class);
207
208         doAnswer(invocation -> {
209             actual.validate(invocation.getArgumentAt(0, DataTreeModification.class));
210             return null;
211         }).when(mock).validate(any(DataTreeModification.class));
212
213         doAnswer(invocation -> actual.prepare(invocation.getArgumentAt(0, DataTreeModification.class))).when(
214                 mock).prepare(any(DataTreeModification.class));
215
216         doAnswer(invocation -> {
217             actual.commit(invocation.getArgumentAt(0, DataTreeCandidate.class));
218             return null;
219         }).when(mock).commit(any(DataTreeCandidate.class));
220
221         doAnswer(invocation -> {
222             actual.setSchemaContext(invocation.getArgumentAt(0, SchemaContext.class));
223             return null;
224         }).when(mock).setSchemaContext(any(SchemaContext.class));
225
226         doAnswer(invocation -> actual.takeSnapshot()).when(mock).takeSnapshot();
227
228         doAnswer(invocation -> actual.getRootPath()).when(mock).getRootPath();
229
230         return mock;
231     }
232
233     protected ShardDataTreeCohort mockShardDataTreeCohort() {
234         ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class);
235         DataTreeCandidate candidate = mockCandidate("candidate");
236         successfulCanCommit(cohort);
237         successfulPreCommit(cohort, candidate);
238         successfulCommit(cohort);
239         doReturn(candidate).when(cohort).getCandidate();
240         return cohort;
241     }
242
243     protected Map<TransactionIdentifier, CapturingShardDataTreeCohort> setupCohortDecorator(final Shard shard,
244             final TransactionIdentifier... transactionIDs) {
245         final Map<TransactionIdentifier, CapturingShardDataTreeCohort> cohortMap = new HashMap<>();
246         for (TransactionIdentifier id: transactionIDs) {
247             cohortMap.put(id, new CapturingShardDataTreeCohort());
248         }
249
250         shard.getCommitCoordinator().setCohortDecorator((transactionID, actual) -> {
251             CapturingShardDataTreeCohort cohort = cohortMap.get(transactionID);
252             cohort.setDelegate(actual);
253             return cohort;
254         });
255
256         return cohortMap;
257     }
258
259     protected BatchedModifications prepareBatchedModifications(final TransactionIdentifier transactionID,
260                                                                final MutableCompositeModification modification) {
261         return prepareBatchedModifications(transactionID, modification, false);
262     }
263
264     protected static BatchedModifications prepareBatchedModifications(final TransactionIdentifier transactionID,
265                                                              final MutableCompositeModification modification,
266                                                              final boolean doCommitOnReady) {
267         final BatchedModifications batchedModifications = new BatchedModifications(transactionID, CURRENT_VERSION);
268         batchedModifications.addModification(modification);
269         batchedModifications.setReady();
270         batchedModifications.setDoCommitOnReady(doCommitOnReady);
271         batchedModifications.setTotalMessagesSent(1);
272         return batchedModifications;
273     }
274
275     protected static BatchedModifications prepareBatchedModifications(final TransactionIdentifier transactionID,
276             final YangInstanceIdentifier path, final NormalizedNode<?, ?> data, final boolean doCommitOnReady) {
277         final MutableCompositeModification modification = new MutableCompositeModification();
278         modification.addModification(new WriteModification(path, data));
279         return prepareBatchedModifications(transactionID, modification, doCommitOnReady);
280     }
281
282     protected static ForwardedReadyTransaction prepareForwardedReadyTransaction(final TestActorRef<Shard> shard,
283             final TransactionIdentifier transactionID, final YangInstanceIdentifier path,
284             final NormalizedNode<?, ?> data, final boolean doCommitOnReady) {
285         ReadWriteShardDataTreeTransaction rwTx = shard.underlyingActor().getDataStore()
286                 .newReadWriteTransaction(transactionID);
287         rwTx.getSnapshot().write(path, data);
288         return new ForwardedReadyTransaction(transactionID, CURRENT_VERSION, rwTx, doCommitOnReady, Optional.empty());
289     }
290
291     public static NormalizedNode<?,?> readStore(final TestActorRef<? extends Shard> shard,
292             final YangInstanceIdentifier id) {
293         return shard.underlyingActor().getDataStore().readNode(id).orNull();
294     }
295
296     public static NormalizedNode<?,?> readStore(final DataTree store, final YangInstanceIdentifier id) {
297         return store.takeSnapshot().readNode(id).orElse(null);
298     }
299
300     public void writeToStore(final TestActorRef<Shard> shard, final YangInstanceIdentifier id,
301             final NormalizedNode<?,?> node) throws InterruptedException, ExecutionException {
302         Future<Object> future = Patterns.ask(shard, newBatchedModifications(nextTransactionId(),
303                 id, node, true, true, 1), new Timeout(5, TimeUnit.SECONDS));
304         try {
305             Await.ready(future, Duration.create(5, TimeUnit.SECONDS));
306         } catch (TimeoutException e) {
307             throw new ExecutionException(e);
308         }
309     }
310
311     public static void writeToStore(final ShardDataTree store, final YangInstanceIdentifier id,
312             final NormalizedNode<?,?> node) throws DataValidationFailedException {
313         BatchedModifications batched = newBatchedModifications(nextTransactionId(), id, node, true, true, 1);
314         DataTreeModification modification = store.getDataTree().takeSnapshot().newModification();
315         batched.apply(modification);
316         store.notifyListeners(commitTransaction(store.getDataTree(), modification));
317     }
318
319     public static void writeToStore(final DataTree store, final YangInstanceIdentifier id,
320             final NormalizedNode<?,?> node) throws DataValidationFailedException {
321         final DataTreeModification transaction = store.takeSnapshot().newModification();
322
323         transaction.write(id, node);
324         transaction.ready();
325         store.validate(transaction);
326         final DataTreeCandidate candidate = store.prepare(transaction);
327         store.commit(candidate);
328     }
329
330     public void mergeToStore(final ShardDataTree store, final YangInstanceIdentifier id,
331             final NormalizedNode<?,?> node) throws DataValidationFailedException {
332         final BatchedModifications batched = new BatchedModifications(nextTransactionId(), CURRENT_VERSION);
333         batched.addModification(new MergeModification(id, node));
334         batched.setReady();
335         batched.setDoCommitOnReady(true);
336         batched.setTotalMessagesSent(1);
337
338         DataTreeModification modification = store.getDataTree().takeSnapshot().newModification();
339         batched.apply(modification);
340         store.notifyListeners(commitTransaction(store.getDataTree(), modification));
341     }
342
343     DataTree setupInMemorySnapshotStore() throws DataValidationFailedException {
344         final DataTree testStore = new InMemoryDataTreeFactory().create(
345             DataTreeConfiguration.DEFAULT_OPERATIONAL, SCHEMA_CONTEXT);
346
347         writeToStore(testStore, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
348
349         final NormalizedNode<?, ?> root = readStore(testStore, YangInstanceIdentifier.EMPTY);
350
351         InMemorySnapshotStore.addSnapshot(shardID.toString(), Snapshot.create(
352                 new ShardSnapshotState(new MetadataShardDataTreeSnapshot(root)),
353                 Collections.<ReplicatedLogEntry>emptyList(), 0, 1, -1, -1, 1, null, null));
354         return testStore;
355     }
356
357     static CommitTransactionPayload payloadForModification(final DataTree source, final DataTreeModification mod,
358             final TransactionIdentifier transactionId) throws DataValidationFailedException, IOException {
359         source.validate(mod);
360         final DataTreeCandidate candidate = source.prepare(mod);
361         source.commit(candidate);
362         return CommitTransactionPayload.create(transactionId, candidate);
363     }
364
365     static BatchedModifications newBatchedModifications(final TransactionIdentifier transactionID,
366             final YangInstanceIdentifier path, final NormalizedNode<?, ?> data, final boolean ready,
367             final boolean doCommitOnReady, final int messagesSent) {
368         final BatchedModifications batched = new BatchedModifications(transactionID, CURRENT_VERSION);
369         batched.addModification(new WriteModification(path, data));
370         if (ready) {
371             batched.setReady();
372         }
373         batched.setDoCommitOnReady(doCommitOnReady);
374         batched.setTotalMessagesSent(messagesSent);
375         return batched;
376     }
377
378     static BatchedModifications newReadyBatchedModifications(final TransactionIdentifier transactionID,
379             final YangInstanceIdentifier path, final NormalizedNode<?, ?> data,
380             final SortedSet<String> participatingShardNames) {
381         final BatchedModifications batched = new BatchedModifications(transactionID, CURRENT_VERSION);
382         batched.addModification(new WriteModification(path, data));
383         batched.setReady(Optional.of(participatingShardNames));
384         batched.setTotalMessagesSent(1);
385         return batched;
386     }
387
388     @SuppressWarnings("unchecked")
389     static void verifyOuterListEntry(final TestActorRef<Shard> shard, final Object expIDValue) {
390         final NormalizedNode<?, ?> outerList = readStore(shard, TestModel.OUTER_LIST_PATH);
391         assertNotNull(TestModel.OUTER_LIST_QNAME.getLocalName() + " not found", outerList);
392         assertTrue(TestModel.OUTER_LIST_QNAME.getLocalName() + " value is not Iterable",
393                 outerList.getValue() instanceof Iterable);
394         final Object entry = ((Iterable<Object>)outerList.getValue()).iterator().next();
395         assertTrue(TestModel.OUTER_LIST_QNAME.getLocalName() + " entry is not MapEntryNode",
396                 entry instanceof MapEntryNode);
397         final MapEntryNode mapEntry = (MapEntryNode)entry;
398         final Optional<DataContainerChild<? extends PathArgument, ?>> idLeaf =
399                 mapEntry.getChild(new YangInstanceIdentifier.NodeIdentifier(TestModel.ID_QNAME));
400         assertTrue("Missing leaf " + TestModel.ID_QNAME.getLocalName(), idLeaf.isPresent());
401         assertEquals(TestModel.ID_QNAME.getLocalName() + " value", expIDValue, idLeaf.get().getValue());
402     }
403
404     public static DataTreeCandidateTip mockCandidate(final String name) {
405         final DataTreeCandidateTip mockCandidate = mock(DataTreeCandidateTip.class, name);
406         final DataTreeCandidateNode mockCandidateNode = mock(DataTreeCandidateNode.class, name + "-node");
407         doReturn(ModificationType.WRITE).when(mockCandidateNode).getModificationType();
408         doReturn(Optional.of(ImmutableNodes.containerNode(CarsModel.CARS_QNAME)))
409                 .when(mockCandidateNode).getDataAfter();
410         doReturn(CarsModel.BASE_PATH).when(mockCandidate).getRootPath();
411         doReturn(mockCandidateNode).when(mockCandidate).getRootNode();
412         return mockCandidate;
413     }
414
415     static DataTreeCandidateTip mockUnmodifiedCandidate(final String name) {
416         final DataTreeCandidateTip mockCandidate = mock(DataTreeCandidateTip.class, name);
417         final DataTreeCandidateNode mockCandidateNode = mock(DataTreeCandidateNode.class, name + "-node");
418         doReturn(ModificationType.UNMODIFIED).when(mockCandidateNode).getModificationType();
419         doReturn(YangInstanceIdentifier.EMPTY).when(mockCandidate).getRootPath();
420         doReturn(mockCandidateNode).when(mockCandidate).getRootNode();
421         return mockCandidate;
422     }
423
424     static DataTreeCandidate commitTransaction(final DataTree store, final DataTreeModification modification)
425             throws DataValidationFailedException {
426         modification.ready();
427         store.validate(modification);
428         final DataTreeCandidate candidate = store.prepare(modification);
429         store.commit(candidate);
430         return candidate;
431     }
432
433     @SuppressWarnings("serial")
434     public static final class DelegatingShardCreator implements Creator<Shard> {
435         private final Creator<Shard> delegate;
436
437         DelegatingShardCreator(final Creator<Shard> delegate) {
438             this.delegate = delegate;
439         }
440
441         @Override
442         public Shard create() throws Exception {
443             return delegate.create();
444         }
445     }
446
447     public static class CapturingShardDataTreeCohort extends ShardDataTreeCohort {
448         private volatile ShardDataTreeCohort delegate;
449         private FutureCallback<Void> canCommit;
450         private FutureCallback<DataTreeCandidate> preCommit;
451         private FutureCallback<UnsignedLong> commit;
452
453         public void setDelegate(final ShardDataTreeCohort delegate) {
454             this.delegate = delegate;
455         }
456
457         public FutureCallback<Void> getCanCommit() {
458             assertNotNull("canCommit was not invoked", canCommit);
459             return canCommit;
460         }
461
462         public FutureCallback<DataTreeCandidate> getPreCommit() {
463             assertNotNull("preCommit was not invoked", preCommit);
464             return preCommit;
465         }
466
467         public FutureCallback<UnsignedLong> getCommit() {
468             assertNotNull("commit was not invoked", commit);
469             return commit;
470         }
471
472         @Override
473         public TransactionIdentifier getIdentifier() {
474             return delegate.getIdentifier();
475         }
476
477         @Override
478         DataTreeCandidateTip getCandidate() {
479             return delegate.getCandidate();
480         }
481
482         @Override
483         DataTreeModification getDataTreeModification() {
484             return delegate.getDataTreeModification();
485         }
486
487         @Override
488         public void canCommit(final FutureCallback<Void> callback) {
489             canCommit = mockFutureCallback(callback);
490             delegate.canCommit(canCommit);
491         }
492
493         @Override
494         public void preCommit(final FutureCallback<DataTreeCandidate> callback) {
495             preCommit = mockFutureCallback(callback);
496             delegate.preCommit(preCommit);
497         }
498
499         @Override
500         public void commit(final FutureCallback<UnsignedLong> callback) {
501             commit = mockFutureCallback(callback);
502             delegate.commit(commit);
503         }
504
505         @SuppressWarnings("unchecked")
506         private static <T> FutureCallback<T> mockFutureCallback(final FutureCallback<T> actual) {
507             FutureCallback<T> mock = mock(FutureCallback.class);
508             doAnswer(invocation -> {
509                 actual.onFailure(invocation.getArgumentAt(0, Throwable.class));
510                 return null;
511             }).when(mock).onFailure(any(Throwable.class));
512
513             doAnswer(invocation -> {
514                 actual.onSuccess((T) invocation.getArgumentAt(0, Throwable.class));
515                 return null;
516             }).when(mock).onSuccess((T) any(Object.class));
517
518             return mock;
519         }
520
521         @Override
522         public void abort(final FutureCallback<Void> callback) {
523             delegate.abort(callback);
524         }
525
526         @Override
527         public boolean isFailed() {
528             return delegate.isFailed();
529         }
530
531         @Override
532         public State getState() {
533             return delegate.getState();
534         }
535
536         @Override
537         Optional<SortedSet<String>> getParticipatingShardNames() {
538             return delegate.getParticipatingShardNames();
539         }
540     }
541 }