Merge changes I880310f2,I9f437328,I552372db,I587fb203,I05f0bd94, ...
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / TransactionProxy.java
1 /*
2  * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
3  *
4  * This program and the accompanying materials are made available under the
5  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6  * and is available at http://www.eclipse.org/legal/epl-v10.html
7  */
8
9 package org.opendaylight.controller.cluster.datastore;
10
11 import akka.actor.ActorSelection;
12 import akka.dispatch.Mapper;
13 import akka.dispatch.OnComplete;
14 import com.google.common.annotations.VisibleForTesting;
15 import com.google.common.base.FinalizablePhantomReference;
16 import com.google.common.base.FinalizableReferenceQueue;
17 import com.google.common.base.Optional;
18 import com.google.common.base.Preconditions;
19 import com.google.common.collect.Lists;
20 import com.google.common.util.concurrent.CheckedFuture;
21 import com.google.common.util.concurrent.FutureCallback;
22 import com.google.common.util.concurrent.Futures;
23 import com.google.common.util.concurrent.SettableFuture;
24 import java.util.HashMap;
25 import java.util.List;
26 import java.util.Map;
27 import java.util.concurrent.ConcurrentHashMap;
28 import java.util.concurrent.TimeUnit;
29 import java.util.concurrent.atomic.AtomicBoolean;
30 import java.util.concurrent.atomic.AtomicLong;
31 import javax.annotation.concurrent.GuardedBy;
32 import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException;
33 import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier;
34 import org.opendaylight.controller.cluster.datastore.messages.CloseTransaction;
35 import org.opendaylight.controller.cluster.datastore.messages.CreateTransaction;
36 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionReply;
37 import org.opendaylight.controller.cluster.datastore.messages.DataExists;
38 import org.opendaylight.controller.cluster.datastore.messages.DataExistsReply;
39 import org.opendaylight.controller.cluster.datastore.messages.DeleteData;
40 import org.opendaylight.controller.cluster.datastore.messages.MergeData;
41 import org.opendaylight.controller.cluster.datastore.messages.ReadData;
42 import org.opendaylight.controller.cluster.datastore.messages.ReadDataReply;
43 import org.opendaylight.controller.cluster.datastore.messages.ReadyTransaction;
44 import org.opendaylight.controller.cluster.datastore.messages.ReadyTransactionReply;
45 import org.opendaylight.controller.cluster.datastore.messages.SerializableMessage;
46 import org.opendaylight.controller.cluster.datastore.messages.VersionedSerializableMessage;
47 import org.opendaylight.controller.cluster.datastore.messages.WriteData;
48 import org.opendaylight.controller.cluster.datastore.shardstrategy.ShardStrategyFactory;
49 import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
50 import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
51 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadWriteTransaction;
52 import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
53 import org.opendaylight.yangtools.util.concurrent.MappingCheckedFuture;
54 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
55 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
56 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
57 import org.slf4j.Logger;
58 import org.slf4j.LoggerFactory;
59 import scala.concurrent.Future;
60 import scala.concurrent.Promise;
61 import scala.concurrent.duration.FiniteDuration;
62
63 /**
64  * TransactionProxy acts as a proxy for one or more transactions that were created on a remote shard
65  * <p>
66  * Creating a transaction on the consumer side will create one instance of a transaction proxy. If during
67  * the transaction reads and writes are done on data that belongs to different shards then a separate transaction will
68  * be created on each of those shards by the TransactionProxy
69  *</p>
70  * <p>
71  * The TransactionProxy does not make any guarantees about atomicity or order in which the transactions on the various
72  * shards will be executed.
73  * </p>
74  */
75 public class TransactionProxy implements DOMStoreReadWriteTransaction {
76
77     public static enum TransactionType {
78         READ_ONLY,
79         WRITE_ONLY,
80         READ_WRITE
81     }
82
83     static final Mapper<Throwable, Throwable> SAME_FAILURE_TRANSFORMER =
84                                                               new Mapper<Throwable, Throwable>() {
85         @Override
86         public Throwable apply(Throwable failure) {
87             return failure;
88         }
89     };
90
91     private static final AtomicLong counter = new AtomicLong();
92
93     private static final Logger LOG = LoggerFactory.getLogger(TransactionProxy.class);
94
95     /**
96      * Time interval in between transaction create retries.
97      */
98     private static final FiniteDuration CREATE_TX_TRY_INTERVAL =
99             FiniteDuration.create(1, TimeUnit.SECONDS);
100
101     /**
102      * Used to enqueue the PhantomReferences for read-only TransactionProxy instances. The
103      * FinalizableReferenceQueue is safe to use statically in an OSGi environment as it uses some
104      * trickery to clean up its internal thread when the bundle is unloaded.
105      */
106     private static final FinalizableReferenceQueue phantomReferenceQueue =
107                                                                   new FinalizableReferenceQueue();
108
109     /**
110      * This stores the TransactionProxyCleanupPhantomReference instances statically, This is
111      * necessary because PhantomReferences need a hard reference so they're not garbage collected.
112      * Once finalized, the TransactionProxyCleanupPhantomReference removes itself from this map
113      * and thus becomes eligible for garbage collection.
114      */
115     private static final Map<TransactionProxyCleanupPhantomReference,
116                              TransactionProxyCleanupPhantomReference> phantomReferenceCache =
117                                                                         new ConcurrentHashMap<>();
118
119     /**
120      * A PhantomReference that closes remote transactions for a TransactionProxy when it's
121      * garbage collected. This is used for read-only transactions as they're not explicitly closed
122      * by clients. So the only way to detect that a transaction is no longer in use and it's safe
123      * to clean up is when it's garbage collected. It's inexact as to when an instance will be GC'ed
124      * but TransactionProxy instances should generally be short-lived enough to avoid being moved
125      * to the old generation space and thus should be cleaned up in a timely manner as the GC
126      * runs on the young generation (eden, swap1...) space much more frequently.
127      */
128     private static class TransactionProxyCleanupPhantomReference
129                                            extends FinalizablePhantomReference<TransactionProxy> {
130
131         private final List<ActorSelection> remoteTransactionActors;
132         private final AtomicBoolean remoteTransactionActorsMB;
133         private final ActorContext actorContext;
134         private final TransactionIdentifier identifier;
135
136         protected TransactionProxyCleanupPhantomReference(TransactionProxy referent) {
137             super(referent, phantomReferenceQueue);
138
139             // Note we need to cache the relevant fields from the TransactionProxy as we can't
140             // have a hard reference to the TransactionProxy instance itself.
141
142             remoteTransactionActors = referent.remoteTransactionActors;
143             remoteTransactionActorsMB = referent.remoteTransactionActorsMB;
144             actorContext = referent.actorContext;
145             identifier = referent.identifier;
146         }
147
148         @Override
149         public void finalizeReferent() {
150             LOG.trace("Cleaning up {} Tx actors for TransactionProxy {}",
151                     remoteTransactionActors.size(), identifier);
152
153             phantomReferenceCache.remove(this);
154
155             // Access the memory barrier volatile to ensure all previous updates to the
156             // remoteTransactionActors list are visible to this thread.
157
158             if(remoteTransactionActorsMB.get()) {
159                 for(ActorSelection actor : remoteTransactionActors) {
160                     LOG.trace("Sending CloseTransaction to {}", actor);
161                     actorContext.sendOperationAsync(actor, CloseTransaction.INSTANCE.toSerializable());
162                 }
163             }
164         }
165     }
166
167     /**
168      * Stores the remote Tx actors for each requested data store path to be used by the
169      * PhantomReference to close the remote Tx's. This is only used for read-only Tx's. The
170      * remoteTransactionActorsMB volatile serves as a memory barrier to publish updates to the
171      * remoteTransactionActors list so they will be visible to the thread accessing the
172      * PhantomReference.
173      */
174     private List<ActorSelection> remoteTransactionActors;
175     private AtomicBoolean remoteTransactionActorsMB;
176
177     /**
178      * Stores the create transaction results per shard.
179      */
180     private final Map<String, TransactionFutureCallback> txFutureCallbackMap = new HashMap<>();
181
182     private final TransactionType transactionType;
183     private final ActorContext actorContext;
184     private final TransactionIdentifier identifier;
185     private final String transactionChainId;
186     private final SchemaContext schemaContext;
187     private boolean inReadyState;
188
189     public TransactionProxy(ActorContext actorContext, TransactionType transactionType) {
190         this(actorContext, transactionType, "");
191     }
192
193     public TransactionProxy(ActorContext actorContext, TransactionType transactionType,
194             String transactionChainId) {
195         this.actorContext = Preconditions.checkNotNull(actorContext,
196             "actorContext should not be null");
197         this.transactionType = Preconditions.checkNotNull(transactionType,
198             "transactionType should not be null");
199         this.schemaContext = Preconditions.checkNotNull(actorContext.getSchemaContext(),
200             "schemaContext should not be null");
201         this.transactionChainId = transactionChainId;
202
203         String memberName = actorContext.getCurrentMemberName();
204         if(memberName == null){
205             memberName = "UNKNOWN-MEMBER";
206         }
207
208         this.identifier = TransactionIdentifier.builder().memberName(memberName).counter(
209             counter.getAndIncrement()).build();
210
211         if(transactionType == TransactionType.READ_ONLY) {
212             // Read-only Tx's aren't explicitly closed by the client so we create a PhantomReference
213             // to close the remote Tx's when this instance is no longer in use and is garbage
214             // collected.
215
216             remoteTransactionActors = Lists.newArrayList();
217             remoteTransactionActorsMB = new AtomicBoolean();
218
219             TransactionProxyCleanupPhantomReference cleanup =
220                 new TransactionProxyCleanupPhantomReference(this);
221             phantomReferenceCache.put(cleanup, cleanup);
222         }
223
224         LOG.debug("Created txn {} of type {} on chain {}", identifier, transactionType, transactionChainId);
225     }
226
227     @VisibleForTesting
228     List<Future<Object>> getRecordedOperationFutures() {
229         List<Future<Object>> recordedOperationFutures = Lists.newArrayList();
230         for(TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
231             TransactionContext transactionContext = txFutureCallback.getTransactionContext();
232             if(transactionContext != null) {
233                 recordedOperationFutures.addAll(transactionContext.getRecordedOperationFutures());
234             }
235         }
236
237         return recordedOperationFutures;
238     }
239
240     @VisibleForTesting
241     boolean hasTransactionContext() {
242         for(TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
243             TransactionContext transactionContext = txFutureCallback.getTransactionContext();
244             if(transactionContext != null) {
245                 return true;
246             }
247         }
248
249         return false;
250     }
251
252     @Override
253     public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> read(final YangInstanceIdentifier path) {
254
255         Preconditions.checkState(transactionType != TransactionType.WRITE_ONLY,
256                 "Read operation on write-only transaction is not allowed");
257
258         LOG.debug("Tx {} read {}", identifier, path);
259
260         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
261         return txFutureCallback.enqueueReadOperation(new ReadOperation<Optional<NormalizedNode<?, ?>>>() {
262             @Override
263             public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> invoke(
264                     TransactionContext transactionContext) {
265                 return transactionContext.readData(path);
266             }
267         });
268     }
269
270     @Override
271     public CheckedFuture<Boolean, ReadFailedException> exists(final YangInstanceIdentifier path) {
272
273         Preconditions.checkState(transactionType != TransactionType.WRITE_ONLY,
274                 "Exists operation on write-only transaction is not allowed");
275
276         LOG.debug("Tx {} exists {}", identifier, path);
277
278         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
279         return txFutureCallback.enqueueReadOperation(new ReadOperation<Boolean>() {
280             @Override
281             public CheckedFuture<Boolean, ReadFailedException> invoke(TransactionContext transactionContext) {
282                 return transactionContext.dataExists(path);
283             }
284         });
285     }
286
287
288     private void checkModificationState() {
289         Preconditions.checkState(transactionType != TransactionType.READ_ONLY,
290                 "Modification operation on read-only transaction is not allowed");
291         Preconditions.checkState(!inReadyState,
292                 "Transaction is sealed - further modifications are not allowed");
293     }
294
295     @Override
296     public void write(final YangInstanceIdentifier path, final NormalizedNode<?, ?> data) {
297
298         checkModificationState();
299
300         LOG.debug("Tx {} write {}", identifier, path);
301
302         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
303         txFutureCallback.enqueueModifyOperation(new TransactionOperation() {
304             @Override
305             public void invoke(TransactionContext transactionContext) {
306                 transactionContext.writeData(path, data);
307             }
308         });
309     }
310
311     @Override
312     public void merge(final YangInstanceIdentifier path, final NormalizedNode<?, ?> data) {
313
314         checkModificationState();
315
316         LOG.debug("Tx {} merge {}", identifier, path);
317
318         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
319         txFutureCallback.enqueueModifyOperation(new TransactionOperation() {
320             @Override
321             public void invoke(TransactionContext transactionContext) {
322                 transactionContext.mergeData(path, data);
323             }
324         });
325     }
326
327     @Override
328     public void delete(final YangInstanceIdentifier path) {
329
330         checkModificationState();
331
332         LOG.debug("Tx {} delete {}", identifier, path);
333
334         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
335         txFutureCallback.enqueueModifyOperation(new TransactionOperation() {
336             @Override
337             public void invoke(TransactionContext transactionContext) {
338                 transactionContext.deleteData(path);
339             }
340         });
341     }
342
343     @Override
344     public DOMStoreThreePhaseCommitCohort ready() {
345
346         checkModificationState();
347
348         inReadyState = true;
349
350         LOG.debug("Tx {} Readying {} transactions for commit", identifier,
351                     txFutureCallbackMap.size());
352
353         List<Future<ActorSelection>> cohortFutures = Lists.newArrayList();
354
355         for(TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
356
357             LOG.debug("Tx {} Readying transaction for shard {} chain {}", identifier,
358                         txFutureCallback.getShardName(), transactionChainId);
359
360             Future<ActorSelection> future = txFutureCallback.enqueueFutureOperation(new FutureOperation<ActorSelection>() {
361                 @Override
362                 public Future<ActorSelection> invoke(TransactionContext transactionContext) {
363                     return transactionContext.readyTransaction();
364                 }
365             });
366
367             cohortFutures.add(future);
368         }
369
370         onTransactionReady(cohortFutures);
371
372         return new ThreePhaseCommitCohortProxy(actorContext, cohortFutures,
373                 identifier.toString());
374     }
375
376     /**
377      * Method for derived classes to be notified when the transaction has been readied.
378      *
379      * @param cohortFutures the cohort Futures for each shard transaction.
380      */
381     protected void onTransactionReady(List<Future<ActorSelection>> cohortFutures) {
382     }
383
384     /**
385      * Method called to send a CreateTransaction message to a shard.
386      *
387      * @param shard the shard actor to send to
388      * @param serializedCreateMessage the serialized message to send
389      * @return the response Future
390      */
391     protected Future<Object> sendCreateTransaction(ActorSelection shard,
392             Object serializedCreateMessage) {
393         return actorContext.executeOperationAsync(shard, serializedCreateMessage);
394     }
395
396     @Override
397     public Object getIdentifier() {
398         return this.identifier;
399     }
400
401     @Override
402     public void close() {
403         for (TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
404             txFutureCallback.enqueueModifyOperation(new TransactionOperation() {
405                 @Override
406                 public void invoke(TransactionContext transactionContext) {
407                     transactionContext.closeTransaction();
408                 }
409             });
410         }
411
412         txFutureCallbackMap.clear();
413
414         if(transactionType == TransactionType.READ_ONLY) {
415             remoteTransactionActors.clear();
416             remoteTransactionActorsMB.set(true);
417         }
418     }
419
420     private String shardNameFromIdentifier(YangInstanceIdentifier path){
421         return ShardStrategyFactory.getStrategy(path).findShard(path);
422     }
423
424     private TransactionFutureCallback getOrCreateTxFutureCallback(YangInstanceIdentifier path) {
425         String shardName = shardNameFromIdentifier(path);
426         TransactionFutureCallback txFutureCallback = txFutureCallbackMap.get(shardName);
427         if(txFutureCallback == null) {
428             Future<ActorSelection> findPrimaryFuture = actorContext.findPrimaryShardAsync(shardName);
429
430             final TransactionFutureCallback newTxFutureCallback =
431                     new TransactionFutureCallback(shardName);
432
433             txFutureCallback = newTxFutureCallback;
434             txFutureCallbackMap.put(shardName, txFutureCallback);
435
436             findPrimaryFuture.onComplete(new OnComplete<ActorSelection>() {
437                 @Override
438                 public void onComplete(Throwable failure, ActorSelection primaryShard) {
439                     if(failure != null) {
440                         newTxFutureCallback.onComplete(failure, null);
441                     } else {
442                         newTxFutureCallback.setPrimaryShard(primaryShard);
443                     }
444                 }
445             }, actorContext.getActorSystem().dispatcher());
446         }
447
448         return txFutureCallback;
449     }
450
451     public String getTransactionChainId() {
452         return transactionChainId;
453     }
454
455     protected ActorContext getActorContext() {
456         return actorContext;
457     }
458
459     /**
460      * Interfaces for transaction operations to be invoked later.
461      */
462     private static interface TransactionOperation {
463         void invoke(TransactionContext transactionContext);
464     }
465
466     /**
467      * This interface returns a Guava Future
468      */
469     private static interface ReadOperation<T> {
470         CheckedFuture<T, ReadFailedException> invoke(TransactionContext transactionContext);
471     }
472
473     /**
474      * This interface returns a Scala Future
475      */
476     private static interface FutureOperation<T> {
477         Future<T> invoke(TransactionContext transactionContext);
478     }
479
480     /**
481      * Implements a Future OnComplete callback for a CreateTransaction message. This class handles
482      * retries, up to a limit, if the shard doesn't have a leader yet. This is done by scheduling a
483      * retry task after a short delay.
484      * <p>
485      * The end result from a completed CreateTransaction message is a TransactionContext that is
486      * used to perform transaction operations. Transaction operations that occur before the
487      * CreateTransaction completes are cache and executed once the CreateTransaction completes,
488      * successfully or not.
489      */
490     private class TransactionFutureCallback extends OnComplete<Object> {
491
492         /**
493          * The list of transaction operations to execute once the CreateTransaction completes.
494          */
495         @GuardedBy("txOperationsOnComplete")
496         private final List<TransactionOperation> txOperationsOnComplete = Lists.newArrayList();
497
498         /**
499          * The TransactionContext resulting from the CreateTransaction reply.
500          */
501         private volatile TransactionContext transactionContext;
502
503         /**
504          * The target primary shard.
505          */
506         private volatile ActorSelection primaryShard;
507
508         private volatile int createTxTries = (int) (actorContext.getDatastoreContext().
509                 getShardLeaderElectionTimeout().duration().toMillis() /
510                 CREATE_TX_TRY_INTERVAL.toMillis());
511
512         private final String shardName;
513
514         TransactionFutureCallback(String shardName) {
515             this.shardName = shardName;
516         }
517
518         String getShardName() {
519             return shardName;
520         }
521
522         TransactionContext getTransactionContext() {
523             return transactionContext;
524         }
525
526
527         /**
528          * Sets the target primary shard and initiates a CreateTransaction try.
529          */
530         void setPrimaryShard(ActorSelection primaryShard) {
531             LOG.debug("Tx {} Primary shard found - trying create transaction", identifier);
532
533             this.primaryShard = primaryShard;
534             tryCreateTransaction();
535         }
536
537         /**
538          * Adds a TransactionOperation to be executed after the CreateTransaction completes.
539          */
540         void addTxOperationOnComplete(TransactionOperation operation) {
541             synchronized(txOperationsOnComplete) {
542                 if(transactionContext == null) {
543                     LOG.debug("Tx {} Adding operation on complete {}", identifier);
544
545                     txOperationsOnComplete.add(operation);
546                 } else {
547                     operation.invoke(transactionContext);
548                 }
549             }
550         }
551
552
553         <T> Future<T> enqueueFutureOperation(final FutureOperation<T> op) {
554
555             Future<T> future;
556
557             if (transactionContext != null) {
558                 future = op.invoke(transactionContext);
559             } else {
560                 // The shard Tx hasn't been created yet so add the Tx operation to the Tx Future
561                 // callback to be executed after the Tx is created.
562                 final Promise<T> promise = akka.dispatch.Futures.promise();
563                 addTxOperationOnComplete(new TransactionOperation() {
564                     @Override
565                     public void invoke(TransactionContext transactionContext) {
566                         promise.completeWith(op.invoke(transactionContext));
567                     }
568                 });
569
570                 future = promise.future();
571             }
572
573             return future;
574         }
575
576         <T> CheckedFuture<T, ReadFailedException> enqueueReadOperation(final ReadOperation<T> op) {
577
578             CheckedFuture<T, ReadFailedException> future;
579
580             if (transactionContext != null) {
581                 future = op.invoke(transactionContext);
582             } else {
583                 // The shard Tx hasn't been created yet so add the Tx operation to the Tx Future
584                 // callback to be executed after the Tx is created.
585                 final SettableFuture<T> proxyFuture = SettableFuture.create();
586                 addTxOperationOnComplete(new TransactionOperation() {
587                     @Override
588                     public void invoke(TransactionContext transactionContext) {
589                         Futures.addCallback(op.invoke(transactionContext), new FutureCallback<T>() {
590                             @Override
591                             public void onSuccess(T data) {
592                                 proxyFuture.set(data);
593                             }
594
595                             @Override
596                             public void onFailure(Throwable t) {
597                                 proxyFuture.setException(t);
598                             }
599                         });
600                     }
601                 });
602
603                 future = MappingCheckedFuture.create(proxyFuture, ReadFailedException.MAPPER);
604             }
605
606             return future;
607         }
608
609         void enqueueModifyOperation(final TransactionOperation op) {
610
611             if (transactionContext != null) {
612                 op.invoke(transactionContext);
613             } else {
614                 // The shard Tx hasn't been created yet so add the Tx operation to the Tx Future
615                 // callback to be executed after the Tx is created.
616                 addTxOperationOnComplete(op);
617             }
618         }
619
620         /**
621          * Performs a CreateTransaction try async.
622          */
623         private void tryCreateTransaction() {
624             Future<Object> createTxFuture = sendCreateTransaction(primaryShard,
625                     new CreateTransaction(identifier.toString(),
626                             TransactionProxy.this.transactionType.ordinal(),
627                             getTransactionChainId()).toSerializable());
628
629             createTxFuture.onComplete(this, actorContext.getActorSystem().dispatcher());
630         }
631
632         @Override
633         public void onComplete(Throwable failure, Object response) {
634             if(failure instanceof NoShardLeaderException) {
635                 // There's no leader for the shard yet - schedule and try again, unless we're out
636                 // of retries. Note: createTxTries is volatile as it may be written by different
637                 // threads however not concurrently, therefore decrementing it non-atomically here
638                 // is ok.
639                 if(--createTxTries > 0) {
640                     LOG.debug("Tx {} Shard {} has no leader yet - scheduling create Tx retry",
641                             identifier, shardName);
642
643                     actorContext.getActorSystem().scheduler().scheduleOnce(CREATE_TX_TRY_INTERVAL,
644                             new Runnable() {
645                                 @Override
646                                 public void run() {
647                                     tryCreateTransaction();
648                                 }
649                             }, actorContext.getActorSystem().dispatcher());
650                     return;
651                 }
652             }
653
654             // Create the TransactionContext from the response or failure and execute delayed
655             // TransactionOperations. This entire section is done atomically (ie synchronized) with
656             // respect to #addTxOperationOnComplete to handle timing issues and ensure no
657             // TransactionOperation is missed and that they are processed in the order they occurred.
658             synchronized(txOperationsOnComplete) {
659                 // Store the new TransactionContext locally until we've completed invoking the
660                 // TransactionOperations. This avoids thread timing issues which could cause
661                 // out-of-order TransactionOperations. Eg, on a modification operation, if the
662                 // TransactionContext is non-null, then we directly call the TransactionContext.
663                 // However, at the same time, the code may be executing the cached
664                 // TransactionOperations. So to avoid thus timing, we don't publish the
665                 // TransactionContext until after we've executed all cached TransactionOperations.
666                 TransactionContext localTransactionContext;
667                 if(failure != null) {
668                     LOG.debug("Tx {} Creating NoOpTransaction because of error: {}", identifier,
669                             failure.getMessage());
670
671                     localTransactionContext = new NoOpTransactionContext(failure, identifier);
672                 } else if (response.getClass().equals(CreateTransactionReply.SERIALIZABLE_CLASS)) {
673                     localTransactionContext = createValidTransactionContext(
674                             CreateTransactionReply.fromSerializable(response));
675                 } else {
676                     IllegalArgumentException exception = new IllegalArgumentException(String.format(
677                         "Invalid reply type %s for CreateTransaction", response.getClass()));
678
679                     localTransactionContext = new NoOpTransactionContext(exception, identifier);
680                 }
681
682                 for(TransactionOperation oper: txOperationsOnComplete) {
683                     oper.invoke(localTransactionContext);
684                 }
685
686                 txOperationsOnComplete.clear();
687
688                 // We're done invoking the TransactionOperations so we can now publish the
689                 // TransactionContext.
690                 transactionContext = localTransactionContext;
691             }
692         }
693
694         private TransactionContext createValidTransactionContext(CreateTransactionReply reply) {
695             String transactionPath = reply.getTransactionPath();
696
697             LOG.debug("Tx {} Received transaction actor path {}", identifier, transactionPath);
698
699             ActorSelection transactionActor = actorContext.actorSelection(transactionPath);
700
701             if (transactionType == TransactionType.READ_ONLY) {
702                 // Add the actor to the remoteTransactionActors list for access by the
703                 // cleanup PhantonReference.
704                 remoteTransactionActors.add(transactionActor);
705
706                 // Write to the memory barrier volatile to publish the above update to the
707                 // remoteTransactionActors list for thread visibility.
708                 remoteTransactionActorsMB.set(true);
709             }
710
711             // TxActor is always created where the leader of the shard is.
712             // Check if TxActor is created in the same node
713             boolean isTxActorLocal = actorContext.isPathLocal(transactionPath);
714
715             return new TransactionContextImpl(transactionPath, transactionActor, identifier,
716                 actorContext, schemaContext, isTxActorLocal, reply.getVersion());
717         }
718     }
719
720     private interface TransactionContext {
721         void closeTransaction();
722
723         Future<ActorSelection> readyTransaction();
724
725         void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data);
726
727         void deleteData(YangInstanceIdentifier path);
728
729         void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data);
730
731         CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readData(
732                 final YangInstanceIdentifier path);
733
734         CheckedFuture<Boolean, ReadFailedException> dataExists(YangInstanceIdentifier path);
735
736         List<Future<Object>> getRecordedOperationFutures();
737     }
738
739     private static abstract class AbstractTransactionContext implements TransactionContext {
740
741         protected final TransactionIdentifier identifier;
742         protected final List<Future<Object>> recordedOperationFutures = Lists.newArrayList();
743
744         AbstractTransactionContext(TransactionIdentifier identifier) {
745             this.identifier = identifier;
746         }
747
748         @Override
749         public List<Future<Object>> getRecordedOperationFutures() {
750             return recordedOperationFutures;
751         }
752     }
753
754     private static class TransactionContextImpl extends AbstractTransactionContext {
755         private final Logger LOG = LoggerFactory.getLogger(TransactionContextImpl.class);
756
757         private final ActorContext actorContext;
758         private final SchemaContext schemaContext;
759         private final String transactionPath;
760         private final ActorSelection actor;
761         private final boolean isTxActorLocal;
762         private final short remoteTransactionVersion;
763
764         private TransactionContextImpl(String transactionPath, ActorSelection actor, TransactionIdentifier identifier,
765                 ActorContext actorContext, SchemaContext schemaContext,
766                 boolean isTxActorLocal, short remoteTransactionVersion) {
767             super(identifier);
768             this.transactionPath = transactionPath;
769             this.actor = actor;
770             this.actorContext = actorContext;
771             this.schemaContext = schemaContext;
772             this.isTxActorLocal = isTxActorLocal;
773             this.remoteTransactionVersion = remoteTransactionVersion;
774         }
775
776         private ActorSelection getActor() {
777             return actor;
778         }
779
780         private Future<Object> executeOperationAsync(SerializableMessage msg) {
781             return actorContext.executeOperationAsync(getActor(), isTxActorLocal ? msg : msg.toSerializable());
782         }
783
784         private Future<Object> executeOperationAsync(VersionedSerializableMessage msg) {
785             return actorContext.executeOperationAsync(getActor(), isTxActorLocal ? msg :
786                 msg.toSerializable(remoteTransactionVersion));
787         }
788
789         @Override
790         public void closeTransaction() {
791             LOG.debug("Tx {} closeTransaction called", identifier);
792
793             actorContext.sendOperationAsync(getActor(), CloseTransaction.INSTANCE.toSerializable());
794         }
795
796         @Override
797         public Future<ActorSelection> readyTransaction() {
798             LOG.debug("Tx {} readyTransaction called with {} previous recorded operations pending",
799                     identifier, recordedOperationFutures.size());
800
801             // Send the ReadyTransaction message to the Tx actor.
802
803             final Future<Object> replyFuture = executeOperationAsync(ReadyTransaction.INSTANCE);
804
805             // Combine all the previously recorded put/merge/delete operation reply Futures and the
806             // ReadyTransactionReply Future into one Future. If any one fails then the combined
807             // Future will fail. We need all prior operations and the ready operation to succeed
808             // in order to attempt commit.
809
810             List<Future<Object>> futureList =
811                     Lists.newArrayListWithCapacity(recordedOperationFutures.size() + 1);
812             futureList.addAll(recordedOperationFutures);
813             futureList.add(replyFuture);
814
815             Future<Iterable<Object>> combinedFutures = akka.dispatch.Futures.sequence(futureList,
816                     actorContext.getActorSystem().dispatcher());
817
818             // Transform the combined Future into a Future that returns the cohort actor path from
819             // the ReadyTransactionReply. That's the end result of the ready operation.
820
821             return combinedFutures.transform(new Mapper<Iterable<Object>, ActorSelection>() {
822                 @Override
823                 public ActorSelection checkedApply(Iterable<Object> notUsed) {
824                     LOG.debug("Tx {} readyTransaction: pending recorded operations succeeded",
825                             identifier);
826
827                     // At this point all the Futures succeeded and we need to extract the cohort
828                     // actor path from the ReadyTransactionReply. For the recorded operations, they
829                     // don't return any data so we're only interested that they completed
830                     // successfully. We could be paranoid and verify the correct reply types but
831                     // that really should never happen so it's not worth the overhead of
832                     // de-serializing each reply.
833
834                     // Note the Future get call here won't block as it's complete.
835                     Object serializedReadyReply = replyFuture.value().get().get();
836                     if (serializedReadyReply instanceof ReadyTransactionReply) {
837                         return actorContext.actorSelection(((ReadyTransactionReply)serializedReadyReply).getCohortPath());
838
839                     } else if(serializedReadyReply.getClass().equals(ReadyTransactionReply.SERIALIZABLE_CLASS)) {
840                         ReadyTransactionReply reply = ReadyTransactionReply.fromSerializable(serializedReadyReply);
841                         String cohortPath = reply.getCohortPath();
842
843                         // In Helium we used to return the local path of the actor which represented
844                         // a remote ThreePhaseCommitCohort. The local path would then be converted to
845                         // a remote path using this resolvePath method. To maintain compatibility with
846                         // a Helium node we need to continue to do this conversion.
847                         // At some point in the future when upgrades from Helium are not supported
848                         // we could remove this code to resolvePath and just use the cohortPath as the
849                         // resolved cohortPath
850                         if(TransactionContextImpl.this.remoteTransactionVersion <
851                                 DataStoreVersions.HELIUM_1_VERSION) {
852                             cohortPath = actorContext.resolvePath(transactionPath, cohortPath);
853                         }
854
855                         return actorContext.actorSelection(cohortPath);
856
857                     } else {
858                         // Throwing an exception here will fail the Future.
859                         throw new IllegalArgumentException(String.format("Invalid reply type {}",
860                                 serializedReadyReply.getClass()));
861                     }
862                 }
863             }, SAME_FAILURE_TRANSFORMER, actorContext.getActorSystem().dispatcher());
864         }
865
866         @Override
867         public void deleteData(YangInstanceIdentifier path) {
868             LOG.debug("Tx {} deleteData called path = {}", identifier, path);
869
870             recordedOperationFutures.add(executeOperationAsync(new DeleteData(path)));
871         }
872
873         @Override
874         public void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
875             LOG.debug("Tx {} mergeData called path = {}", identifier, path);
876
877             recordedOperationFutures.add(executeOperationAsync(new MergeData(path, data)));
878         }
879
880         @Override
881         public void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
882             LOG.debug("Tx {} writeData called path = {}", identifier, path);
883
884             recordedOperationFutures.add(executeOperationAsync(new WriteData(path, data)));
885         }
886
887         @Override
888         public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readData(
889                 final YangInstanceIdentifier path) {
890
891             LOG.debug("Tx {} readData called path = {}", identifier, path);
892
893             final SettableFuture<Optional<NormalizedNode<?, ?>>> returnFuture = SettableFuture.create();
894
895             // If there were any previous recorded put/merge/delete operation reply Futures then we
896             // must wait for them to successfully complete. This is necessary to honor the read
897             // uncommitted semantics of the public API contract. If any one fails then fail the read.
898
899             if(recordedOperationFutures.isEmpty()) {
900                 finishReadData(path, returnFuture);
901             } else {
902                 LOG.debug("Tx {} readData: verifying {} previous recorded operations",
903                         identifier, recordedOperationFutures.size());
904
905                 // Note: we make a copy of recordedOperationFutures to be on the safe side in case
906                 // Futures#sequence accesses the passed List on a different thread, as
907                 // recordedOperationFutures is not synchronized.
908
909                 Future<Iterable<Object>> combinedFutures = akka.dispatch.Futures.sequence(
910                         Lists.newArrayList(recordedOperationFutures),
911                         actorContext.getActorSystem().dispatcher());
912
913                 OnComplete<Iterable<Object>> onComplete = new OnComplete<Iterable<Object>>() {
914                     @Override
915                     public void onComplete(Throwable failure, Iterable<Object> notUsed)
916                             throws Throwable {
917                         if(failure != null) {
918                             LOG.debug("Tx {} readData: a recorded operation failed: {}",
919                                     identifier, failure);
920                             returnFuture.setException(new ReadFailedException(
921                                     "The read could not be performed because a previous put, merge,"
922                                     + "or delete operation failed", failure));
923                         } else {
924                             finishReadData(path, returnFuture);
925                         }
926                     }
927                 };
928
929                 combinedFutures.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
930             }
931
932             return MappingCheckedFuture.create(returnFuture, ReadFailedException.MAPPER);
933         }
934
935         private void finishReadData(final YangInstanceIdentifier path,
936                 final SettableFuture<Optional<NormalizedNode<?, ?>>> returnFuture) {
937
938             LOG.debug("Tx {} finishReadData called path = {}", identifier, path);
939
940             OnComplete<Object> onComplete = new OnComplete<Object>() {
941                 @Override
942                 public void onComplete(Throwable failure, Object readResponse) throws Throwable {
943                     if(failure != null) {
944                         LOG.debug("Tx {} read operation failed: {}", identifier, failure);
945                         returnFuture.setException(new ReadFailedException(
946                                 "Error reading data for path " + path, failure));
947
948                     } else {
949                         LOG.debug("Tx {} read operation succeeded", identifier, failure);
950
951                         if (readResponse instanceof ReadDataReply) {
952                             ReadDataReply reply = (ReadDataReply) readResponse;
953                             returnFuture.set(Optional.<NormalizedNode<?, ?>>fromNullable(reply.getNormalizedNode()));
954
955                         } else if (ReadDataReply.isSerializedType(readResponse)) {
956                             ReadDataReply reply = ReadDataReply.fromSerializable(readResponse);
957                             returnFuture.set(Optional.<NormalizedNode<?, ?>>fromNullable(reply.getNormalizedNode()));
958
959                         } else {
960                             returnFuture.setException(new ReadFailedException(
961                                 "Invalid response reading data for path " + path));
962                         }
963                     }
964                 }
965             };
966
967             Future<Object> readFuture = executeOperationAsync(new ReadData(path));
968
969             readFuture.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
970         }
971
972         @Override
973         public CheckedFuture<Boolean, ReadFailedException> dataExists(
974                 final YangInstanceIdentifier path) {
975
976             LOG.debug("Tx {} dataExists called path = {}", identifier, path);
977
978             final SettableFuture<Boolean> returnFuture = SettableFuture.create();
979
980             // If there were any previous recorded put/merge/delete operation reply Futures then we
981             // must wait for them to successfully complete. This is necessary to honor the read
982             // uncommitted semantics of the public API contract. If any one fails then fail this
983             // request.
984
985             if(recordedOperationFutures.isEmpty()) {
986                 finishDataExists(path, returnFuture);
987             } else {
988                 LOG.debug("Tx {} dataExists: verifying {} previous recorded operations",
989                         identifier, recordedOperationFutures.size());
990
991                 // Note: we make a copy of recordedOperationFutures to be on the safe side in case
992                 // Futures#sequence accesses the passed List on a different thread, as
993                 // recordedOperationFutures is not synchronized.
994
995                 Future<Iterable<Object>> combinedFutures = akka.dispatch.Futures.sequence(
996                         Lists.newArrayList(recordedOperationFutures),
997                         actorContext.getActorSystem().dispatcher());
998                 OnComplete<Iterable<Object>> onComplete = new OnComplete<Iterable<Object>>() {
999                     @Override
1000                     public void onComplete(Throwable failure, Iterable<Object> notUsed)
1001                             throws Throwable {
1002                         if(failure != null) {
1003                             LOG.debug("Tx {} dataExists: a recorded operation failed: {}",
1004                                     identifier, failure);
1005                             returnFuture.setException(new ReadFailedException(
1006                                     "The data exists could not be performed because a previous "
1007                                     + "put, merge, or delete operation failed", failure));
1008                         } else {
1009                             finishDataExists(path, returnFuture);
1010                         }
1011                     }
1012                 };
1013
1014                 combinedFutures.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
1015             }
1016
1017             return MappingCheckedFuture.create(returnFuture, ReadFailedException.MAPPER);
1018         }
1019
1020         private void finishDataExists(final YangInstanceIdentifier path,
1021                 final SettableFuture<Boolean> returnFuture) {
1022
1023             LOG.debug("Tx {} finishDataExists called path = {}", identifier, path);
1024
1025             OnComplete<Object> onComplete = new OnComplete<Object>() {
1026                 @Override
1027                 public void onComplete(Throwable failure, Object response) throws Throwable {
1028                     if(failure != null) {
1029                         LOG.debug("Tx {} dataExists operation failed: {}", identifier, failure);
1030                         returnFuture.setException(new ReadFailedException(
1031                                 "Error checking data exists for path " + path, failure));
1032                     } else {
1033                         LOG.debug("Tx {} dataExists operation succeeded", identifier, failure);
1034
1035                         if (response instanceof DataExistsReply) {
1036                             returnFuture.set(Boolean.valueOf(((DataExistsReply) response).exists()));
1037
1038                         } else if (response.getClass().equals(DataExistsReply.SERIALIZABLE_CLASS)) {
1039                             returnFuture.set(Boolean.valueOf(DataExistsReply.fromSerializable(response).exists()));
1040
1041                         } else {
1042                             returnFuture.setException(new ReadFailedException(
1043                                     "Invalid response checking exists for path " + path));
1044                         }
1045                     }
1046                 }
1047             };
1048
1049             Future<Object> future = executeOperationAsync(new DataExists(path));
1050
1051             future.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
1052         }
1053     }
1054
1055     private static class NoOpTransactionContext extends AbstractTransactionContext {
1056
1057         private final Logger LOG = LoggerFactory.getLogger(NoOpTransactionContext.class);
1058
1059         private final Throwable failure;
1060
1061         public NoOpTransactionContext(Throwable failure, TransactionIdentifier identifier){
1062             super(identifier);
1063             this.failure = failure;
1064         }
1065
1066         @Override
1067         public void closeTransaction() {
1068             LOG.debug("NoOpTransactionContext {} closeTransaction called", identifier);
1069         }
1070
1071         @Override
1072         public Future<ActorSelection> readyTransaction() {
1073             LOG.debug("Tx {} readyTransaction called", identifier);
1074             return akka.dispatch.Futures.failed(failure);
1075         }
1076
1077         @Override
1078         public void deleteData(YangInstanceIdentifier path) {
1079             LOG.debug("Tx {} deleteData called path = {}", identifier, path);
1080         }
1081
1082         @Override
1083         public void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
1084             LOG.debug("Tx {} mergeData called path = {}", identifier, path);
1085         }
1086
1087         @Override
1088         public void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
1089             LOG.debug("Tx {} writeData called path = {}", identifier, path);
1090         }
1091
1092         @Override
1093         public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readData(
1094                 YangInstanceIdentifier path) {
1095             LOG.debug("Tx {} readData called path = {}", identifier, path);
1096             return Futures.immediateFailedCheckedFuture(new ReadFailedException(
1097                     "Error reading data for path " + path, failure));
1098         }
1099
1100         @Override
1101         public CheckedFuture<Boolean, ReadFailedException> dataExists(
1102                 YangInstanceIdentifier path) {
1103             LOG.debug("Tx {} dataExists called path = {}", identifier, path);
1104             return Futures.immediateFailedCheckedFuture(new ReadFailedException(
1105                     "Error checking exists for path " + path, failure));
1106         }
1107     }
1108 }