Merge "Create transaction on the backend datastore only when neccessary"
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / TransactionProxyTest.java
1 package org.opendaylight.controller.cluster.datastore;
2
3 import static org.junit.Assert.assertEquals;
4 import static org.junit.Assert.assertNotNull;
5 import static org.junit.Assert.assertTrue;
6 import static org.mockito.Matchers.any;
7 import static org.mockito.Matchers.anyString;
8 import static org.mockito.Matchers.eq;
9 import static org.mockito.Matchers.isA;
10 import static org.mockito.Mockito.doReturn;
11 import static org.mockito.Mockito.mock;
12 import static org.mockito.Mockito.never;
13 import static org.mockito.Mockito.verify;
14 import static org.opendaylight.controller.cluster.datastore.TransactionProxy.TransactionType.READ_ONLY;
15 import static org.opendaylight.controller.cluster.datastore.TransactionProxy.TransactionType.READ_WRITE;
16 import static org.opendaylight.controller.cluster.datastore.TransactionProxy.TransactionType.WRITE_ONLY;
17 import akka.actor.ActorRef;
18 import akka.actor.ActorSelection;
19 import akka.actor.ActorSystem;
20 import akka.actor.Props;
21 import akka.dispatch.Futures;
22 import com.google.common.base.Optional;
23 import com.google.common.collect.Sets;
24 import com.google.common.util.concurrent.CheckedFuture;
25 import com.google.common.util.concurrent.FutureCallback;
26 import com.google.common.util.concurrent.Uninterruptibles;
27 import java.util.Collection;
28 import java.util.List;
29 import java.util.concurrent.CountDownLatch;
30 import java.util.concurrent.ExecutionException;
31 import java.util.concurrent.TimeUnit;
32 import java.util.concurrent.atomic.AtomicReference;
33 import org.junit.Assert;
34 import org.junit.Test;
35 import org.mockito.InOrder;
36 import org.mockito.Mockito;
37 import org.opendaylight.controller.cluster.datastore.TransactionProxy.TransactionType;
38 import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException;
39 import org.opendaylight.controller.cluster.datastore.exceptions.NotInitializedException;
40 import org.opendaylight.controller.cluster.datastore.exceptions.PrimaryNotFoundException;
41 import org.opendaylight.controller.cluster.datastore.exceptions.TimeoutException;
42 import org.opendaylight.controller.cluster.datastore.messages.BatchedModifications;
43 import org.opendaylight.controller.cluster.datastore.messages.CloseTransaction;
44 import org.opendaylight.controller.cluster.datastore.messages.CommitTransactionReply;
45 import org.opendaylight.controller.cluster.datastore.messages.ReadyTransaction;
46 import org.opendaylight.controller.cluster.datastore.modification.DeleteModification;
47 import org.opendaylight.controller.cluster.datastore.modification.MergeModification;
48 import org.opendaylight.controller.cluster.datastore.modification.WriteModification;
49 import org.opendaylight.controller.cluster.datastore.shardstrategy.DefaultShardStrategy;
50 import org.opendaylight.controller.cluster.datastore.utils.DoNothingActor;
51 import org.opendaylight.controller.cluster.datastore.utils.NormalizedNodeAggregatorTest;
52 import org.opendaylight.controller.md.cluster.datastore.model.CarsModel;
53 import org.opendaylight.controller.md.cluster.datastore.model.SchemaContextHelper;
54 import org.opendaylight.controller.md.cluster.datastore.model.TestModel;
55 import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
56 import org.opendaylight.controller.protobuff.messages.transaction.ShardTransactionMessages.CreateTransactionReply;
57 import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
58 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
59 import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
60 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
61 import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
62 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
63 import scala.concurrent.Promise;
64
65 @SuppressWarnings("resource")
66 public class TransactionProxyTest extends AbstractTransactionProxyTest {
67
68     @SuppressWarnings("serial")
69     static class TestException extends RuntimeException {
70     }
71
72     static interface Invoker {
73         CheckedFuture<?, ReadFailedException> invoke(TransactionProxy proxy) throws Exception;
74     }
75
76     @Test
77     public void testRead() throws Exception {
78         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), READ_ONLY);
79
80         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, READ_ONLY);
81
82         doReturn(readSerializedDataReply(null)).when(mockActorContext).executeOperationAsync(
83                 eq(actorSelection(actorRef)), eqSerializedReadData());
84
85         Optional<NormalizedNode<?, ?>> readOptional = transactionProxy.read(
86                 TestModel.TEST_PATH).get(5, TimeUnit.SECONDS);
87
88         assertEquals("NormalizedNode isPresent", false, readOptional.isPresent());
89
90         NormalizedNode<?, ?> expectedNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
91
92         doReturn(readSerializedDataReply(expectedNode)).when(mockActorContext).executeOperationAsync(
93                 eq(actorSelection(actorRef)), eqSerializedReadData());
94
95         readOptional = transactionProxy.read(TestModel.TEST_PATH).get(5, TimeUnit.SECONDS);
96
97         assertEquals("NormalizedNode isPresent", true, readOptional.isPresent());
98
99         assertEquals("Response NormalizedNode", expectedNode, readOptional.get());
100     }
101
102     @Test(expected = ReadFailedException.class)
103     public void testReadWithInvalidReplyMessageType() throws Exception {
104         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), READ_ONLY);
105
106         doReturn(Futures.successful(new Object())).when(mockActorContext).
107                 executeOperationAsync(eq(actorSelection(actorRef)), eqSerializedReadData());
108
109         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, READ_ONLY);
110
111         transactionProxy.read(TestModel.TEST_PATH).checkedGet(5, TimeUnit.SECONDS);
112     }
113
114     @Test(expected = TestException.class)
115     public void testReadWithAsyncRemoteOperatonFailure() throws Throwable {
116         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), READ_ONLY);
117
118         doReturn(Futures.failed(new TestException())).when(mockActorContext).
119                 executeOperationAsync(eq(actorSelection(actorRef)), eqSerializedReadData());
120
121         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, READ_ONLY);
122
123         propagateReadFailedExceptionCause(transactionProxy.read(TestModel.TEST_PATH));
124     }
125
126     private void testExceptionOnInitialCreateTransaction(Exception exToThrow, Invoker invoker)
127             throws Throwable {
128         ActorRef actorRef = getSystem().actorOf(Props.create(DoNothingActor.class));
129
130         if (exToThrow instanceof PrimaryNotFoundException) {
131             doReturn(Futures.failed(exToThrow)).when(mockActorContext).findPrimaryShardAsync(anyString());
132         } else {
133             doReturn(primaryShardInfoReply(getSystem(), actorRef)).
134                     when(mockActorContext).findPrimaryShardAsync(anyString());
135         }
136
137         doReturn(Futures.failed(exToThrow)).when(mockActorContext).executeOperationAsync(
138                 any(ActorSelection.class), any());
139
140         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, READ_ONLY);
141
142         propagateReadFailedExceptionCause(invoker.invoke(transactionProxy));
143     }
144
145     private void testReadWithExceptionOnInitialCreateTransaction(Exception exToThrow) throws Throwable {
146         testExceptionOnInitialCreateTransaction(exToThrow, new Invoker() {
147             @Override
148             public CheckedFuture<?, ReadFailedException> invoke(TransactionProxy proxy) throws Exception {
149                 return proxy.read(TestModel.TEST_PATH);
150             }
151         });
152     }
153
154     @Test(expected = PrimaryNotFoundException.class)
155     public void testReadWhenAPrimaryNotFoundExceptionIsThrown() throws Throwable {
156         testReadWithExceptionOnInitialCreateTransaction(new PrimaryNotFoundException("test"));
157     }
158
159     @Test(expected = TimeoutException.class)
160     public void testReadWhenATimeoutExceptionIsThrown() throws Throwable {
161         testReadWithExceptionOnInitialCreateTransaction(new TimeoutException("test",
162                 new Exception("reason")));
163     }
164
165     @Test(expected = TestException.class)
166     public void testReadWhenAnyOtherExceptionIsThrown() throws Throwable {
167         testReadWithExceptionOnInitialCreateTransaction(new TestException());
168     }
169
170     @Test
171     public void testReadWithPriorRecordingOperationSuccessful() throws Throwable {
172         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), READ_WRITE);
173
174         NormalizedNode<?, ?> expectedNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
175
176         expectBatchedModifications(actorRef, 1);
177
178         doReturn(readSerializedDataReply(expectedNode)).when(mockActorContext).executeOperationAsync(
179                 eq(actorSelection(actorRef)), eqSerializedReadData());
180
181         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, READ_WRITE);
182
183         transactionProxy.write(TestModel.TEST_PATH, expectedNode);
184
185         Optional<NormalizedNode<?, ?>> readOptional = transactionProxy.read(
186                 TestModel.TEST_PATH).get(5, TimeUnit.SECONDS);
187
188         assertEquals("NormalizedNode isPresent", true, readOptional.isPresent());
189         assertEquals("Response NormalizedNode", expectedNode, readOptional.get());
190
191         InOrder inOrder = Mockito.inOrder(mockActorContext);
192         inOrder.verify(mockActorContext).executeOperationAsync(
193                 eq(actorSelection(actorRef)), isA(BatchedModifications.class));
194
195         inOrder.verify(mockActorContext).executeOperationAsync(
196                 eq(actorSelection(actorRef)), eqSerializedReadData());
197     }
198
199     @Test(expected=IllegalStateException.class)
200     public void testReadPreConditionCheck() {
201         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, WRITE_ONLY);
202         transactionProxy.read(TestModel.TEST_PATH);
203     }
204
205     @Test(expected=IllegalArgumentException.class)
206     public void testInvalidCreateTransactionReply() throws Throwable {
207         ActorRef actorRef = getSystem().actorOf(Props.create(DoNothingActor.class));
208
209         doReturn(getSystem().actorSelection(actorRef.path())).when(mockActorContext).
210             actorSelection(actorRef.path().toString());
211
212         doReturn(primaryShardInfoReply(getSystem(), actorRef)).
213             when(mockActorContext).findPrimaryShardAsync(eq(DefaultShardStrategy.DEFAULT_SHARD));
214
215         doReturn(Futures.successful(new Object())).when(mockActorContext).executeOperationAsync(
216             eq(getSystem().actorSelection(actorRef.path())), eqCreateTransaction(memberName, READ_ONLY));
217
218         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, READ_ONLY);
219
220         propagateReadFailedExceptionCause(transactionProxy.read(TestModel.TEST_PATH));
221     }
222
223     @Test
224     public void testExists() throws Exception {
225         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), READ_ONLY);
226
227         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, READ_ONLY);
228
229         doReturn(dataExistsSerializedReply(false)).when(mockActorContext).executeOperationAsync(
230                 eq(actorSelection(actorRef)), eqSerializedDataExists());
231
232         Boolean exists = transactionProxy.exists(TestModel.TEST_PATH).checkedGet();
233
234         assertEquals("Exists response", false, exists);
235
236         doReturn(dataExistsSerializedReply(true)).when(mockActorContext).executeOperationAsync(
237                 eq(actorSelection(actorRef)), eqSerializedDataExists());
238
239         exists = transactionProxy.exists(TestModel.TEST_PATH).checkedGet();
240
241         assertEquals("Exists response", true, exists);
242     }
243
244     @Test(expected = PrimaryNotFoundException.class)
245     public void testExistsWhenAPrimaryNotFoundExceptionIsThrown() throws Throwable {
246         testExceptionOnInitialCreateTransaction(new PrimaryNotFoundException("test"), new Invoker() {
247             @Override
248             public CheckedFuture<?, ReadFailedException> invoke(TransactionProxy proxy) throws Exception {
249                 return proxy.exists(TestModel.TEST_PATH);
250             }
251         });
252     }
253
254     @Test(expected = ReadFailedException.class)
255     public void testExistsWithInvalidReplyMessageType() throws Exception {
256         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), READ_ONLY);
257
258         doReturn(Futures.successful(new Object())).when(mockActorContext).
259                 executeOperationAsync(eq(actorSelection(actorRef)), eqSerializedDataExists());
260
261         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext,
262                 READ_ONLY);
263
264         transactionProxy.exists(TestModel.TEST_PATH).checkedGet(5, TimeUnit.SECONDS);
265     }
266
267     @Test(expected = TestException.class)
268     public void testExistsWithAsyncRemoteOperatonFailure() throws Throwable {
269         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), READ_ONLY);
270
271         doReturn(Futures.failed(new TestException())).when(mockActorContext).
272                 executeOperationAsync(eq(actorSelection(actorRef)), eqSerializedDataExists());
273
274         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, READ_ONLY);
275
276         propagateReadFailedExceptionCause(transactionProxy.exists(TestModel.TEST_PATH));
277     }
278
279     @Test
280     public void testExistsWithPriorRecordingOperationSuccessful() throws Throwable {
281         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), READ_WRITE);
282
283         NormalizedNode<?, ?> nodeToWrite = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
284
285         expectBatchedModifications(actorRef, 1);
286
287         doReturn(dataExistsSerializedReply(true)).when(mockActorContext).executeOperationAsync(
288                 eq(actorSelection(actorRef)), eqSerializedDataExists());
289
290         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, READ_WRITE);
291
292         transactionProxy.write(TestModel.TEST_PATH, nodeToWrite);
293
294         Boolean exists = transactionProxy.exists(TestModel.TEST_PATH).checkedGet();
295
296         assertEquals("Exists response", true, exists);
297
298         InOrder inOrder = Mockito.inOrder(mockActorContext);
299         inOrder.verify(mockActorContext).executeOperationAsync(
300                 eq(actorSelection(actorRef)), isA(BatchedModifications.class));
301
302         inOrder.verify(mockActorContext).executeOperationAsync(
303                 eq(actorSelection(actorRef)), eqSerializedDataExists());
304     }
305
306     @Test(expected=IllegalStateException.class)
307     public void testExistsPreConditionCheck() {
308         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, WRITE_ONLY);
309         transactionProxy.exists(TestModel.TEST_PATH);
310     }
311
312     @Test
313     public void testWrite() throws Exception {
314         dataStoreContextBuilder.shardBatchedModificationCount(1);
315         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), WRITE_ONLY);
316
317         NormalizedNode<?, ?> nodeToWrite = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
318
319         expectBatchedModifications(actorRef, 1);
320
321         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, WRITE_ONLY);
322
323         transactionProxy.write(TestModel.TEST_PATH, nodeToWrite);
324
325         verifyOneBatchedModification(actorRef, new WriteModification(TestModel.TEST_PATH, nodeToWrite), false);
326     }
327
328     @Test
329     public void testWriteAfterAsyncRead() throws Throwable {
330         ActorRef actorRef = setupActorContextWithoutInitialCreateTransaction(getSystem(), DefaultShardStrategy.DEFAULT_SHARD);
331
332         Promise<Object> createTxPromise = akka.dispatch.Futures.promise();
333         doReturn(createTxPromise).when(mockActorContext).executeOperationAsync(
334                 eq(getSystem().actorSelection(actorRef.path())),
335                 eqCreateTransaction(memberName, READ_WRITE));
336
337         doReturn(readSerializedDataReply(null)).when(mockActorContext).executeOperationAsync(
338                 eq(actorSelection(actorRef)), eqSerializedReadData());
339
340         expectBatchedModificationsReady(actorRef);
341
342         final NormalizedNode<?, ?> nodeToWrite = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
343
344         final TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, READ_WRITE);
345
346         final CountDownLatch readComplete = new CountDownLatch(1);
347         final AtomicReference<Throwable> caughtEx = new AtomicReference<>();
348         com.google.common.util.concurrent.Futures.addCallback(transactionProxy.read(TestModel.TEST_PATH),
349                 new  FutureCallback<Optional<NormalizedNode<?, ?>>>() {
350                     @Override
351                     public void onSuccess(Optional<NormalizedNode<?, ?>> result) {
352                         try {
353                             transactionProxy.write(TestModel.TEST_PATH, nodeToWrite);
354                         } catch (Exception e) {
355                             caughtEx.set(e);
356                         } finally {
357                             readComplete.countDown();
358                         }
359                     }
360
361                     @Override
362                     public void onFailure(Throwable t) {
363                         caughtEx.set(t);
364                         readComplete.countDown();
365                     }
366                 });
367
368         createTxPromise.success(createTransactionReply(actorRef, DataStoreVersions.CURRENT_VERSION));
369
370         Uninterruptibles.awaitUninterruptibly(readComplete, 5, TimeUnit.SECONDS);
371
372         if(caughtEx.get() != null) {
373             throw caughtEx.get();
374         }
375
376         // This sends the batched modification.
377         transactionProxy.ready();
378
379         verifyOneBatchedModification(actorRef, new WriteModification(TestModel.TEST_PATH, nodeToWrite), true);
380     }
381
382     @Test(expected=IllegalStateException.class)
383     public void testWritePreConditionCheck() {
384         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, READ_ONLY);
385         transactionProxy.write(TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
386     }
387
388     @Test(expected=IllegalStateException.class)
389     public void testWriteAfterReadyPreConditionCheck() {
390         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, WRITE_ONLY);
391
392         transactionProxy.ready();
393
394         transactionProxy.write(TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
395     }
396
397     @Test
398     public void testMerge() throws Exception {
399         dataStoreContextBuilder.shardBatchedModificationCount(1);
400         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), WRITE_ONLY);
401
402         NormalizedNode<?, ?> nodeToWrite = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
403
404         expectBatchedModifications(actorRef, 1);
405
406         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, WRITE_ONLY);
407
408         transactionProxy.merge(TestModel.TEST_PATH, nodeToWrite);
409
410         verifyOneBatchedModification(actorRef, new MergeModification(TestModel.TEST_PATH, nodeToWrite), false);
411     }
412
413     @Test
414     public void testDelete() throws Exception {
415         dataStoreContextBuilder.shardBatchedModificationCount(1);
416         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), WRITE_ONLY);
417
418         expectBatchedModifications(actorRef, 1);
419
420         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, WRITE_ONLY);
421
422         transactionProxy.delete(TestModel.TEST_PATH);
423
424         verifyOneBatchedModification(actorRef, new DeleteModification(TestModel.TEST_PATH), false);
425     }
426
427     @Test
428     public void testReadWrite() throws Exception {
429         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), READ_WRITE);
430
431         NormalizedNode<?, ?> nodeToWrite = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
432
433         doReturn(readSerializedDataReply(null)).when(mockActorContext).executeOperationAsync(
434                 eq(actorSelection(actorRef)), eqSerializedReadData());
435
436         expectBatchedModifications(actorRef, 1);
437
438         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, READ_WRITE);
439
440         transactionProxy.read(TestModel.TEST_PATH);
441
442         transactionProxy.write(TestModel.TEST_PATH, nodeToWrite);
443
444         transactionProxy.read(TestModel.TEST_PATH);
445
446         transactionProxy.read(TestModel.TEST_PATH);
447
448         List<BatchedModifications> batchedModifications = captureBatchedModifications(actorRef);
449         assertEquals("Captured BatchedModifications count", 1, batchedModifications.size());
450
451         verifyBatchedModifications(batchedModifications.get(0), false,
452                 new WriteModification(TestModel.TEST_PATH, nodeToWrite));
453     }
454
455     @Test
456     public void testReadyWithReadWrite() throws Exception {
457         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), READ_WRITE);
458
459         NormalizedNode<?, ?> nodeToWrite = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
460
461         doReturn(readSerializedDataReply(null)).when(mockActorContext).executeOperationAsync(
462                 eq(actorSelection(actorRef)), eqSerializedReadData());
463
464         expectBatchedModificationsReady(actorRef, true);
465
466         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, READ_WRITE);
467
468         transactionProxy.read(TestModel.TEST_PATH);
469
470         transactionProxy.write(TestModel.TEST_PATH, nodeToWrite);
471
472         DOMStoreThreePhaseCommitCohort ready = transactionProxy.ready();
473
474         assertTrue(ready instanceof SingleCommitCohortProxy);
475
476         verifyCohortFutures((SingleCommitCohortProxy)ready, new CommitTransactionReply().toSerializable());
477
478         List<BatchedModifications> batchedModifications = captureBatchedModifications(actorRef);
479         assertEquals("Captured BatchedModifications count", 1, batchedModifications.size());
480
481         verifyBatchedModifications(batchedModifications.get(0), true, true,
482                 new WriteModification(TestModel.TEST_PATH, nodeToWrite));
483
484         assertEquals("getTotalMessageCount", 1, batchedModifications.get(0).getTotalMessagesSent());
485     }
486
487     @Test
488     public void testReadyWithNoModifications() throws Exception {
489         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), READ_WRITE);
490
491         doReturn(readSerializedDataReply(null)).when(mockActorContext).executeOperationAsync(
492                 eq(actorSelection(actorRef)), eqSerializedReadData());
493
494         expectBatchedModificationsReady(actorRef, true);
495
496         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, READ_WRITE);
497
498         transactionProxy.read(TestModel.TEST_PATH);
499
500         DOMStoreThreePhaseCommitCohort ready = transactionProxy.ready();
501
502         assertTrue(ready instanceof SingleCommitCohortProxy);
503
504         verifyCohortFutures((SingleCommitCohortProxy)ready, new CommitTransactionReply().toSerializable());
505
506         List<BatchedModifications> batchedModifications = captureBatchedModifications(actorRef);
507         assertEquals("Captured BatchedModifications count", 1, batchedModifications.size());
508
509         verifyBatchedModifications(batchedModifications.get(0), true, true);
510     }
511
512     @Test
513     public void testReadyWithMultipleShardWrites() throws Exception {
514         ActorRef actorRef1 = setupActorContextWithInitialCreateTransaction(getSystem(), WRITE_ONLY);
515
516         ActorRef actorRef2 = setupActorContextWithInitialCreateTransaction(getSystem(), WRITE_ONLY, "junk");
517
518         expectBatchedModificationsReady(actorRef1);
519         expectBatchedModificationsReady(actorRef2);
520
521         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, WRITE_ONLY);
522
523         transactionProxy.write(TestModel.JUNK_PATH, ImmutableNodes.containerNode(TestModel.JUNK_QNAME));
524         transactionProxy.write(TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
525
526         DOMStoreThreePhaseCommitCohort ready = transactionProxy.ready();
527
528         assertTrue(ready instanceof ThreePhaseCommitCohortProxy);
529
530         verifyCohortFutures((ThreePhaseCommitCohortProxy)ready, actorSelection(actorRef1),
531                 actorSelection(actorRef2));
532     }
533
534     @Test
535     public void testReadyWithWriteOnlyAndLastBatchPending() throws Exception {
536         dataStoreContextBuilder.writeOnlyTransactionOptimizationsEnabled(true);
537
538         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), WRITE_ONLY);
539
540         NormalizedNode<?, ?> nodeToWrite = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
541
542         expectBatchedModificationsReady(actorRef, true);
543
544         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, WRITE_ONLY);
545
546         transactionProxy.write(TestModel.TEST_PATH, nodeToWrite);
547
548         DOMStoreThreePhaseCommitCohort ready = transactionProxy.ready();
549
550         assertTrue(ready instanceof SingleCommitCohortProxy);
551
552         verifyCohortFutures((SingleCommitCohortProxy)ready, new CommitTransactionReply().toSerializable());
553
554         List<BatchedModifications> batchedModifications = captureBatchedModifications(actorRef);
555         assertEquals("Captured BatchedModifications count", 1, batchedModifications.size());
556
557         verifyBatchedModifications(batchedModifications.get(0), true, true,
558                 new WriteModification(TestModel.TEST_PATH, nodeToWrite));
559
560         verify(mockActorContext, never()).executeOperationAsync(eq(actorSelection(actorRef)),
561                 isA(ReadyTransaction.SERIALIZABLE_CLASS));
562     }
563
564     @Test
565     public void testReadyWithWriteOnlyAndLastBatchEmpty() throws Exception {
566         dataStoreContextBuilder.shardBatchedModificationCount(1).writeOnlyTransactionOptimizationsEnabled(true);
567         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), WRITE_ONLY);
568
569         NormalizedNode<?, ?> nodeToWrite = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
570
571         expectBatchedModificationsReady(actorRef, true);
572
573         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, WRITE_ONLY);
574
575         transactionProxy.write(TestModel.TEST_PATH, nodeToWrite);
576
577         DOMStoreThreePhaseCommitCohort ready = transactionProxy.ready();
578
579         assertTrue(ready instanceof SingleCommitCohortProxy);
580
581         verifyCohortFutures((SingleCommitCohortProxy)ready, new CommitTransactionReply().toSerializable());
582
583         List<BatchedModifications> batchedModifications = captureBatchedModifications(actorRef);
584         assertEquals("Captured BatchedModifications count", 2, batchedModifications.size());
585
586         verifyBatchedModifications(batchedModifications.get(0), false,
587                 new WriteModification(TestModel.TEST_PATH, nodeToWrite));
588
589         verifyBatchedModifications(batchedModifications.get(1), true, true);
590
591         verify(mockActorContext, never()).executeOperationAsync(eq(actorSelection(actorRef)),
592                 isA(ReadyTransaction.SERIALIZABLE_CLASS));
593     }
594
595     @Test
596     public void testReadyWithReplyFailure() throws Exception {
597         dataStoreContextBuilder.writeOnlyTransactionOptimizationsEnabled(true);
598
599         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), WRITE_ONLY);
600
601         NormalizedNode<?, ?> nodeToWrite = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
602
603         expectFailedBatchedModifications(actorRef);
604
605         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, WRITE_ONLY);
606
607         transactionProxy.merge(TestModel.TEST_PATH, nodeToWrite);
608
609         DOMStoreThreePhaseCommitCohort ready = transactionProxy.ready();
610
611         assertTrue(ready instanceof SingleCommitCohortProxy);
612
613         verifyCohortFutures((SingleCommitCohortProxy)ready, TestException.class);
614     }
615
616     private void testWriteOnlyTxWithFindPrimaryShardFailure(Exception toThrow) throws Exception {
617         doReturn(Futures.failed(toThrow)).when(mockActorContext).findPrimaryShardAsync(anyString());
618
619         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, WRITE_ONLY);
620
621         NormalizedNode<?, ?> nodeToWrite = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
622
623         transactionProxy.merge(TestModel.TEST_PATH, nodeToWrite);
624
625         transactionProxy.write(TestModel.TEST_PATH, nodeToWrite);
626
627         transactionProxy.delete(TestModel.TEST_PATH);
628
629         DOMStoreThreePhaseCommitCohort ready = transactionProxy.ready();
630
631         assertTrue(ready instanceof SingleCommitCohortProxy);
632
633         verifyCohortFutures((SingleCommitCohortProxy)ready, toThrow.getClass());
634     }
635
636     @Test
637     public void testWriteOnlyTxWithPrimaryNotFoundException() throws Exception {
638         testWriteOnlyTxWithFindPrimaryShardFailure(new PrimaryNotFoundException("mock"));
639     }
640
641     @Test
642     public void testWriteOnlyTxWithNotInitializedException() throws Exception {
643         testWriteOnlyTxWithFindPrimaryShardFailure(new NotInitializedException("mock"));
644     }
645
646     @Test
647     public void testWriteOnlyTxWithNoShardLeaderException() throws Exception {
648         testWriteOnlyTxWithFindPrimaryShardFailure(new NoShardLeaderException("mock"));
649     }
650
651     @Test
652     public void testReadyWithInvalidReplyMessageType() throws Exception {
653         dataStoreContextBuilder.writeOnlyTransactionOptimizationsEnabled(true);
654         ActorRef actorRef1 = setupActorContextWithInitialCreateTransaction(getSystem(), WRITE_ONLY);
655
656         ActorRef actorRef2 = setupActorContextWithInitialCreateTransaction(getSystem(), WRITE_ONLY, "junk");
657
658         doReturn(Futures.successful(new Object())).when(mockActorContext).
659                 executeOperationAsync(eq(actorSelection(actorRef1)), isA(BatchedModifications.class));
660
661         expectBatchedModificationsReady(actorRef2);
662
663         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, WRITE_ONLY);
664
665         transactionProxy.write(TestModel.JUNK_PATH, ImmutableNodes.containerNode(TestModel.JUNK_QNAME));
666         transactionProxy.write(TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
667
668         DOMStoreThreePhaseCommitCohort ready = transactionProxy.ready();
669
670         assertTrue(ready instanceof ThreePhaseCommitCohortProxy);
671
672         verifyCohortFutures((ThreePhaseCommitCohortProxy)ready, actorSelection(actorRef2),
673                 IllegalArgumentException.class);
674     }
675
676     @Test
677     public void testGetIdentifier() {
678         setupActorContextWithInitialCreateTransaction(getSystem(), READ_ONLY);
679         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext,
680                 TransactionProxy.TransactionType.READ_ONLY);
681
682         Object id = transactionProxy.getIdentifier();
683         assertNotNull("getIdentifier returned null", id);
684         assertTrue("Invalid identifier: " + id, id.toString().startsWith(memberName));
685     }
686
687     @Test
688     public void testClose() throws Exception{
689         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), READ_WRITE);
690
691         doReturn(readSerializedDataReply(null)).when(mockActorContext).executeOperationAsync(
692                 eq(actorSelection(actorRef)), eqSerializedReadData());
693
694         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, READ_WRITE);
695
696         transactionProxy.read(TestModel.TEST_PATH);
697
698         transactionProxy.close();
699
700         verify(mockActorContext).sendOperationAsync(
701                 eq(actorSelection(actorRef)), isA(CloseTransaction.SERIALIZABLE_CLASS));
702     }
703
704
705     /**
706      * Method to test a local Tx actor. The Tx paths are matched to decide if the
707      * Tx actor is local or not. This is done by mocking the Tx actor path
708      * and the caller paths and ensuring that the paths have the remote-address format
709      *
710      * Note: Since the default akka provider for test is not a RemoteActorRefProvider,
711      * the paths returned for the actors for all the tests are not qualified remote paths.
712      * Hence are treated as non-local/remote actors. In short, all tests except
713      * few below run for remote actors
714      *
715      * @throws Exception
716      */
717     @Test
718     public void testLocalTxActorRead() throws Exception {
719         setupActorContextWithInitialCreateTransaction(getSystem(), READ_ONLY);
720         doReturn(true).when(mockActorContext).isPathLocal(anyString());
721
722         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext,READ_ONLY);
723
724         // negative test case with null as the reply
725         doReturn(readDataReply(null)).when(mockActorContext).executeOperationAsync(
726             any(ActorSelection.class), eqReadData());
727
728         Optional<NormalizedNode<?, ?>> readOptional = transactionProxy.read(
729             TestModel.TEST_PATH).get(5, TimeUnit.SECONDS);
730
731         assertEquals("NormalizedNode isPresent", false, readOptional.isPresent());
732
733         // test case with node as read data reply
734         NormalizedNode<?, ?> expectedNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
735
736         doReturn(readDataReply(expectedNode)).when(mockActorContext).executeOperationAsync(
737             any(ActorSelection.class), eqReadData());
738
739         readOptional = transactionProxy.read(TestModel.TEST_PATH).get(5, TimeUnit.SECONDS);
740
741         assertEquals("NormalizedNode isPresent", true, readOptional.isPresent());
742
743         assertEquals("Response NormalizedNode", expectedNode, readOptional.get());
744
745         // test for local data exists
746         doReturn(dataExistsReply(true)).when(mockActorContext).executeOperationAsync(
747             any(ActorSelection.class), eqDataExists());
748
749         boolean exists = transactionProxy.exists(TestModel.TEST_PATH).checkedGet();
750
751         assertEquals("Exists response", true, exists);
752     }
753
754     @Test
755     public void testLocalTxActorReady() throws Exception {
756         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), READ_WRITE);
757         doReturn(true).when(mockActorContext).isPathLocal(anyString());
758
759         expectBatchedModificationsReady(actorRef, true);
760
761         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, READ_WRITE);
762
763         NormalizedNode<?, ?> nodeToWrite = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
764         transactionProxy.write(TestModel.TEST_PATH, nodeToWrite);
765
766         DOMStoreThreePhaseCommitCohort ready = transactionProxy.ready();
767
768         assertTrue(ready instanceof SingleCommitCohortProxy);
769
770         verifyCohortFutures((SingleCommitCohortProxy)ready, new CommitTransactionReply().toSerializable());
771     }
772
773     private static interface TransactionProxyOperation {
774         void run(TransactionProxy transactionProxy);
775     }
776
777     private void throttleOperation(TransactionProxyOperation operation) {
778         throttleOperation(operation, 1, true);
779     }
780
781     private void throttleOperation(TransactionProxyOperation operation, int outstandingOpsLimit, boolean shardFound){
782         ActorSystem actorSystem = getSystem();
783         ActorRef shardActorRef = actorSystem.actorOf(Props.create(DoNothingActor.class));
784
785         doReturn(outstandingOpsLimit).when(mockActorContext).getTransactionOutstandingOperationLimit();
786
787         doReturn(actorSystem.actorSelection(shardActorRef.path())).
788                 when(mockActorContext).actorSelection(shardActorRef.path().toString());
789
790         if(shardFound) {
791             doReturn(Futures.successful(actorSystem.actorSelection(shardActorRef.path()))).
792                     when(mockActorContext).findPrimaryShardAsync(eq(DefaultShardStrategy.DEFAULT_SHARD));
793         } else {
794             doReturn(Futures.failed(new Exception("not found")))
795                     .when(mockActorContext).findPrimaryShardAsync(eq(DefaultShardStrategy.DEFAULT_SHARD));
796         }
797
798         String actorPath = "akka.tcp://system@127.0.0.1:2550/user/tx-actor";
799         CreateTransactionReply createTransactionReply = CreateTransactionReply.newBuilder().
800                 setTransactionId("txn-1").setTransactionActorPath(actorPath).
801                 setMessageVersion(DataStoreVersions.CURRENT_VERSION).build();
802
803         doReturn(Futures.successful(createTransactionReply)).when(mockActorContext).
804                 executeOperationAsync(eq(actorSystem.actorSelection(shardActorRef.path())),
805                         eqCreateTransaction(memberName, READ_WRITE));
806
807         doReturn(true).when(mockActorContext).isPathLocal(actorPath);
808
809         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, READ_WRITE);
810
811         long start = System.nanoTime();
812
813         operation.run(transactionProxy);
814
815         long end = System.nanoTime();
816
817         long expected = TimeUnit.SECONDS.toNanos(mockActorContext.getDatastoreContext().getOperationTimeoutInSeconds());
818         Assert.assertTrue(String.format("Expected elapsed time: %s. Actual: %s",
819                 expected, (end-start)), (end - start) > expected);
820
821     }
822
823     private void completeOperation(TransactionProxyOperation operation){
824         completeOperation(operation, true);
825     }
826
827     private void completeOperation(TransactionProxyOperation operation, boolean shardFound){
828         ActorSystem actorSystem = getSystem();
829         ActorRef shardActorRef = actorSystem.actorOf(Props.create(DoNothingActor.class));
830
831         doReturn(1).when(mockActorContext).getTransactionOutstandingOperationLimit();
832
833         doReturn(actorSystem.actorSelection(shardActorRef.path())).
834                 when(mockActorContext).actorSelection(shardActorRef.path().toString());
835
836         if(shardFound) {
837             doReturn(primaryShardInfoReply(actorSystem, shardActorRef)).
838                     when(mockActorContext).findPrimaryShardAsync(eq(DefaultShardStrategy.DEFAULT_SHARD));
839         } else {
840             doReturn(Futures.failed(new PrimaryNotFoundException("test")))
841                     .when(mockActorContext).findPrimaryShardAsync(eq(DefaultShardStrategy.DEFAULT_SHARD));
842         }
843
844         ActorRef txActorRef = actorSystem.actorOf(Props.create(DoNothingActor.class));
845         String actorPath = txActorRef.path().toString();
846         CreateTransactionReply createTransactionReply = CreateTransactionReply.newBuilder().
847                 setTransactionId("txn-1").setTransactionActorPath(actorPath).
848                 setMessageVersion(DataStoreVersions.CURRENT_VERSION).build();
849
850         doReturn(actorSystem.actorSelection(actorPath)).when(mockActorContext).actorSelection(actorPath);
851
852         doReturn(Futures.successful(createTransactionReply)).when(mockActorContext).
853                 executeOperationAsync(eq(actorSystem.actorSelection(shardActorRef.path())),
854                         eqCreateTransaction(memberName, READ_WRITE));
855
856         doReturn(true).when(mockActorContext).isPathLocal(anyString());
857
858         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, READ_WRITE);
859
860         long start = System.nanoTime();
861
862         operation.run(transactionProxy);
863
864         long end = System.nanoTime();
865
866         long expected = TimeUnit.SECONDS.toNanos(mockActorContext.getDatastoreContext().getOperationTimeoutInSeconds());
867         Assert.assertTrue(String.format("Expected elapsed time: %s. Actual: %s",
868                 expected, (end-start)), (end - start) <= expected);
869     }
870
871     public void testWriteThrottling(boolean shardFound){
872
873         throttleOperation(new TransactionProxyOperation() {
874             @Override
875             public void run(TransactionProxy transactionProxy) {
876                 NormalizedNode<?, ?> nodeToWrite = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
877
878                 expectBatchedModifications(2);
879
880                 transactionProxy.write(TestModel.TEST_PATH, nodeToWrite);
881
882                 transactionProxy.write(TestModel.TEST_PATH, nodeToWrite);
883             }
884         }, 1, shardFound);
885     }
886
887     @Test
888     public void testWriteThrottlingWhenShardFound(){
889         dataStoreContextBuilder.shardBatchedModificationCount(1);
890         throttleOperation(new TransactionProxyOperation() {
891             @Override
892             public void run(TransactionProxy transactionProxy) {
893                 NormalizedNode<?, ?> nodeToWrite = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
894
895                 expectIncompleteBatchedModifications();
896
897                 transactionProxy.write(TestModel.TEST_PATH, nodeToWrite);
898
899                 transactionProxy.write(TestModel.TEST_PATH, nodeToWrite);
900             }
901         });
902     }
903
904     @Test
905     public void testWriteThrottlingWhenShardNotFound(){
906         // Confirm that there is no throttling when the Shard is not found
907         dataStoreContextBuilder.shardBatchedModificationCount(1);
908         completeOperation(new TransactionProxyOperation() {
909             @Override
910             public void run(TransactionProxy transactionProxy) {
911                 NormalizedNode<?, ?> nodeToWrite = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
912
913                 expectBatchedModifications(2);
914
915                 transactionProxy.write(TestModel.TEST_PATH, nodeToWrite);
916
917                 transactionProxy.write(TestModel.TEST_PATH, nodeToWrite);
918             }
919         }, false);
920
921     }
922
923
924     @Test
925     public void testWriteCompletion(){
926         dataStoreContextBuilder.shardBatchedModificationCount(1);
927         completeOperation(new TransactionProxyOperation() {
928             @Override
929             public void run(TransactionProxy transactionProxy) {
930                 NormalizedNode<?, ?> nodeToWrite = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
931
932                 expectBatchedModifications(2);
933
934                 transactionProxy.write(TestModel.TEST_PATH, nodeToWrite);
935
936                 transactionProxy.write(TestModel.TEST_PATH, nodeToWrite);
937             }
938         });
939     }
940
941     @Test
942     public void testMergeThrottlingWhenShardFound(){
943         dataStoreContextBuilder.shardBatchedModificationCount(1);
944         throttleOperation(new TransactionProxyOperation() {
945             @Override
946             public void run(TransactionProxy transactionProxy) {
947                 NormalizedNode<?, ?> nodeToMerge = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
948
949                 expectIncompleteBatchedModifications();
950
951                 transactionProxy.merge(TestModel.TEST_PATH, nodeToMerge);
952
953                 transactionProxy.merge(TestModel.TEST_PATH, nodeToMerge);
954             }
955         });
956     }
957
958     @Test
959     public void testMergeThrottlingWhenShardNotFound(){
960         dataStoreContextBuilder.shardBatchedModificationCount(1);
961         completeOperation(new TransactionProxyOperation() {
962             @Override
963             public void run(TransactionProxy transactionProxy) {
964                 NormalizedNode<?, ?> nodeToMerge = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
965
966                 expectBatchedModifications(2);
967
968                 transactionProxy.merge(TestModel.TEST_PATH, nodeToMerge);
969
970                 transactionProxy.merge(TestModel.TEST_PATH, nodeToMerge);
971             }
972         }, false);
973     }
974
975     @Test
976     public void testMergeCompletion(){
977         dataStoreContextBuilder.shardBatchedModificationCount(1);
978         completeOperation(new TransactionProxyOperation() {
979             @Override
980             public void run(TransactionProxy transactionProxy) {
981                 NormalizedNode<?, ?> nodeToMerge = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
982
983                 expectBatchedModifications(2);
984
985                 transactionProxy.merge(TestModel.TEST_PATH, nodeToMerge);
986
987                 transactionProxy.merge(TestModel.TEST_PATH, nodeToMerge);
988             }
989         });
990
991     }
992
993     @Test
994     public void testDeleteThrottlingWhenShardFound(){
995
996         throttleOperation(new TransactionProxyOperation() {
997             @Override
998             public void run(TransactionProxy transactionProxy) {
999                 expectIncompleteBatchedModifications();
1000
1001                 transactionProxy.delete(TestModel.TEST_PATH);
1002
1003                 transactionProxy.delete(TestModel.TEST_PATH);
1004             }
1005         });
1006     }
1007
1008
1009     @Test
1010     public void testDeleteThrottlingWhenShardNotFound(){
1011
1012         completeOperation(new TransactionProxyOperation() {
1013             @Override
1014             public void run(TransactionProxy transactionProxy) {
1015                 expectBatchedModifications(2);
1016
1017                 transactionProxy.delete(TestModel.TEST_PATH);
1018
1019                 transactionProxy.delete(TestModel.TEST_PATH);
1020             }
1021         }, false);
1022     }
1023
1024     @Test
1025     public void testDeleteCompletion(){
1026         dataStoreContextBuilder.shardBatchedModificationCount(1);
1027         completeOperation(new TransactionProxyOperation() {
1028             @Override
1029             public void run(TransactionProxy transactionProxy) {
1030                 expectBatchedModifications(2);
1031
1032                 transactionProxy.delete(TestModel.TEST_PATH);
1033
1034                 transactionProxy.delete(TestModel.TEST_PATH);
1035             }
1036         });
1037
1038     }
1039
1040     @Test
1041     public void testReadThrottlingWhenShardFound(){
1042
1043         throttleOperation(new TransactionProxyOperation() {
1044             @Override
1045             public void run(TransactionProxy transactionProxy) {
1046                 doReturn(incompleteFuture()).when(mockActorContext).executeOperationAsync(
1047                         any(ActorSelection.class), eqReadData());
1048
1049                 transactionProxy.read(TestModel.TEST_PATH);
1050
1051                 transactionProxy.read(TestModel.TEST_PATH);
1052             }
1053         });
1054     }
1055
1056     @Test
1057     public void testReadThrottlingWhenShardNotFound(){
1058
1059         completeOperation(new TransactionProxyOperation() {
1060             @Override
1061             public void run(TransactionProxy transactionProxy) {
1062                 doReturn(incompleteFuture()).when(mockActorContext).executeOperationAsync(
1063                         any(ActorSelection.class), eqReadData());
1064
1065                 transactionProxy.read(TestModel.TEST_PATH);
1066
1067                 transactionProxy.read(TestModel.TEST_PATH);
1068             }
1069         }, false);
1070     }
1071
1072
1073     @Test
1074     public void testReadCompletion(){
1075         completeOperation(new TransactionProxyOperation() {
1076             @Override
1077             public void run(TransactionProxy transactionProxy) {
1078                 NormalizedNode<?, ?> nodeToRead = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
1079
1080                 doReturn(readDataReply(nodeToRead)).when(mockActorContext).executeOperationAsync(
1081                         any(ActorSelection.class), eqReadData());
1082
1083                 transactionProxy.read(TestModel.TEST_PATH);
1084
1085                 transactionProxy.read(TestModel.TEST_PATH);
1086             }
1087         });
1088
1089     }
1090
1091     @Test
1092     public void testExistsThrottlingWhenShardFound(){
1093
1094         throttleOperation(new TransactionProxyOperation() {
1095             @Override
1096             public void run(TransactionProxy transactionProxy) {
1097                 doReturn(incompleteFuture()).when(mockActorContext).executeOperationAsync(
1098                         any(ActorSelection.class), eqDataExists());
1099
1100                 transactionProxy.exists(TestModel.TEST_PATH);
1101
1102                 transactionProxy.exists(TestModel.TEST_PATH);
1103             }
1104         });
1105     }
1106
1107     @Test
1108     public void testExistsThrottlingWhenShardNotFound(){
1109
1110         completeOperation(new TransactionProxyOperation() {
1111             @Override
1112             public void run(TransactionProxy transactionProxy) {
1113                 doReturn(incompleteFuture()).when(mockActorContext).executeOperationAsync(
1114                         any(ActorSelection.class), eqDataExists());
1115
1116                 transactionProxy.exists(TestModel.TEST_PATH);
1117
1118                 transactionProxy.exists(TestModel.TEST_PATH);
1119             }
1120         }, false);
1121     }
1122
1123
1124     @Test
1125     public void testExistsCompletion(){
1126         completeOperation(new TransactionProxyOperation() {
1127             @Override
1128             public void run(TransactionProxy transactionProxy) {
1129                 doReturn(dataExistsReply(true)).when(mockActorContext).executeOperationAsync(
1130                         any(ActorSelection.class), eqDataExists());
1131
1132                 transactionProxy.exists(TestModel.TEST_PATH);
1133
1134                 transactionProxy.exists(TestModel.TEST_PATH);
1135             }
1136         });
1137
1138     }
1139
1140     @Test
1141     public void testReadyThrottling(){
1142
1143         throttleOperation(new TransactionProxyOperation() {
1144             @Override
1145             public void run(TransactionProxy transactionProxy) {
1146                 NormalizedNode<?, ?> nodeToWrite = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
1147
1148                 expectBatchedModifications(1);
1149
1150                 doReturn(incompleteFuture()).when(mockActorContext).executeOperationAsync(
1151                         any(ActorSelection.class), any(ReadyTransaction.class));
1152
1153                 transactionProxy.write(TestModel.TEST_PATH, nodeToWrite);
1154
1155                 transactionProxy.ready();
1156             }
1157         });
1158     }
1159
1160     @Test
1161     public void testReadyThrottlingWithTwoTransactionContexts(){
1162
1163         throttleOperation(new TransactionProxyOperation() {
1164             @Override
1165             public void run(TransactionProxy transactionProxy) {
1166                 NormalizedNode<?, ?> nodeToWrite = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
1167                 NormalizedNode<?, ?> carsNode = ImmutableNodes.containerNode(CarsModel.BASE_QNAME);
1168
1169                 expectBatchedModifications(2);
1170
1171                 doReturn(incompleteFuture()).when(mockActorContext).executeOperationAsync(
1172                         any(ActorSelection.class), any(ReadyTransaction.class));
1173
1174                 transactionProxy.write(TestModel.TEST_PATH, nodeToWrite);
1175
1176                 transactionProxy.write(TestModel.TEST_PATH, carsNode);
1177
1178                 transactionProxy.ready();
1179             }
1180         }, 2, true);
1181     }
1182
1183     private void testModificationOperationBatching(TransactionType type) throws Exception {
1184         int shardBatchedModificationCount = 3;
1185         dataStoreContextBuilder.shardBatchedModificationCount(shardBatchedModificationCount);
1186
1187         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), type);
1188
1189         expectBatchedModifications(actorRef, shardBatchedModificationCount);
1190
1191         YangInstanceIdentifier writePath1 = TestModel.TEST_PATH;
1192         NormalizedNode<?, ?> writeNode1 = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
1193
1194         YangInstanceIdentifier writePath2 = TestModel.OUTER_LIST_PATH;
1195         NormalizedNode<?, ?> writeNode2 = ImmutableNodes.containerNode(TestModel.OUTER_LIST_QNAME);
1196
1197         YangInstanceIdentifier writePath3 = TestModel.INNER_LIST_PATH;
1198         NormalizedNode<?, ?> writeNode3 = ImmutableNodes.containerNode(TestModel.INNER_LIST_QNAME);
1199
1200         YangInstanceIdentifier mergePath1 = TestModel.TEST_PATH;
1201         NormalizedNode<?, ?> mergeNode1 = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
1202
1203         YangInstanceIdentifier mergePath2 = TestModel.OUTER_LIST_PATH;
1204         NormalizedNode<?, ?> mergeNode2 = ImmutableNodes.containerNode(TestModel.OUTER_LIST_QNAME);
1205
1206         YangInstanceIdentifier mergePath3 = TestModel.INNER_LIST_PATH;
1207         NormalizedNode<?, ?> mergeNode3 = ImmutableNodes.containerNode(TestModel.INNER_LIST_QNAME);
1208
1209         YangInstanceIdentifier deletePath1 = TestModel.TEST_PATH;
1210         YangInstanceIdentifier deletePath2 = TestModel.OUTER_LIST_PATH;
1211
1212         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, type);
1213
1214         transactionProxy.write(writePath1, writeNode1);
1215         transactionProxy.write(writePath2, writeNode2);
1216         transactionProxy.delete(deletePath1);
1217         transactionProxy.merge(mergePath1, mergeNode1);
1218         transactionProxy.merge(mergePath2, mergeNode2);
1219         transactionProxy.write(writePath3, writeNode3);
1220         transactionProxy.merge(mergePath3, mergeNode3);
1221         transactionProxy.delete(deletePath2);
1222
1223         // This sends the last batch.
1224         transactionProxy.ready();
1225
1226         List<BatchedModifications> batchedModifications = captureBatchedModifications(actorRef);
1227         assertEquals("Captured BatchedModifications count", 3, batchedModifications.size());
1228
1229         verifyBatchedModifications(batchedModifications.get(0), false, new WriteModification(writePath1, writeNode1),
1230                 new WriteModification(writePath2, writeNode2), new DeleteModification(deletePath1));
1231
1232         verifyBatchedModifications(batchedModifications.get(1), false, new MergeModification(mergePath1, mergeNode1),
1233                 new MergeModification(mergePath2, mergeNode2), new WriteModification(writePath3, writeNode3));
1234
1235         verifyBatchedModifications(batchedModifications.get(2), true, true,
1236                 new MergeModification(mergePath3, mergeNode3), new DeleteModification(deletePath2));
1237
1238         assertEquals("getTotalMessageCount", 3, batchedModifications.get(2).getTotalMessagesSent());
1239     }
1240
1241     @Test
1242     public void testReadWriteModificationOperationBatching() throws Throwable {
1243         testModificationOperationBatching(READ_WRITE);
1244     }
1245
1246     @Test
1247     public void testWriteOnlyModificationOperationBatching() throws Throwable {
1248         testModificationOperationBatching(WRITE_ONLY);
1249     }
1250
1251     @Test
1252     public void testOptimizedWriteOnlyModificationOperationBatching() throws Throwable {
1253         dataStoreContextBuilder.writeOnlyTransactionOptimizationsEnabled(true);
1254         testModificationOperationBatching(WRITE_ONLY);
1255     }
1256
1257     @Test
1258     public void testModificationOperationBatchingWithInterleavedReads() throws Throwable {
1259
1260         int shardBatchedModificationCount = 10;
1261         dataStoreContextBuilder.shardBatchedModificationCount(shardBatchedModificationCount);
1262
1263         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), READ_WRITE);
1264
1265         expectBatchedModifications(actorRef, shardBatchedModificationCount);
1266
1267         YangInstanceIdentifier writePath1 = TestModel.TEST_PATH;
1268         NormalizedNode<?, ?> writeNode1 = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
1269
1270         YangInstanceIdentifier writePath2 = TestModel.OUTER_LIST_PATH;
1271         NormalizedNode<?, ?> writeNode2 = ImmutableNodes.containerNode(TestModel.OUTER_LIST_QNAME);
1272
1273         YangInstanceIdentifier mergePath1 = TestModel.TEST_PATH;
1274         NormalizedNode<?, ?> mergeNode1 = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
1275
1276         YangInstanceIdentifier mergePath2 = TestModel.INNER_LIST_PATH;
1277         NormalizedNode<?, ?> mergeNode2 = ImmutableNodes.containerNode(TestModel.INNER_LIST_QNAME);
1278
1279         YangInstanceIdentifier deletePath = TestModel.OUTER_LIST_PATH;
1280
1281         doReturn(readSerializedDataReply(writeNode2)).when(mockActorContext).executeOperationAsync(
1282                 eq(actorSelection(actorRef)), eqSerializedReadData(writePath2));
1283
1284         doReturn(readSerializedDataReply(mergeNode2)).when(mockActorContext).executeOperationAsync(
1285                 eq(actorSelection(actorRef)), eqSerializedReadData(mergePath2));
1286
1287         doReturn(dataExistsSerializedReply(true)).when(mockActorContext).executeOperationAsync(
1288                 eq(actorSelection(actorRef)), eqSerializedDataExists());
1289
1290         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, READ_WRITE);
1291
1292         transactionProxy.write(writePath1, writeNode1);
1293         transactionProxy.write(writePath2, writeNode2);
1294
1295         Optional<NormalizedNode<?, ?>> readOptional = transactionProxy.read(writePath2).
1296                 get(5, TimeUnit.SECONDS);
1297
1298         assertEquals("NormalizedNode isPresent", true, readOptional.isPresent());
1299         assertEquals("Response NormalizedNode", writeNode2, readOptional.get());
1300
1301         transactionProxy.merge(mergePath1, mergeNode1);
1302         transactionProxy.merge(mergePath2, mergeNode2);
1303
1304         readOptional = transactionProxy.read(mergePath2).get(5, TimeUnit.SECONDS);
1305
1306         transactionProxy.delete(deletePath);
1307
1308         Boolean exists = transactionProxy.exists(TestModel.TEST_PATH).checkedGet();
1309         assertEquals("Exists response", true, exists);
1310
1311         assertEquals("NormalizedNode isPresent", true, readOptional.isPresent());
1312         assertEquals("Response NormalizedNode", mergeNode2, readOptional.get());
1313
1314         List<BatchedModifications> batchedModifications = captureBatchedModifications(actorRef);
1315         assertEquals("Captured BatchedModifications count", 3, batchedModifications.size());
1316
1317         verifyBatchedModifications(batchedModifications.get(0), false, new WriteModification(writePath1, writeNode1),
1318                 new WriteModification(writePath2, writeNode2));
1319
1320         verifyBatchedModifications(batchedModifications.get(1), false, new MergeModification(mergePath1, mergeNode1),
1321                 new MergeModification(mergePath2, mergeNode2));
1322
1323         verifyBatchedModifications(batchedModifications.get(2), false, new DeleteModification(deletePath));
1324
1325         InOrder inOrder = Mockito.inOrder(mockActorContext);
1326         inOrder.verify(mockActorContext).executeOperationAsync(
1327                 eq(actorSelection(actorRef)), isA(BatchedModifications.class));
1328
1329         inOrder.verify(mockActorContext).executeOperationAsync(
1330                 eq(actorSelection(actorRef)), eqSerializedReadData(writePath2));
1331
1332         inOrder.verify(mockActorContext).executeOperationAsync(
1333                 eq(actorSelection(actorRef)), isA(BatchedModifications.class));
1334
1335         inOrder.verify(mockActorContext).executeOperationAsync(
1336                 eq(actorSelection(actorRef)), eqSerializedReadData(mergePath2));
1337
1338         inOrder.verify(mockActorContext).executeOperationAsync(
1339                 eq(actorSelection(actorRef)), isA(BatchedModifications.class));
1340
1341         inOrder.verify(mockActorContext).executeOperationAsync(
1342                 eq(actorSelection(actorRef)), eqSerializedDataExists());
1343     }
1344
1345     @Test
1346     public void testReadRoot() throws ReadFailedException, InterruptedException, ExecutionException, java.util.concurrent.TimeoutException {
1347
1348         SchemaContext schemaContext = SchemaContextHelper.full();
1349         Configuration configuration = mock(Configuration.class);
1350         doReturn(configuration).when(mockActorContext).getConfiguration();
1351         doReturn(schemaContext).when(mockActorContext).getSchemaContext();
1352         doReturn(Sets.newHashSet("test", "cars")).when(configuration).getAllShardNames();
1353
1354         NormalizedNode<?, ?> expectedNode1 = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
1355         NormalizedNode<?, ?> expectedNode2 = ImmutableNodes.containerNode(CarsModel.CARS_QNAME);
1356
1357         setUpReadData("test", NormalizedNodeAggregatorTest.getRootNode(expectedNode1, schemaContext));
1358         setUpReadData("cars", NormalizedNodeAggregatorTest.getRootNode(expectedNode2, schemaContext));
1359
1360         doReturn(memberName).when(mockActorContext).getCurrentMemberName();
1361
1362         doReturn(10).when(mockActorContext).getTransactionOutstandingOperationLimit();
1363
1364         doReturn(getSystem().dispatchers().defaultGlobalDispatcher()).when(mockActorContext).getClientDispatcher();
1365
1366         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext, READ_ONLY);
1367
1368         Optional<NormalizedNode<?, ?>> readOptional = transactionProxy.read(
1369                 YangInstanceIdentifier.builder().build()).get(5, TimeUnit.SECONDS);
1370
1371         assertEquals("NormalizedNode isPresent", true, readOptional.isPresent());
1372
1373         NormalizedNode<?, ?> normalizedNode = readOptional.get();
1374
1375         assertTrue("Expect value to be a Collection", normalizedNode.getValue() instanceof Collection);
1376
1377         Collection<NormalizedNode<?,?>> collection = (Collection<NormalizedNode<?,?>>) normalizedNode.getValue();
1378
1379         for(NormalizedNode<?,?> node : collection){
1380             assertTrue("Expected " + node + " to be a ContainerNode", node instanceof ContainerNode);
1381         }
1382
1383         assertTrue("Child with QName = " + TestModel.TEST_QNAME + " not found",
1384                 NormalizedNodeAggregatorTest.findChildWithQName(collection, TestModel.TEST_QNAME) != null);
1385
1386         assertEquals(expectedNode1, NormalizedNodeAggregatorTest.findChildWithQName(collection, TestModel.TEST_QNAME));
1387
1388         assertTrue("Child with QName = " + CarsModel.BASE_QNAME + " not found",
1389                 NormalizedNodeAggregatorTest.findChildWithQName(collection, CarsModel.BASE_QNAME) != null);
1390
1391         assertEquals(expectedNode2, NormalizedNodeAggregatorTest.findChildWithQName(collection, CarsModel.BASE_QNAME));
1392     }
1393
1394
1395     private void setUpReadData(String shardName, NormalizedNode<?, ?> expectedNode) {
1396         ActorSystem actorSystem = getSystem();
1397         ActorRef shardActorRef = getSystem().actorOf(Props.create(DoNothingActor.class));
1398
1399         doReturn(getSystem().actorSelection(shardActorRef.path())).
1400                 when(mockActorContext).actorSelection(shardActorRef.path().toString());
1401
1402         doReturn(primaryShardInfoReply(getSystem(), shardActorRef)).
1403                 when(mockActorContext).findPrimaryShardAsync(eq(shardName));
1404
1405         doReturn(true).when(mockActorContext).isPathLocal(shardActorRef.path().toString());
1406
1407         ActorRef txActorRef = actorSystem.actorOf(Props.create(DoNothingActor.class));
1408
1409         doReturn(actorSystem.actorSelection(txActorRef.path())).
1410                 when(mockActorContext).actorSelection(txActorRef.path().toString());
1411
1412         doReturn(Futures.successful(createTransactionReply(txActorRef, DataStoreVersions.CURRENT_VERSION))).when(mockActorContext).
1413                 executeOperationAsync(eq(actorSystem.actorSelection(shardActorRef.path())),
1414                         eqCreateTransaction(memberName, TransactionType.READ_ONLY));
1415
1416         doReturn(readSerializedDataReply(expectedNode)).when(mockActorContext).executeOperationAsync(
1417                 eq(actorSelection(txActorRef)), eqSerializedReadData(YangInstanceIdentifier.builder().build()));
1418     }
1419 }