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