Merge "Removing { } from NormalizedNodeJsonBodyWriter"
[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.SettableFuture;
22 import java.util.ArrayList;
23 import java.util.Collection;
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         final SettableFuture<Optional<NormalizedNode<?, ?>>> proxyFuture = SettableFuture.create();
259
260         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
261         txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
262             @Override
263             public void invoke(TransactionContext transactionContext) {
264                 transactionContext.readData(path, proxyFuture);
265             }
266         });
267
268         return MappingCheckedFuture.create(proxyFuture, ReadFailedException.MAPPER);
269     }
270
271     @Override
272     public CheckedFuture<Boolean, ReadFailedException> exists(final YangInstanceIdentifier path) {
273
274         Preconditions.checkState(transactionType != TransactionType.WRITE_ONLY,
275                 "Exists operation on write-only transaction is not allowed");
276
277         LOG.debug("Tx {} exists {}", identifier, path);
278
279         throttleOperation();
280
281         final SettableFuture<Boolean> proxyFuture = SettableFuture.create();
282
283         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
284         txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
285             @Override
286             public void invoke(TransactionContext transactionContext) {
287                 transactionContext.dataExists(path, proxyFuture);
288             }
289         });
290
291         return MappingCheckedFuture.create(proxyFuture, ReadFailedException.MAPPER);
292     }
293
294     private void checkModificationState() {
295         Preconditions.checkState(transactionType != TransactionType.READ_ONLY,
296                 "Modification operation on read-only transaction is not allowed");
297         Preconditions.checkState(!inReadyState,
298                 "Transaction is sealed - further modifications are not allowed");
299     }
300
301     private void throttleOperation() {
302         throttleOperation(1);
303     }
304
305     private void throttleOperation(int acquirePermits) {
306         try {
307             if(!operationLimiter.tryAcquire(acquirePermits,
308                     actorContext.getDatastoreContext().getOperationTimeoutInSeconds(), TimeUnit.SECONDS)){
309                 LOG.warn("Failed to acquire operation permit for transaction {}", getIdentifier());
310             }
311         } catch (InterruptedException e) {
312             if(LOG.isDebugEnabled()) {
313                 LOG.debug("Interrupted when trying to acquire operation permit for transaction " + getIdentifier().toString(), e);
314             } else {
315                 LOG.warn("Interrupted when trying to acquire operation permit for transaction {}", getIdentifier());
316             }
317         }
318     }
319
320
321     @Override
322     public void write(final YangInstanceIdentifier path, final NormalizedNode<?, ?> data) {
323
324         checkModificationState();
325
326         LOG.debug("Tx {} write {}", identifier, path);
327
328         throttleOperation();
329
330         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
331         txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
332             @Override
333             public void invoke(TransactionContext transactionContext) {
334                 transactionContext.writeData(path, data);
335             }
336         });
337     }
338
339     @Override
340     public void merge(final YangInstanceIdentifier path, final NormalizedNode<?, ?> data) {
341
342         checkModificationState();
343
344         LOG.debug("Tx {} merge {}", identifier, path);
345
346         throttleOperation();
347
348         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
349         txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
350             @Override
351             public void invoke(TransactionContext transactionContext) {
352                 transactionContext.mergeData(path, data);
353             }
354         });
355     }
356
357     @Override
358     public void delete(final YangInstanceIdentifier path) {
359
360         checkModificationState();
361
362         LOG.debug("Tx {} delete {}", identifier, path);
363
364         throttleOperation();
365
366         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
367         txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
368             @Override
369             public void invoke(TransactionContext transactionContext) {
370                 transactionContext.deleteData(path);
371             }
372         });
373     }
374
375     @Override
376     public DOMStoreThreePhaseCommitCohort ready() {
377
378         checkModificationState();
379
380         throttleOperation(txFutureCallbackMap.size());
381
382         inReadyState = true;
383
384         LOG.debug("Tx {} Readying {} transactions for commit", identifier,
385                     txFutureCallbackMap.size());
386
387         List<Future<ActorSelection>> cohortFutures = Lists.newArrayList();
388
389         for(TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
390
391             LOG.debug("Tx {} Readying transaction for shard {} chain {}", identifier,
392                         txFutureCallback.getShardName(), transactionChainId);
393
394             final TransactionContext transactionContext = txFutureCallback.getTransactionContext();
395             final Future<ActorSelection> future;
396             if (transactionContext != null) {
397                 // avoid the creation of a promise and a TransactionOperation
398                 future = transactionContext.readyTransaction();
399             } else {
400                 final Promise<ActorSelection> promise = akka.dispatch.Futures.promise();
401                 txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
402                     @Override
403                     public void invoke(TransactionContext transactionContext) {
404                         promise.completeWith(transactionContext.readyTransaction());
405                     }
406                 });
407                 future = promise.future();
408             }
409
410             cohortFutures.add(future);
411         }
412
413         onTransactionReady(cohortFutures);
414
415         return new ThreePhaseCommitCohortProxy(actorContext, cohortFutures,
416                 identifier.toString());
417     }
418
419     /**
420      * Method for derived classes to be notified when the transaction has been readied.
421      *
422      * @param cohortFutures the cohort Futures for each shard transaction.
423      */
424     protected void onTransactionReady(List<Future<ActorSelection>> cohortFutures) {
425     }
426
427     /**
428      * Method called to send a CreateTransaction message to a shard.
429      *
430      * @param shard the shard actor to send to
431      * @param serializedCreateMessage the serialized message to send
432      * @return the response Future
433      */
434     protected Future<Object> sendCreateTransaction(ActorSelection shard,
435             Object serializedCreateMessage) {
436         return actorContext.executeOperationAsync(shard, serializedCreateMessage);
437     }
438
439     @Override
440     public Object getIdentifier() {
441         return this.identifier;
442     }
443
444     @Override
445     public void close() {
446         for (TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
447             txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
448                 @Override
449                 public void invoke(TransactionContext transactionContext) {
450                     transactionContext.closeTransaction();
451                 }
452             });
453         }
454
455         txFutureCallbackMap.clear();
456
457         if(transactionType == TransactionType.READ_ONLY) {
458             remoteTransactionActors.clear();
459             remoteTransactionActorsMB.set(true);
460         }
461     }
462
463     private String shardNameFromIdentifier(YangInstanceIdentifier path){
464         return ShardStrategyFactory.getStrategy(path).findShard(path);
465     }
466
467     private TransactionFutureCallback getOrCreateTxFutureCallback(YangInstanceIdentifier path) {
468         String shardName = shardNameFromIdentifier(path);
469         TransactionFutureCallback txFutureCallback = txFutureCallbackMap.get(shardName);
470         if(txFutureCallback == null) {
471             Future<ActorSelection> findPrimaryFuture = actorContext.findPrimaryShardAsync(shardName);
472
473             final TransactionFutureCallback newTxFutureCallback =
474                     new TransactionFutureCallback(shardName);
475
476             txFutureCallback = newTxFutureCallback;
477             txFutureCallbackMap.put(shardName, txFutureCallback);
478
479             findPrimaryFuture.onComplete(new OnComplete<ActorSelection>() {
480                 @Override
481                 public void onComplete(Throwable failure, ActorSelection primaryShard) {
482                     if(failure != null) {
483                         newTxFutureCallback.onComplete(failure, null);
484                     } else {
485                         newTxFutureCallback.setPrimaryShard(primaryShard);
486                     }
487                 }
488             }, actorContext.getClientDispatcher());
489         }
490
491         return txFutureCallback;
492     }
493
494     public String getTransactionChainId() {
495         return transactionChainId;
496     }
497
498     protected ActorContext getActorContext() {
499         return actorContext;
500     }
501
502     /**
503      * Interfaces for transaction operations to be invoked later.
504      */
505     private static interface TransactionOperation {
506         void 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         void enqueueTransactionOperation(final TransactionOperation op) {
586
587             if (transactionContext != null) {
588                 op.invoke(transactionContext);
589             } else {
590                 // The shard Tx hasn't been created yet so add the Tx operation to the Tx Future
591                 // callback to be executed after the Tx is created.
592                 addTxOperationOnComplete(op);
593             }
594         }
595
596         /**
597          * Performs a CreateTransaction try async.
598          */
599         private void tryCreateTransaction() {
600             Future<Object> createTxFuture = sendCreateTransaction(primaryShard,
601                     new CreateTransaction(identifier.toString(),
602                             TransactionProxy.this.transactionType.ordinal(),
603                             getTransactionChainId()).toSerializable());
604
605             createTxFuture.onComplete(this, actorContext.getClientDispatcher());
606         }
607
608         @Override
609         public void onComplete(Throwable failure, Object response) {
610             if(failure instanceof NoShardLeaderException) {
611                 // There's no leader for the shard yet - schedule and try again, unless we're out
612                 // of retries. Note: createTxTries is volatile as it may be written by different
613                 // threads however not concurrently, therefore decrementing it non-atomically here
614                 // is ok.
615                 if(--createTxTries > 0) {
616                     LOG.debug("Tx {} Shard {} has no leader yet - scheduling create Tx retry",
617                             identifier, shardName);
618
619                     actorContext.getActorSystem().scheduler().scheduleOnce(CREATE_TX_TRY_INTERVAL,
620                             new Runnable() {
621                                 @Override
622                                 public void run() {
623                                     tryCreateTransaction();
624                                 }
625                             }, actorContext.getClientDispatcher());
626                     return;
627                 }
628             }
629
630             // Create the TransactionContext from the response or failure. Store the new
631             // TransactionContext locally until we've completed invoking the
632             // TransactionOperations. This avoids thread timing issues which could cause
633             // out-of-order TransactionOperations. Eg, on a modification operation, if the
634             // TransactionContext is non-null, then we directly call the TransactionContext.
635             // However, at the same time, the code may be executing the cached
636             // TransactionOperations. So to avoid thus timing, we don't publish the
637             // TransactionContext until after we've executed all cached TransactionOperations.
638             TransactionContext localTransactionContext;
639             if(failure != null) {
640                 LOG.debug("Tx {} Creating NoOpTransaction because of error: {}", identifier,
641                         failure.getMessage());
642
643                 localTransactionContext = new NoOpTransactionContext(failure, identifier, operationLimiter);
644             } else if (response.getClass().equals(CreateTransactionReply.SERIALIZABLE_CLASS)) {
645                 localTransactionContext = createValidTransactionContext(
646                         CreateTransactionReply.fromSerializable(response));
647             } else {
648                 IllegalArgumentException exception = new IllegalArgumentException(String.format(
649                         "Invalid reply type %s for CreateTransaction", response.getClass()));
650
651                 localTransactionContext = new NoOpTransactionContext(exception, identifier, operationLimiter);
652             }
653
654             executeTxOperatonsOnComplete(localTransactionContext);
655         }
656
657         private void executeTxOperatonsOnComplete(TransactionContext localTransactionContext) {
658             while(true) {
659                 // Access to txOperationsOnComplete and transactionContext must be protected and atomic
660                 // (ie synchronized) with respect to #addTxOperationOnComplete to handle timing
661                 // issues and ensure no TransactionOperation is missed and that they are processed
662                 // in the order they occurred.
663
664                 // We'll make a local copy of the txOperationsOnComplete list to handle re-entrancy
665                 // in case a TransactionOperation results in another transaction operation being
666                 // queued (eg a put operation from a client read Future callback that is notified
667                 // synchronously).
668                 Collection<TransactionOperation> operationsBatch = null;
669                 synchronized(txOperationsOnComplete) {
670                     if(txOperationsOnComplete.isEmpty()) {
671                         // We're done invoking the TransactionOperations so we can now publish the
672                         // TransactionContext.
673                         transactionContext = localTransactionContext;
674                         break;
675                     }
676
677                     operationsBatch = new ArrayList<>(txOperationsOnComplete);
678                     txOperationsOnComplete.clear();
679                 }
680
681                 // Invoke TransactionOperations outside the sync block to avoid unnecessary blocking.
682                 // A slight down-side is that we need to re-acquire the lock below but this should
683                 // be negligible.
684                 for(TransactionOperation oper: operationsBatch) {
685                     oper.invoke(localTransactionContext);
686                 }
687             }
688         }
689
690         private TransactionContext createValidTransactionContext(CreateTransactionReply reply) {
691             String transactionPath = reply.getTransactionPath();
692
693             LOG.debug("Tx {} Received {}", identifier, reply);
694
695             ActorSelection transactionActor = actorContext.actorSelection(transactionPath);
696
697             if (transactionType == TransactionType.READ_ONLY) {
698                 // Add the actor to the remoteTransactionActors list for access by the
699                 // cleanup PhantonReference.
700                 remoteTransactionActors.add(transactionActor);
701
702                 // Write to the memory barrier volatile to publish the above update to the
703                 // remoteTransactionActors list for thread visibility.
704                 remoteTransactionActorsMB.set(true);
705             }
706
707             // TxActor is always created where the leader of the shard is.
708             // Check if TxActor is created in the same node
709             boolean isTxActorLocal = actorContext.isPathLocal(transactionPath);
710
711             if(reply.getVersion() >= DataStoreVersions.LITHIUM_VERSION) {
712                 return new TransactionContextImpl(transactionPath, transactionActor, identifier,
713                     actorContext, schemaContext, isTxActorLocal, reply.getVersion(), operationCompleter);
714             } else {
715                 return new LegacyTransactionContextImpl(transactionPath, transactionActor, identifier,
716                         actorContext, schemaContext, isTxActorLocal, reply.getVersion(), operationCompleter);
717             }
718         }
719     }
720 }