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