f28a1e5f73d4823fe31ba20d14fa4d986f9700da
[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.shardstrategy.ShardStrategyFactory;
39 import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
40 import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
41 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadWriteTransaction;
42 import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
43 import org.opendaylight.yangtools.util.concurrent.MappingCheckedFuture;
44 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
45 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
46 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
47 import org.slf4j.Logger;
48 import org.slf4j.LoggerFactory;
49 import scala.concurrent.Future;
50 import scala.concurrent.Promise;
51 import scala.concurrent.duration.FiniteDuration;
52
53 /**
54  * TransactionProxy acts as a proxy for one or more transactions that were created on a remote shard
55  * <p>
56  * Creating a transaction on the consumer side will create one instance of a transaction proxy. If during
57  * the transaction reads and writes are done on data that belongs to different shards then a separate transaction will
58  * be created on each of those shards by the TransactionProxy
59  *</p>
60  * <p>
61  * The TransactionProxy does not make any guarantees about atomicity or order in which the transactions on the various
62  * shards will be executed.
63  * </p>
64  */
65 public class TransactionProxy implements DOMStoreReadWriteTransaction {
66
67     public static enum TransactionType {
68         READ_ONLY,
69         WRITE_ONLY,
70         READ_WRITE
71     }
72
73     static final Mapper<Throwable, Throwable> SAME_FAILURE_TRANSFORMER =
74                                                               new Mapper<Throwable, Throwable>() {
75         @Override
76         public Throwable apply(Throwable failure) {
77             return failure;
78         }
79     };
80
81     private static final AtomicLong counter = new AtomicLong();
82
83     private static final Logger LOG = LoggerFactory.getLogger(TransactionProxy.class);
84
85     /**
86      * Time interval in between transaction create retries.
87      */
88     private static final FiniteDuration CREATE_TX_TRY_INTERVAL =
89             FiniteDuration.create(1, TimeUnit.SECONDS);
90
91     /**
92      * Used to enqueue the PhantomReferences for read-only TransactionProxy instances. The
93      * FinalizableReferenceQueue is safe to use statically in an OSGi environment as it uses some
94      * trickery to clean up its internal thread when the bundle is unloaded.
95      */
96     private static final FinalizableReferenceQueue phantomReferenceQueue =
97                                                                   new FinalizableReferenceQueue();
98
99     /**
100      * This stores the TransactionProxyCleanupPhantomReference instances statically, This is
101      * necessary because PhantomReferences need a hard reference so they're not garbage collected.
102      * Once finalized, the TransactionProxyCleanupPhantomReference removes itself from this map
103      * and thus becomes eligible for garbage collection.
104      */
105     private static final Map<TransactionProxyCleanupPhantomReference,
106                              TransactionProxyCleanupPhantomReference> phantomReferenceCache =
107                                                                         new ConcurrentHashMap<>();
108
109     /**
110      * A PhantomReference that closes remote transactions for a TransactionProxy when it's
111      * garbage collected. This is used for read-only transactions as they're not explicitly closed
112      * by clients. So the only way to detect that a transaction is no longer in use and it's safe
113      * to clean up is when it's garbage collected. It's inexact as to when an instance will be GC'ed
114      * but TransactionProxy instances should generally be short-lived enough to avoid being moved
115      * to the old generation space and thus should be cleaned up in a timely manner as the GC
116      * runs on the young generation (eden, swap1...) space much more frequently.
117      */
118     private static class TransactionProxyCleanupPhantomReference
119                                            extends FinalizablePhantomReference<TransactionProxy> {
120
121         private final List<ActorSelection> remoteTransactionActors;
122         private final AtomicBoolean remoteTransactionActorsMB;
123         private final ActorContext actorContext;
124         private final TransactionIdentifier identifier;
125
126         protected TransactionProxyCleanupPhantomReference(TransactionProxy referent) {
127             super(referent, phantomReferenceQueue);
128
129             // Note we need to cache the relevant fields from the TransactionProxy as we can't
130             // have a hard reference to the TransactionProxy instance itself.
131
132             remoteTransactionActors = referent.remoteTransactionActors;
133             remoteTransactionActorsMB = referent.remoteTransactionActorsMB;
134             actorContext = referent.actorContext;
135             identifier = referent.identifier;
136         }
137
138         @Override
139         public void finalizeReferent() {
140             LOG.trace("Cleaning up {} Tx actors for TransactionProxy {}",
141                     remoteTransactionActors.size(), identifier);
142
143             phantomReferenceCache.remove(this);
144
145             // Access the memory barrier volatile to ensure all previous updates to the
146             // remoteTransactionActors list are visible to this thread.
147
148             if(remoteTransactionActorsMB.get()) {
149                 for(ActorSelection actor : remoteTransactionActors) {
150                     LOG.trace("Sending CloseTransaction to {}", actor);
151                     actorContext.sendOperationAsync(actor, CloseTransaction.INSTANCE.toSerializable());
152                 }
153             }
154         }
155     }
156
157     /**
158      * Stores the remote Tx actors for each requested data store path to be used by the
159      * PhantomReference to close the remote Tx's. This is only used for read-only Tx's. The
160      * remoteTransactionActorsMB volatile serves as a memory barrier to publish updates to the
161      * remoteTransactionActors list so they will be visible to the thread accessing the
162      * PhantomReference.
163      */
164     private List<ActorSelection> remoteTransactionActors;
165     private AtomicBoolean remoteTransactionActorsMB;
166
167     /**
168      * Stores the create transaction results per shard.
169      */
170     private final Map<String, TransactionFutureCallback> txFutureCallbackMap = new HashMap<>();
171
172     private final TransactionType transactionType;
173     private final ActorContext actorContext;
174     private final TransactionIdentifier identifier;
175     private final String transactionChainId;
176     private final SchemaContext schemaContext;
177     private boolean inReadyState;
178     private final Semaphore operationLimiter;
179     private final OperationCompleter operationCompleter;
180
181     public TransactionProxy(ActorContext actorContext, TransactionType transactionType) {
182         this(actorContext, transactionType, "");
183     }
184
185     public TransactionProxy(ActorContext actorContext, TransactionType transactionType,
186             String transactionChainId) {
187         this.actorContext = Preconditions.checkNotNull(actorContext,
188             "actorContext should not be null");
189         this.transactionType = Preconditions.checkNotNull(transactionType,
190             "transactionType should not be null");
191         this.schemaContext = Preconditions.checkNotNull(actorContext.getSchemaContext(),
192             "schemaContext should not be null");
193         this.transactionChainId = transactionChainId;
194
195         String memberName = actorContext.getCurrentMemberName();
196         if(memberName == null){
197             memberName = "UNKNOWN-MEMBER";
198         }
199
200         this.identifier = TransactionIdentifier.builder().memberName(memberName).counter(
201             counter.getAndIncrement()).build();
202
203         if(transactionType == TransactionType.READ_ONLY) {
204             // Read-only Tx's aren't explicitly closed by the client so we create a PhantomReference
205             // to close the remote Tx's when this instance is no longer in use and is garbage
206             // collected.
207
208             remoteTransactionActors = Lists.newArrayList();
209             remoteTransactionActorsMB = new AtomicBoolean();
210
211             TransactionProxyCleanupPhantomReference cleanup =
212                 new TransactionProxyCleanupPhantomReference(this);
213             phantomReferenceCache.put(cleanup, cleanup);
214         }
215
216         // Note : Currently mailbox-capacity comes from akka.conf and not from the config-subsystem
217         this.operationLimiter = new Semaphore(actorContext.getTransactionOutstandingOperationLimit());
218         this.operationCompleter = new OperationCompleter(operationLimiter);
219
220         LOG.debug("Created txn {} of type {} on chain {}", identifier, transactionType, transactionChainId);
221     }
222
223     @VisibleForTesting
224     List<Future<Object>> getRecordedOperationFutures() {
225         List<Future<Object>> recordedOperationFutures = Lists.newArrayList();
226         for(TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
227             TransactionContext transactionContext = txFutureCallback.getTransactionContext();
228             if(transactionContext != null) {
229                 recordedOperationFutures.addAll(transactionContext.getRecordedOperationFutures());
230             }
231         }
232
233         return recordedOperationFutures;
234     }
235
236     @VisibleForTesting
237     boolean hasTransactionContext() {
238         for(TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
239             TransactionContext transactionContext = txFutureCallback.getTransactionContext();
240             if(transactionContext != null) {
241                 return true;
242             }
243         }
244
245         return false;
246     }
247
248     @Override
249     public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> read(final YangInstanceIdentifier path) {
250
251         Preconditions.checkState(transactionType != TransactionType.WRITE_ONLY,
252                 "Read operation on write-only transaction is not allowed");
253
254         LOG.debug("Tx {} read {}", identifier, path);
255
256         throttleOperation();
257
258         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
259         return txFutureCallback.enqueueReadOperation(new ReadOperation<Optional<NormalizedNode<?, ?>>>() {
260             @Override
261             public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> invoke(
262                     TransactionContext transactionContext) {
263                 return transactionContext.readData(path);
264             }
265         });
266     }
267
268     @Override
269     public CheckedFuture<Boolean, ReadFailedException> exists(final YangInstanceIdentifier path) {
270
271         Preconditions.checkState(transactionType != TransactionType.WRITE_ONLY,
272                 "Exists operation on write-only transaction is not allowed");
273
274         LOG.debug("Tx {} exists {}", identifier, path);
275
276         throttleOperation();
277
278         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
279         return txFutureCallback.enqueueReadOperation(new ReadOperation<Boolean>() {
280             @Override
281             public CheckedFuture<Boolean, ReadFailedException> invoke(TransactionContext transactionContext) {
282                 return transactionContext.dataExists(path);
283             }
284         });
285     }
286
287
288     private void checkModificationState() {
289         Preconditions.checkState(transactionType != TransactionType.READ_ONLY,
290                 "Modification operation on read-only transaction is not allowed");
291         Preconditions.checkState(!inReadyState,
292                 "Transaction is sealed - further modifications are not allowed");
293     }
294
295     private void throttleOperation() {
296         throttleOperation(1);
297     }
298
299     private void throttleOperation(int acquirePermits) {
300         try {
301             if(!operationLimiter.tryAcquire(acquirePermits, actorContext.getDatastoreContext().getOperationTimeoutInSeconds(), TimeUnit.SECONDS)){
302                 LOG.warn("Failed to acquire operation permit for transaction {}", getIdentifier());
303             }
304         } catch (InterruptedException e) {
305             if(LOG.isDebugEnabled()) {
306                 LOG.debug("Interrupted when trying to acquire operation permit for transaction " + getIdentifier().toString(), e);
307             } else {
308                 LOG.warn("Interrupted when trying to acquire operation permit for transaction {}", getIdentifier());
309             }
310         }
311     }
312
313
314     @Override
315     public void write(final YangInstanceIdentifier path, final NormalizedNode<?, ?> data) {
316
317         checkModificationState();
318
319         LOG.debug("Tx {} write {}", identifier, path);
320
321         throttleOperation();
322
323         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
324         txFutureCallback.enqueueModifyOperation(new TransactionOperation() {
325             @Override
326             public void invoke(TransactionContext transactionContext) {
327                 transactionContext.writeData(path, data);
328             }
329         });
330     }
331
332     @Override
333     public void merge(final YangInstanceIdentifier path, final NormalizedNode<?, ?> data) {
334
335         checkModificationState();
336
337         LOG.debug("Tx {} merge {}", identifier, path);
338
339         throttleOperation();
340
341         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
342         txFutureCallback.enqueueModifyOperation(new TransactionOperation() {
343             @Override
344             public void invoke(TransactionContext transactionContext) {
345                 transactionContext.mergeData(path, data);
346             }
347         });
348     }
349
350     @Override
351     public void delete(final YangInstanceIdentifier path) {
352
353         checkModificationState();
354
355         LOG.debug("Tx {} delete {}", identifier, path);
356
357         throttleOperation();
358
359         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
360         txFutureCallback.enqueueModifyOperation(new TransactionOperation() {
361             @Override
362             public void invoke(TransactionContext transactionContext) {
363                 transactionContext.deleteData(path);
364             }
365         });
366     }
367
368     @Override
369     public DOMStoreThreePhaseCommitCohort ready() {
370
371         checkModificationState();
372
373         throttleOperation(txFutureCallbackMap.size());
374
375         inReadyState = true;
376
377         LOG.debug("Tx {} Readying {} transactions for commit", identifier,
378                     txFutureCallbackMap.size());
379
380         List<Future<ActorSelection>> cohortFutures = Lists.newArrayList();
381
382         for(TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
383
384             LOG.debug("Tx {} Readying transaction for shard {} chain {}", identifier,
385                         txFutureCallback.getShardName(), transactionChainId);
386
387             Future<ActorSelection> future = txFutureCallback.enqueueFutureOperation(new FutureOperation<ActorSelection>() {
388                 @Override
389                 public Future<ActorSelection> invoke(TransactionContext transactionContext) {
390                     return transactionContext.readyTransaction();
391                 }
392             });
393
394             cohortFutures.add(future);
395         }
396
397         onTransactionReady(cohortFutures);
398
399         return new ThreePhaseCommitCohortProxy(actorContext, cohortFutures,
400                 identifier.toString());
401     }
402
403     /**
404      * Method for derived classes to be notified when the transaction has been readied.
405      *
406      * @param cohortFutures the cohort Futures for each shard transaction.
407      */
408     protected void onTransactionReady(List<Future<ActorSelection>> cohortFutures) {
409     }
410
411     /**
412      * Method called to send a CreateTransaction message to a shard.
413      *
414      * @param shard the shard actor to send to
415      * @param serializedCreateMessage the serialized message to send
416      * @return the response Future
417      */
418     protected Future<Object> sendCreateTransaction(ActorSelection shard,
419             Object serializedCreateMessage) {
420         return actorContext.executeOperationAsync(shard, serializedCreateMessage);
421     }
422
423     @Override
424     public Object getIdentifier() {
425         return this.identifier;
426     }
427
428     @Override
429     public void close() {
430         for (TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
431             txFutureCallback.enqueueModifyOperation(new TransactionOperation() {
432                 @Override
433                 public void invoke(TransactionContext transactionContext) {
434                     transactionContext.closeTransaction();
435                 }
436             });
437         }
438
439         txFutureCallbackMap.clear();
440
441         if(transactionType == TransactionType.READ_ONLY) {
442             remoteTransactionActors.clear();
443             remoteTransactionActorsMB.set(true);
444         }
445     }
446
447     private String shardNameFromIdentifier(YangInstanceIdentifier path){
448         return ShardStrategyFactory.getStrategy(path).findShard(path);
449     }
450
451     private TransactionFutureCallback getOrCreateTxFutureCallback(YangInstanceIdentifier path) {
452         String shardName = shardNameFromIdentifier(path);
453         TransactionFutureCallback txFutureCallback = txFutureCallbackMap.get(shardName);
454         if(txFutureCallback == null) {
455             Future<ActorSelection> findPrimaryFuture = actorContext.findPrimaryShardAsync(shardName);
456
457             final TransactionFutureCallback newTxFutureCallback =
458                     new TransactionFutureCallback(shardName);
459
460             txFutureCallback = newTxFutureCallback;
461             txFutureCallbackMap.put(shardName, txFutureCallback);
462
463             findPrimaryFuture.onComplete(new OnComplete<ActorSelection>() {
464                 @Override
465                 public void onComplete(Throwable failure, ActorSelection primaryShard) {
466                     if(failure != null) {
467                         newTxFutureCallback.onComplete(failure, null);
468                     } else {
469                         newTxFutureCallback.setPrimaryShard(primaryShard);
470                     }
471                 }
472             }, actorContext.getActorSystem().dispatcher());
473         }
474
475         return txFutureCallback;
476     }
477
478     public String getTransactionChainId() {
479         return transactionChainId;
480     }
481
482     protected ActorContext getActorContext() {
483         return actorContext;
484     }
485
486     /**
487      * Interfaces for transaction operations to be invoked later.
488      */
489     private static interface TransactionOperation {
490         void invoke(TransactionContext transactionContext);
491     }
492
493     /**
494      * This interface returns a Guava Future
495      */
496     private static interface ReadOperation<T> {
497         CheckedFuture<T, ReadFailedException> invoke(TransactionContext transactionContext);
498     }
499
500     /**
501      * This interface returns a Scala Future
502      */
503     private static interface FutureOperation<T> {
504         Future<T> invoke(TransactionContext transactionContext);
505     }
506
507     /**
508      * Implements a Future OnComplete callback for a CreateTransaction message. This class handles
509      * retries, up to a limit, if the shard doesn't have a leader yet. This is done by scheduling a
510      * retry task after a short delay.
511      * <p>
512      * The end result from a completed CreateTransaction message is a TransactionContext that is
513      * used to perform transaction operations. Transaction operations that occur before the
514      * CreateTransaction completes are cache and executed once the CreateTransaction completes,
515      * successfully or not.
516      */
517     private class TransactionFutureCallback extends OnComplete<Object> {
518
519         /**
520          * The list of transaction operations to execute once the CreateTransaction completes.
521          */
522         @GuardedBy("txOperationsOnComplete")
523         private final List<TransactionOperation> txOperationsOnComplete = Lists.newArrayList();
524
525         /**
526          * The TransactionContext resulting from the CreateTransaction reply.
527          */
528         private volatile TransactionContext transactionContext;
529
530         /**
531          * The target primary shard.
532          */
533         private volatile ActorSelection primaryShard;
534
535         private volatile int createTxTries = (int) (actorContext.getDatastoreContext().
536                 getShardLeaderElectionTimeout().duration().toMillis() /
537                 CREATE_TX_TRY_INTERVAL.toMillis());
538
539         private final String shardName;
540
541         TransactionFutureCallback(String shardName) {
542             this.shardName = shardName;
543         }
544
545         String getShardName() {
546             return shardName;
547         }
548
549         TransactionContext getTransactionContext() {
550             return transactionContext;
551         }
552
553
554         /**
555          * Sets the target primary shard and initiates a CreateTransaction try.
556          */
557         void setPrimaryShard(ActorSelection primaryShard) {
558             LOG.debug("Tx {} Primary shard found - trying create transaction", identifier);
559
560             this.primaryShard = primaryShard;
561             tryCreateTransaction();
562         }
563
564         /**
565          * Adds a TransactionOperation to be executed after the CreateTransaction completes.
566          */
567         void addTxOperationOnComplete(TransactionOperation operation) {
568             synchronized(txOperationsOnComplete) {
569                 if(transactionContext == null) {
570                     LOG.debug("Tx {} Adding operation on complete {}", identifier);
571
572                     txOperationsOnComplete.add(operation);
573                 } else {
574                     operation.invoke(transactionContext);
575                 }
576             }
577         }
578
579
580         <T> Future<T> enqueueFutureOperation(final FutureOperation<T> op) {
581
582             Future<T> future;
583
584             if (transactionContext != null) {
585                 future = op.invoke(transactionContext);
586             } else {
587                 // The shard Tx hasn't been created yet so add the Tx operation to the Tx Future
588                 // callback to be executed after the Tx is created.
589                 final Promise<T> promise = akka.dispatch.Futures.promise();
590                 addTxOperationOnComplete(new TransactionOperation() {
591                     @Override
592                     public void invoke(TransactionContext transactionContext) {
593                         promise.completeWith(op.invoke(transactionContext));
594                     }
595                 });
596
597                 future = promise.future();
598             }
599
600             return future;
601         }
602
603         <T> CheckedFuture<T, ReadFailedException> enqueueReadOperation(final ReadOperation<T> op) {
604
605             CheckedFuture<T, ReadFailedException> future;
606
607             if (transactionContext != null) {
608                 future = op.invoke(transactionContext);
609             } else {
610                 // The shard Tx hasn't been created yet so add the Tx operation to the Tx Future
611                 // callback to be executed after the Tx is created.
612                 final SettableFuture<T> proxyFuture = SettableFuture.create();
613                 addTxOperationOnComplete(new TransactionOperation() {
614                     @Override
615                     public void invoke(TransactionContext transactionContext) {
616                         Futures.addCallback(op.invoke(transactionContext), new FutureCallback<T>() {
617                             @Override
618                             public void onSuccess(T data) {
619                                 proxyFuture.set(data);
620                             }
621
622                             @Override
623                             public void onFailure(Throwable t) {
624                                 proxyFuture.setException(t);
625                             }
626                         });
627                     }
628                 });
629
630                 future = MappingCheckedFuture.create(proxyFuture, ReadFailedException.MAPPER);
631             }
632
633             return future;
634         }
635
636         void enqueueModifyOperation(final TransactionOperation op) {
637
638             if (transactionContext != null) {
639                 op.invoke(transactionContext);
640             } else {
641                 // The shard Tx hasn't been created yet so add the Tx operation to the Tx Future
642                 // callback to be executed after the Tx is created.
643                 addTxOperationOnComplete(op);
644             }
645         }
646
647         /**
648          * Performs a CreateTransaction try async.
649          */
650         private void tryCreateTransaction() {
651             Future<Object> createTxFuture = sendCreateTransaction(primaryShard,
652                     new CreateTransaction(identifier.toString(),
653                             TransactionProxy.this.transactionType.ordinal(),
654                             getTransactionChainId()).toSerializable());
655
656             createTxFuture.onComplete(this, actorContext.getActorSystem().dispatcher());
657         }
658
659         @Override
660         public void onComplete(Throwable failure, Object response) {
661             if(failure instanceof NoShardLeaderException) {
662                 // There's no leader for the shard yet - schedule and try again, unless we're out
663                 // of retries. Note: createTxTries is volatile as it may be written by different
664                 // threads however not concurrently, therefore decrementing it non-atomically here
665                 // is ok.
666                 if(--createTxTries > 0) {
667                     LOG.debug("Tx {} Shard {} has no leader yet - scheduling create Tx retry",
668                             identifier, shardName);
669
670                     actorContext.getActorSystem().scheduler().scheduleOnce(CREATE_TX_TRY_INTERVAL,
671                             new Runnable() {
672                                 @Override
673                                 public void run() {
674                                     tryCreateTransaction();
675                                 }
676                             }, actorContext.getActorSystem().dispatcher());
677                     return;
678                 }
679             }
680
681             // Create the TransactionContext from the response or failure and execute delayed
682             // TransactionOperations. This entire section is done atomically (ie synchronized) with
683             // respect to #addTxOperationOnComplete to handle timing issues and ensure no
684             // TransactionOperation is missed and that they are processed in the order they occurred.
685             synchronized(txOperationsOnComplete) {
686                 // Store the new TransactionContext locally until we've completed invoking the
687                 // TransactionOperations. This avoids thread timing issues which could cause
688                 // out-of-order TransactionOperations. Eg, on a modification operation, if the
689                 // TransactionContext is non-null, then we directly call the TransactionContext.
690                 // However, at the same time, the code may be executing the cached
691                 // TransactionOperations. So to avoid thus timing, we don't publish the
692                 // TransactionContext until after we've executed all cached TransactionOperations.
693                 TransactionContext localTransactionContext;
694                 if(failure != null) {
695                     LOG.debug("Tx {} Creating NoOpTransaction because of error: {}", identifier,
696                             failure.getMessage());
697
698                     localTransactionContext = new NoOpTransactionContext(failure, identifier, operationLimiter);
699                 } else if (response.getClass().equals(CreateTransactionReply.SERIALIZABLE_CLASS)) {
700                     localTransactionContext = createValidTransactionContext(
701                             CreateTransactionReply.fromSerializable(response));
702                 } else {
703                     IllegalArgumentException exception = new IllegalArgumentException(String.format(
704                         "Invalid reply type %s for CreateTransaction", response.getClass()));
705
706                     localTransactionContext = new NoOpTransactionContext(exception, identifier, operationLimiter);
707                 }
708
709                 for(TransactionOperation oper: txOperationsOnComplete) {
710                     oper.invoke(localTransactionContext);
711                 }
712
713                 txOperationsOnComplete.clear();
714
715                 // We're done invoking the TransactionOperations so we can now publish the
716                 // TransactionContext.
717                 transactionContext = localTransactionContext;
718             }
719         }
720
721         private TransactionContext createValidTransactionContext(CreateTransactionReply reply) {
722             String transactionPath = reply.getTransactionPath();
723
724             LOG.debug("Tx {} Received transaction actor path {}", identifier, transactionPath);
725
726             ActorSelection transactionActor = actorContext.actorSelection(transactionPath);
727
728             if (transactionType == TransactionType.READ_ONLY) {
729                 // Add the actor to the remoteTransactionActors list for access by the
730                 // cleanup PhantonReference.
731                 remoteTransactionActors.add(transactionActor);
732
733                 // Write to the memory barrier volatile to publish the above update to the
734                 // remoteTransactionActors list for thread visibility.
735                 remoteTransactionActorsMB.set(true);
736             }
737
738             // TxActor is always created where the leader of the shard is.
739             // Check if TxActor is created in the same node
740             boolean isTxActorLocal = actorContext.isPathLocal(transactionPath);
741
742             return new TransactionContextImpl(transactionPath, transactionActor, identifier,
743                 actorContext, schemaContext, isTxActorLocal, reply.getVersion(), operationCompleter);
744         }
745     }
746 }