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