Bug 2340: Fixed chained Tx not ready prior next create
[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.WriteData;
46 import org.opendaylight.controller.cluster.datastore.shardstrategy.ShardStrategyFactory;
47 import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
48 import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
49 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadWriteTransaction;
50 import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
51 import org.opendaylight.yangtools.util.concurrent.MappingCheckedFuture;
52 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
53 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
54 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
55 import org.slf4j.Logger;
56 import org.slf4j.LoggerFactory;
57 import scala.concurrent.Future;
58 import scala.concurrent.Promise;
59 import scala.concurrent.duration.FiniteDuration;
60
61 /**
62  * TransactionProxy acts as a proxy for one or more transactions that were created on a remote shard
63  * <p>
64  * Creating a transaction on the consumer side will create one instance of a transaction proxy. If during
65  * the transaction reads and writes are done on data that belongs to different shards then a separate transaction will
66  * be created on each of those shards by the TransactionProxy
67  *</p>
68  * <p>
69  * The TransactionProxy does not make any guarantees about atomicity or order in which the transactions on the various
70  * shards will be executed.
71  * </p>
72  */
73 public class TransactionProxy implements DOMStoreReadWriteTransaction {
74
75     public static enum TransactionType {
76         READ_ONLY,
77         WRITE_ONLY,
78         READ_WRITE
79     }
80
81     static final Mapper<Throwable, Throwable> SAME_FAILURE_TRANSFORMER =
82                                                               new Mapper<Throwable, Throwable>() {
83         @Override
84         public Throwable apply(Throwable failure) {
85             return failure;
86         }
87     };
88
89     private static final AtomicLong counter = new AtomicLong();
90
91     private static final Logger LOG = LoggerFactory.getLogger(TransactionProxy.class);
92
93     /**
94      * Time interval in between transaction create retries.
95      */
96     private static final FiniteDuration CREATE_TX_TRY_INTERVAL =
97             FiniteDuration.create(1, TimeUnit.SECONDS);
98
99     /**
100      * Used to enqueue the PhantomReferences for read-only TransactionProxy instances. The
101      * FinalizableReferenceQueue is safe to use statically in an OSGi environment as it uses some
102      * trickery to clean up its internal thread when the bundle is unloaded.
103      */
104     private static final FinalizableReferenceQueue phantomReferenceQueue =
105                                                                   new FinalizableReferenceQueue();
106
107     /**
108      * This stores the TransactionProxyCleanupPhantomReference instances statically, This is
109      * necessary because PhantomReferences need a hard reference so they're not garbage collected.
110      * Once finalized, the TransactionProxyCleanupPhantomReference removes itself from this map
111      * and thus becomes eligible for garbage collection.
112      */
113     private static final Map<TransactionProxyCleanupPhantomReference,
114                              TransactionProxyCleanupPhantomReference> phantomReferenceCache =
115                                                                         new ConcurrentHashMap<>();
116
117     /**
118      * A PhantomReference that closes remote transactions for a TransactionProxy when it's
119      * garbage collected. This is used for read-only transactions as they're not explicitly closed
120      * by clients. So the only way to detect that a transaction is no longer in use and it's safe
121      * to clean up is when it's garbage collected. It's inexact as to when an instance will be GC'ed
122      * but TransactionProxy instances should generally be short-lived enough to avoid being moved
123      * to the old generation space and thus should be cleaned up in a timely manner as the GC
124      * runs on the young generation (eden, swap1...) space much more frequently.
125      */
126     private static class TransactionProxyCleanupPhantomReference
127                                            extends FinalizablePhantomReference<TransactionProxy> {
128
129         private final List<ActorSelection> remoteTransactionActors;
130         private final AtomicBoolean remoteTransactionActorsMB;
131         private final ActorContext actorContext;
132         private final TransactionIdentifier identifier;
133
134         protected TransactionProxyCleanupPhantomReference(TransactionProxy referent) {
135             super(referent, phantomReferenceQueue);
136
137             // Note we need to cache the relevant fields from the TransactionProxy as we can't
138             // have a hard reference to the TransactionProxy instance itself.
139
140             remoteTransactionActors = referent.remoteTransactionActors;
141             remoteTransactionActorsMB = referent.remoteTransactionActorsMB;
142             actorContext = referent.actorContext;
143             identifier = referent.identifier;
144         }
145
146         @Override
147         public void finalizeReferent() {
148             LOG.trace("Cleaning up {} Tx actors for TransactionProxy {}",
149                     remoteTransactionActors.size(), identifier);
150
151             phantomReferenceCache.remove(this);
152
153             // Access the memory barrier volatile to ensure all previous updates to the
154             // remoteTransactionActors list are visible to this thread.
155
156             if(remoteTransactionActorsMB.get()) {
157                 for(ActorSelection actor : remoteTransactionActors) {
158                     LOG.trace("Sending CloseTransaction to {}", actor);
159                     actorContext.sendOperationAsync(actor,
160                             new CloseTransaction().toSerializable());
161                 }
162             }
163         }
164     }
165
166     /**
167      * Stores the remote Tx actors for each requested data store path to be used by the
168      * PhantomReference to close the remote Tx's. This is only used for read-only Tx's. The
169      * remoteTransactionActorsMB volatile serves as a memory barrier to publish updates to the
170      * remoteTransactionActors list so they will be visible to the thread accessing the
171      * PhantomReference.
172      */
173     private List<ActorSelection> remoteTransactionActors;
174     private AtomicBoolean remoteTransactionActorsMB;
175
176     /**
177      * Stores the create transaction results per shard.
178      */
179     private final Map<String, TransactionFutureCallback> txFutureCallbackMap = new HashMap<>();
180
181     private final TransactionType transactionType;
182     private final ActorContext actorContext;
183     private final TransactionIdentifier identifier;
184     private final String transactionChainId;
185     private final SchemaContext schemaContext;
186     private boolean inReadyState;
187
188     public TransactionProxy(ActorContext actorContext, TransactionType transactionType) {
189         this(actorContext, transactionType, "");
190     }
191
192     public TransactionProxy(ActorContext actorContext, TransactionType transactionType,
193             String transactionChainId) {
194         this.actorContext = Preconditions.checkNotNull(actorContext,
195             "actorContext should not be null");
196         this.transactionType = Preconditions.checkNotNull(transactionType,
197             "transactionType should not be null");
198         this.schemaContext = Preconditions.checkNotNull(actorContext.getSchemaContext(),
199             "schemaContext should not be null");
200         this.transactionChainId = transactionChainId;
201
202         String memberName = actorContext.getCurrentMemberName();
203         if(memberName == null){
204             memberName = "UNKNOWN-MEMBER";
205         }
206
207         this.identifier = TransactionIdentifier.builder().memberName(memberName).counter(
208             counter.getAndIncrement()).build();
209
210         if(transactionType == TransactionType.READ_ONLY) {
211             // Read-only Tx's aren't explicitly closed by the client so we create a PhantomReference
212             // to close the remote Tx's when this instance is no longer in use and is garbage
213             // collected.
214
215             remoteTransactionActors = Lists.newArrayList();
216             remoteTransactionActorsMB = new AtomicBoolean();
217
218             TransactionProxyCleanupPhantomReference cleanup =
219                 new TransactionProxyCleanupPhantomReference(this);
220             phantomReferenceCache.put(cleanup, cleanup);
221         }
222
223         LOG.debug("Created txn {} of type {} on chain {}", identifier, transactionType, transactionChainId);
224     }
225
226     @VisibleForTesting
227     List<Future<Object>> getRecordedOperationFutures() {
228         List<Future<Object>> recordedOperationFutures = Lists.newArrayList();
229         for(TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
230             TransactionContext transactionContext = txFutureCallback.getTransactionContext();
231             if(transactionContext != null) {
232                 recordedOperationFutures.addAll(transactionContext.getRecordedOperationFutures());
233             }
234         }
235
236         return recordedOperationFutures;
237     }
238
239     @VisibleForTesting
240     boolean hasTransactionContext() {
241         for(TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
242             TransactionContext transactionContext = txFutureCallback.getTransactionContext();
243             if(transactionContext != null) {
244                 return true;
245             }
246         }
247
248         return false;
249     }
250
251     @Override
252     public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> read(
253             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         TransactionContext transactionContext = txFutureCallback.getTransactionContext();
262
263         CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> future;
264         if(transactionContext != null) {
265             future = transactionContext.readData(path);
266         } else {
267             // The shard Tx hasn't been created yet so add the Tx operation to the Tx Future
268             // callback to be executed after the Tx is created.
269             final SettableFuture<Optional<NormalizedNode<?, ?>>> proxyFuture = SettableFuture.create();
270             txFutureCallback.addTxOperationOnComplete(new TransactionOperation() {
271                 @Override
272                 public void invoke(TransactionContext transactionContext) {
273                     Futures.addCallback(transactionContext.readData(path),
274                         new FutureCallback<Optional<NormalizedNode<?, ?>>>() {
275                             @Override
276                             public void onSuccess(Optional<NormalizedNode<?, ?>> data) {
277                                 proxyFuture.set(data);
278                             }
279
280                             @Override
281                             public void onFailure(Throwable t) {
282                                 proxyFuture.setException(t);
283                             }
284                         });
285                 }
286             });
287
288             future = MappingCheckedFuture.create(proxyFuture, ReadFailedException.MAPPER);
289         }
290
291         return future;
292     }
293
294     @Override
295     public CheckedFuture<Boolean, ReadFailedException> exists(final YangInstanceIdentifier path) {
296
297         Preconditions.checkState(transactionType != TransactionType.WRITE_ONLY,
298                 "Exists operation on write-only transaction is not allowed");
299
300         LOG.debug("Tx {} exists {}", identifier, path);
301
302         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
303         TransactionContext transactionContext = txFutureCallback.getTransactionContext();
304
305         CheckedFuture<Boolean, ReadFailedException> future;
306         if(transactionContext != null) {
307             future = transactionContext.dataExists(path);
308         } else {
309             // The shard Tx hasn't been created yet so add the Tx operation to the Tx Future
310             // callback to be executed after the Tx is created.
311             final SettableFuture<Boolean> proxyFuture = SettableFuture.create();
312             txFutureCallback.addTxOperationOnComplete(new TransactionOperation() {
313                 @Override
314                 public void invoke(TransactionContext transactionContext) {
315                     Futures.addCallback(transactionContext.dataExists(path),
316                         new FutureCallback<Boolean>() {
317                             @Override
318                             public void onSuccess(Boolean exists) {
319                                 proxyFuture.set(exists);
320                             }
321
322                             @Override
323                             public void onFailure(Throwable t) {
324                                 proxyFuture.setException(t);
325                             }
326                         });
327                 }
328             });
329
330             future = MappingCheckedFuture.create(proxyFuture, ReadFailedException.MAPPER);
331         }
332
333         return future;
334     }
335
336     private void checkModificationState() {
337         Preconditions.checkState(transactionType != TransactionType.READ_ONLY,
338                 "Modification operation on read-only transaction is not allowed");
339         Preconditions.checkState(!inReadyState,
340                 "Transaction is sealed - further modifications are not allowed");
341     }
342
343     @Override
344     public void write(final YangInstanceIdentifier path, final NormalizedNode<?, ?> data) {
345
346         checkModificationState();
347
348         LOG.debug("Tx {} write {}", identifier, path);
349
350         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
351         TransactionContext transactionContext = txFutureCallback.getTransactionContext();
352         if(transactionContext != null) {
353             transactionContext.writeData(path, data);
354         } else {
355             // The shard Tx hasn't been created yet so add the Tx operation to the Tx Future
356             // callback to be executed after the Tx is created.
357             txFutureCallback.addTxOperationOnComplete(new TransactionOperation() {
358                 @Override
359                 public void invoke(TransactionContext transactionContext) {
360                     transactionContext.writeData(path, data);
361                 }
362             });
363         }
364     }
365
366     @Override
367     public void merge(final YangInstanceIdentifier path, final NormalizedNode<?, ?> data) {
368
369         checkModificationState();
370
371         LOG.debug("Tx {} merge {}", identifier, path);
372
373         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
374         TransactionContext transactionContext = txFutureCallback.getTransactionContext();
375         if(transactionContext != null) {
376             transactionContext.mergeData(path, data);
377         } else {
378             // The shard Tx hasn't been created yet so add the Tx operation to the Tx Future
379             // callback to be executed after the Tx is created.
380             txFutureCallback.addTxOperationOnComplete(new TransactionOperation() {
381                 @Override
382                 public void invoke(TransactionContext transactionContext) {
383                     transactionContext.mergeData(path, data);
384                 }
385             });
386         }
387     }
388
389     @Override
390     public void delete(final YangInstanceIdentifier path) {
391
392         checkModificationState();
393
394         LOG.debug("Tx {} delete {}", identifier, path);
395
396         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
397         TransactionContext transactionContext = txFutureCallback.getTransactionContext();
398         if(transactionContext != null) {
399             transactionContext.deleteData(path);
400         } else {
401             // The shard Tx hasn't been created yet so add the Tx operation to the Tx Future
402             // callback to be executed after the Tx is created.
403             txFutureCallback.addTxOperationOnComplete(new TransactionOperation() {
404                 @Override
405                 public void invoke(TransactionContext transactionContext) {
406                     transactionContext.deleteData(path);
407                 }
408             });
409         }
410     }
411
412     @Override
413     public DOMStoreThreePhaseCommitCohort ready() {
414
415         checkModificationState();
416
417         inReadyState = true;
418
419         LOG.debug("Tx {} Readying {} transactions for commit", identifier,
420                     txFutureCallbackMap.size());
421
422         List<Future<ActorSelection>> cohortFutures = Lists.newArrayList();
423
424         for(TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
425
426             LOG.debug("Tx {} Readying transaction for shard {} chain {}", identifier,
427                         txFutureCallback.getShardName(), transactionChainId);
428
429             TransactionContext transactionContext = txFutureCallback.getTransactionContext();
430             if(transactionContext != null) {
431                 cohortFutures.add(transactionContext.readyTransaction());
432             } else {
433                 // The shard Tx hasn't been created yet so create a promise to ready the Tx later
434                 // after it's created.
435                 final Promise<ActorSelection> cohortPromise = akka.dispatch.Futures.promise();
436                 txFutureCallback.addTxOperationOnComplete(new TransactionOperation() {
437                     @Override
438                     public void invoke(TransactionContext transactionContext) {
439                         cohortPromise.completeWith(transactionContext.readyTransaction());
440                     }
441                 });
442
443                 cohortFutures.add(cohortPromise.future());
444             }
445         }
446
447         onTransactionReady(cohortFutures);
448
449         return new ThreePhaseCommitCohortProxy(actorContext, cohortFutures,
450                 identifier.toString());
451     }
452
453     /**
454      * Method for derived classes to be notified when the transaction has been readied.
455      *
456      * @param cohortFutures the cohort Futures for each shard transaction.
457      */
458     protected void onTransactionReady(List<Future<ActorSelection>> cohortFutures) {
459     }
460
461     /**
462      * Method called to send a CreateTransaction message to a shard.
463      *
464      * @param shard the shard actor to send to
465      * @param serializedCreateMessage the serialized message to send
466      * @return the response Future
467      */
468     protected Future<Object> sendCreateTransaction(ActorSelection shard,
469             Object serializedCreateMessage) {
470         return actorContext.executeOperationAsync(shard, serializedCreateMessage);
471     }
472
473     @Override
474     public Object getIdentifier() {
475         return this.identifier;
476     }
477
478     @Override
479     public void close() {
480         for(TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
481             TransactionContext transactionContext = txFutureCallback.getTransactionContext();
482             if(transactionContext != null) {
483                 transactionContext.closeTransaction();
484             } else {
485                 txFutureCallback.addTxOperationOnComplete(new TransactionOperation() {
486                     @Override
487                     public void invoke(TransactionContext transactionContext) {
488                         transactionContext.closeTransaction();
489                     }
490                 });
491             }
492         }
493
494         txFutureCallbackMap.clear();
495
496         if(transactionType == TransactionType.READ_ONLY) {
497             remoteTransactionActors.clear();
498             remoteTransactionActorsMB.set(true);
499         }
500     }
501
502     private String shardNameFromIdentifier(YangInstanceIdentifier path){
503         return ShardStrategyFactory.getStrategy(path).findShard(path);
504     }
505
506     private TransactionFutureCallback getOrCreateTxFutureCallback(YangInstanceIdentifier path) {
507         String shardName = shardNameFromIdentifier(path);
508         TransactionFutureCallback txFutureCallback = txFutureCallbackMap.get(shardName);
509         if(txFutureCallback == null) {
510             Future<ActorSelection> findPrimaryFuture = actorContext.findPrimaryShardAsync(shardName);
511
512             final TransactionFutureCallback newTxFutureCallback =
513                     new TransactionFutureCallback(shardName);
514
515             txFutureCallback = newTxFutureCallback;
516             txFutureCallbackMap.put(shardName, txFutureCallback);
517
518             findPrimaryFuture.onComplete(new OnComplete<ActorSelection>() {
519                 @Override
520                 public void onComplete(Throwable failure, ActorSelection primaryShard) {
521                     if(failure != null) {
522                         newTxFutureCallback.onComplete(failure, null);
523                     } else {
524                         newTxFutureCallback.setPrimaryShard(primaryShard);
525                     }
526                 }
527             }, actorContext.getActorSystem().dispatcher());
528         }
529
530         return txFutureCallback;
531     }
532
533     public String getTransactionChainId() {
534         return transactionChainId;
535     }
536
537     protected ActorContext getActorContext() {
538         return actorContext;
539     }
540
541     /**
542      * Interface for a transaction operation to be invoked later.
543      */
544     private static interface TransactionOperation {
545         void invoke(TransactionContext transactionContext);
546     }
547
548     /**
549      * Implements a Future OnComplete callback for a CreateTransaction message. This class handles
550      * retries, up to a limit, if the shard doesn't have a leader yet. This is done by scheduling a
551      * retry task after a short delay.
552      * <p>
553      * The end result from a completed CreateTransaction message is a TransactionContext that is
554      * used to perform transaction operations. Transaction operations that occur before the
555      * CreateTransaction completes are cache and executed once the CreateTransaction completes,
556      * successfully or not.
557      */
558     private class TransactionFutureCallback extends OnComplete<Object> {
559
560         /**
561          * The list of transaction operations to execute once the CreateTransaction completes.
562          */
563         @GuardedBy("txOperationsOnComplete")
564         private final List<TransactionOperation> txOperationsOnComplete = Lists.newArrayList();
565
566         /**
567          * The TransactionContext resulting from the CreateTransaction reply.
568          */
569         private volatile TransactionContext transactionContext;
570
571         /**
572          * The target primary shard.
573          */
574         private volatile ActorSelection primaryShard;
575
576         private volatile int createTxTries = (int) (actorContext.getDatastoreContext().
577                 getShardLeaderElectionTimeout().duration().toMillis() /
578                 CREATE_TX_TRY_INTERVAL.toMillis());
579
580         private final String shardName;
581
582         TransactionFutureCallback(String shardName) {
583             this.shardName = shardName;
584         }
585
586         String getShardName() {
587             return shardName;
588         }
589
590         TransactionContext getTransactionContext() {
591             return transactionContext;
592         }
593
594
595         /**
596          * Sets the target primary shard and initiates a CreateTransaction try.
597          */
598         void setPrimaryShard(ActorSelection primaryShard) {
599             LOG.debug("Tx {} Primary shard found - trying create transaction", identifier);
600
601             this.primaryShard = primaryShard;
602             tryCreateTransaction();
603         }
604
605         /**
606          * Adds a TransactionOperation to be executed after the CreateTransaction completes.
607          */
608         void addTxOperationOnComplete(TransactionOperation operation) {
609             synchronized(txOperationsOnComplete) {
610                 if(transactionContext == null) {
611                     LOG.debug("Tx {} Adding operation on complete {}", identifier);
612
613                     txOperationsOnComplete.add(operation);
614                 } else {
615                     operation.invoke(transactionContext);
616                 }
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.isLocalPath(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 int remoteTransactionVersion;
763
764         private TransactionContextImpl(String transactionPath, ActorSelection actor, TransactionIdentifier identifier,
765                 ActorContext actorContext, SchemaContext schemaContext,
766                 boolean isTxActorLocal, int 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         @Override
781         public void closeTransaction() {
782             LOG.debug("Tx {} closeTransaction called", identifier);
783
784             actorContext.sendOperationAsync(getActor(), new CloseTransaction().toSerializable());
785         }
786
787         @Override
788         public Future<ActorSelection> readyTransaction() {
789             LOG.debug("Tx {} readyTransaction called with {} previous recorded operations pending",
790                     identifier, recordedOperationFutures.size());
791
792             // Send the ReadyTransaction message to the Tx actor.
793
794             ReadyTransaction readyTransaction = new ReadyTransaction();
795             final Future<Object> replyFuture = actorContext.executeOperationAsync(getActor(),
796                 isTxActorLocal ? readyTransaction : readyTransaction.toSerializable());
797
798             // Combine all the previously recorded put/merge/delete operation reply Futures and the
799             // ReadyTransactionReply Future into one Future. If any one fails then the combined
800             // Future will fail. We need all prior operations and the ready operation to succeed
801             // in order to attempt commit.
802
803             List<Future<Object>> futureList =
804                     Lists.newArrayListWithCapacity(recordedOperationFutures.size() + 1);
805             futureList.addAll(recordedOperationFutures);
806             futureList.add(replyFuture);
807
808             Future<Iterable<Object>> combinedFutures = akka.dispatch.Futures.sequence(futureList,
809                     actorContext.getActorSystem().dispatcher());
810
811             // Transform the combined Future into a Future that returns the cohort actor path from
812             // the ReadyTransactionReply. That's the end result of the ready operation.
813
814             return combinedFutures.transform(new Mapper<Iterable<Object>, ActorSelection>() {
815                 @Override
816                 public ActorSelection checkedApply(Iterable<Object> notUsed) {
817                     LOG.debug("Tx {} readyTransaction: pending recorded operations succeeded",
818                             identifier);
819
820                     // At this point all the Futures succeeded and we need to extract the cohort
821                     // actor path from the ReadyTransactionReply. For the recorded operations, they
822                     // don't return any data so we're only interested that they completed
823                     // successfully. We could be paranoid and verify the correct reply types but
824                     // that really should never happen so it's not worth the overhead of
825                     // de-serializing each reply.
826
827                     // Note the Future get call here won't block as it's complete.
828                     Object serializedReadyReply = replyFuture.value().get().get();
829                     if (serializedReadyReply instanceof ReadyTransactionReply) {
830                         return actorContext.actorSelection(((ReadyTransactionReply)serializedReadyReply).getCohortPath());
831
832                     } else if(serializedReadyReply.getClass().equals(ReadyTransactionReply.SERIALIZABLE_CLASS)) {
833                         ReadyTransactionReply reply = ReadyTransactionReply.fromSerializable(serializedReadyReply);
834                         String cohortPath = reply.getCohortPath();
835
836                         // In Helium we used to return the local path of the actor which represented
837                         // a remote ThreePhaseCommitCohort. The local path would then be converted to
838                         // a remote path using this resolvePath method. To maintain compatibility with
839                         // a Helium node we need to continue to do this conversion.
840                         // At some point in the future when upgrades from Helium are not supported
841                         // we could remove this code to resolvePath and just use the cohortPath as the
842                         // resolved cohortPath
843                         if(TransactionContextImpl.this.remoteTransactionVersion < CreateTransaction.HELIUM_1_VERSION) {
844                             cohortPath = actorContext.resolvePath(transactionPath, cohortPath);
845                         }
846
847                         return actorContext.actorSelection(cohortPath);
848
849                     } else {
850                         // Throwing an exception here will fail the Future.
851                         throw new IllegalArgumentException(String.format("Invalid reply type {}",
852                                 serializedReadyReply.getClass()));
853                     }
854                 }
855             }, SAME_FAILURE_TRANSFORMER, actorContext.getActorSystem().dispatcher());
856         }
857
858         @Override
859         public void deleteData(YangInstanceIdentifier path) {
860             LOG.debug("Tx {} deleteData called path = {}", identifier, path);
861
862             DeleteData deleteData = new DeleteData(path);
863             recordedOperationFutures.add(actorContext.executeOperationAsync(getActor(),
864                 isTxActorLocal ? deleteData : deleteData.toSerializable()));
865         }
866
867         @Override
868         public void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
869             LOG.debug("Tx {} mergeData called path = {}", identifier, path);
870
871             MergeData mergeData = new MergeData(path, data, schemaContext);
872             recordedOperationFutures.add(actorContext.executeOperationAsync(getActor(),
873                 isTxActorLocal ? mergeData : mergeData.toSerializable()));
874         }
875
876         @Override
877         public void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
878             LOG.debug("Tx {} writeData called path = {}", identifier, path);
879
880             WriteData writeData = new WriteData(path, data, schemaContext);
881             recordedOperationFutures.add(actorContext.executeOperationAsync(getActor(),
882                 isTxActorLocal ? writeData : writeData.toSerializable()));
883         }
884
885         @Override
886         public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readData(
887                 final YangInstanceIdentifier path) {
888
889             LOG.debug("Tx {} readData called path = {}", identifier, path);
890
891             final SettableFuture<Optional<NormalizedNode<?, ?>>> returnFuture = SettableFuture.create();
892
893             // If there were any previous recorded put/merge/delete operation reply Futures then we
894             // must wait for them to successfully complete. This is necessary to honor the read
895             // uncommitted semantics of the public API contract. If any one fails then fail the read.
896
897             if(recordedOperationFutures.isEmpty()) {
898                 finishReadData(path, returnFuture);
899             } else {
900                 LOG.debug("Tx {} readData: verifying {} previous recorded operations",
901                         identifier, recordedOperationFutures.size());
902
903                 // Note: we make a copy of recordedOperationFutures to be on the safe side in case
904                 // Futures#sequence accesses the passed List on a different thread, as
905                 // recordedOperationFutures is not synchronized.
906
907                 Future<Iterable<Object>> combinedFutures = akka.dispatch.Futures.sequence(
908                         Lists.newArrayList(recordedOperationFutures),
909                         actorContext.getActorSystem().dispatcher());
910
911                 OnComplete<Iterable<Object>> onComplete = new OnComplete<Iterable<Object>>() {
912                     @Override
913                     public void onComplete(Throwable failure, Iterable<Object> notUsed)
914                             throws Throwable {
915                         if(failure != null) {
916                             LOG.debug("Tx {} readData: a recorded operation failed: {}",
917                                     identifier, failure);
918                             returnFuture.setException(new ReadFailedException(
919                                     "The read could not be performed because a previous put, merge,"
920                                     + "or delete operation failed", failure));
921                         } else {
922                             finishReadData(path, returnFuture);
923                         }
924                     }
925                 };
926
927                 combinedFutures.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
928             }
929
930             return MappingCheckedFuture.create(returnFuture, ReadFailedException.MAPPER);
931         }
932
933         private void finishReadData(final YangInstanceIdentifier path,
934                 final SettableFuture<Optional<NormalizedNode<?, ?>>> returnFuture) {
935
936             LOG.debug("Tx {} finishReadData called path = {}", identifier, path);
937
938             OnComplete<Object> onComplete = new OnComplete<Object>() {
939                 @Override
940                 public void onComplete(Throwable failure, Object readResponse) throws Throwable {
941                     if(failure != null) {
942                         LOG.debug("Tx {} read operation failed: {}", identifier, failure);
943                         returnFuture.setException(new ReadFailedException(
944                                 "Error reading data for path " + path, failure));
945
946                     } else {
947                         LOG.debug("Tx {} read operation succeeded", identifier, failure);
948
949                         if (readResponse instanceof ReadDataReply) {
950                             ReadDataReply reply = (ReadDataReply) readResponse;
951                             returnFuture.set(Optional.<NormalizedNode<?, ?>>fromNullable(reply.getNormalizedNode()));
952
953                         } else if (readResponse.getClass().equals(ReadDataReply.SERIALIZABLE_CLASS)) {
954                             ReadDataReply reply = ReadDataReply.fromSerializable(schemaContext, path, readResponse);
955                             returnFuture.set(Optional.<NormalizedNode<?, ?>>fromNullable(reply.getNormalizedNode()));
956
957                         } else {
958                             returnFuture.setException(new ReadFailedException(
959                                 "Invalid response reading data for path " + path));
960                         }
961                     }
962                 }
963             };
964
965             ReadData readData = new ReadData(path);
966             Future<Object> readFuture = actorContext.executeOperationAsync(getActor(),
967                 isTxActorLocal ? readData : readData.toSerializable());
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             DataExists dataExists = new DataExists(path);
1050             Future<Object> future = actorContext.executeOperationAsync(getActor(),
1051                 isTxActorLocal ? dataExists : dataExists.toSerializable());
1052
1053             future.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
1054         }
1055     }
1056
1057     private static class NoOpTransactionContext extends AbstractTransactionContext {
1058
1059         private final Logger LOG = LoggerFactory.getLogger(NoOpTransactionContext.class);
1060
1061         private final Throwable failure;
1062
1063         public NoOpTransactionContext(Throwable failure, TransactionIdentifier identifier){
1064             super(identifier);
1065             this.failure = failure;
1066         }
1067
1068         @Override
1069         public void closeTransaction() {
1070             LOG.debug("NoOpTransactionContext {} closeTransaction called", identifier);
1071         }
1072
1073         @Override
1074         public Future<ActorSelection> readyTransaction() {
1075             LOG.debug("Tx {} readyTransaction called", identifier);
1076             return akka.dispatch.Futures.failed(failure);
1077         }
1078
1079         @Override
1080         public void deleteData(YangInstanceIdentifier path) {
1081             LOG.debug("Tx {} deleteData called path = {}", identifier, path);
1082         }
1083
1084         @Override
1085         public void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
1086             LOG.debug("Tx {} mergeData called path = {}", identifier, path);
1087         }
1088
1089         @Override
1090         public void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
1091             LOG.debug("Tx {} writeData called path = {}", identifier, path);
1092         }
1093
1094         @Override
1095         public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readData(
1096                 YangInstanceIdentifier path) {
1097             LOG.debug("Tx {} readData called path = {}", identifier, path);
1098             return Futures.immediateFailedCheckedFuture(new ReadFailedException(
1099                     "Error reading data for path " + path, failure));
1100         }
1101
1102         @Override
1103         public CheckedFuture<Boolean, ReadFailedException> dataExists(
1104                 YangInstanceIdentifier path) {
1105             LOG.debug("Tx {} dataExists called path = {}", identifier, path);
1106             return Futures.immediateFailedCheckedFuture(new ReadFailedException(
1107                     "Error checking exists for path " + path, failure));
1108         }
1109     }
1110 }