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