f1ba4eabb999a6455688d93532877ae490df88df
[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.ListenableFuture;
22 import com.google.common.util.concurrent.SettableFuture;
23 import java.util.ArrayList;
24 import java.util.Collection;
25 import java.util.Collections;
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.compat.PreLithiumTransactionContextImpl;
36 import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException;
37 import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier;
38 import org.opendaylight.controller.cluster.datastore.messages.CloseTransaction;
39 import org.opendaylight.controller.cluster.datastore.messages.CreateTransaction;
40 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionReply;
41 import org.opendaylight.controller.cluster.datastore.shardstrategy.ShardStrategyFactory;
42 import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
43 import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
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 implements DOMStoreReadWriteTransaction {
69
70     public static enum TransactionType {
71         READ_ONLY,
72         WRITE_ONLY,
73         READ_WRITE;
74
75         public static TransactionType fromInt(int type) {
76             if(type == WRITE_ONLY.ordinal()) {
77                 return WRITE_ONLY;
78             } else if(type == READ_WRITE.ordinal()) {
79                 return READ_WRITE;
80             } else if(type == READ_ONLY.ordinal()) {
81                 return READ_ONLY;
82             } else {
83                 throw new IllegalArgumentException("In TransactionType enum value" + type);
84             }
85         }
86     }
87
88     static final Mapper<Throwable, Throwable> SAME_FAILURE_TRANSFORMER =
89                                                               new Mapper<Throwable, Throwable>() {
90         @Override
91         public Throwable apply(Throwable failure) {
92             return failure;
93         }
94     };
95
96     private static final AtomicLong counter = new AtomicLong();
97
98     private static final Logger LOG = LoggerFactory.getLogger(TransactionProxy.class);
99
100     /**
101      * Time interval in between transaction create retries.
102      */
103     private static final FiniteDuration CREATE_TX_TRY_INTERVAL =
104             FiniteDuration.create(1, TimeUnit.SECONDS);
105
106     /**
107      * Used to enqueue the PhantomReferences for read-only TransactionProxy instances. The
108      * FinalizableReferenceQueue is safe to use statically in an OSGi environment as it uses some
109      * trickery to clean up its internal thread when the bundle is unloaded.
110      */
111     private static final FinalizableReferenceQueue phantomReferenceQueue =
112                                                                   new FinalizableReferenceQueue();
113
114     /**
115      * This stores the TransactionProxyCleanupPhantomReference instances statically, This is
116      * necessary because PhantomReferences need a hard reference so they're not garbage collected.
117      * Once finalized, the TransactionProxyCleanupPhantomReference removes itself from this map
118      * and thus becomes eligible for garbage collection.
119      */
120     private static final Map<TransactionProxyCleanupPhantomReference,
121                              TransactionProxyCleanupPhantomReference> phantomReferenceCache =
122                                                                         new ConcurrentHashMap<>();
123
124     /**
125      * A PhantomReference that closes remote transactions for a TransactionProxy when it's
126      * garbage collected. This is used for read-only transactions as they're not explicitly closed
127      * by clients. So the only way to detect that a transaction is no longer in use and it's safe
128      * to clean up is when it's garbage collected. It's inexact as to when an instance will be GC'ed
129      * but TransactionProxy instances should generally be short-lived enough to avoid being moved
130      * to the old generation space and thus should be cleaned up in a timely manner as the GC
131      * runs on the young generation (eden, swap1...) space much more frequently.
132      */
133     private static class TransactionProxyCleanupPhantomReference
134                                            extends FinalizablePhantomReference<TransactionProxy> {
135
136         private final List<ActorSelection> remoteTransactionActors;
137         private final AtomicBoolean remoteTransactionActorsMB;
138         private final ActorContext actorContext;
139         private final TransactionIdentifier identifier;
140
141         protected TransactionProxyCleanupPhantomReference(TransactionProxy referent) {
142             super(referent, phantomReferenceQueue);
143
144             // Note we need to cache the relevant fields from the TransactionProxy as we can't
145             // have a hard reference to the TransactionProxy instance itself.
146
147             remoteTransactionActors = referent.remoteTransactionActors;
148             remoteTransactionActorsMB = referent.remoteTransactionActorsMB;
149             actorContext = referent.actorContext;
150             identifier = referent.identifier;
151         }
152
153         @Override
154         public void finalizeReferent() {
155             LOG.trace("Cleaning up {} Tx actors for TransactionProxy {}",
156                     remoteTransactionActors.size(), identifier);
157
158             phantomReferenceCache.remove(this);
159
160             // Access the memory barrier volatile to ensure all previous updates to the
161             // remoteTransactionActors list are visible to this thread.
162
163             if(remoteTransactionActorsMB.get()) {
164                 for(ActorSelection actor : remoteTransactionActors) {
165                     LOG.trace("Sending CloseTransaction to {}", actor);
166                     actorContext.sendOperationAsync(actor, CloseTransaction.INSTANCE.toSerializable());
167                 }
168             }
169         }
170     }
171
172     /**
173      * Stores the remote Tx actors for each requested data store path to be used by the
174      * PhantomReference to close the remote Tx's. This is only used for read-only Tx's. The
175      * remoteTransactionActorsMB volatile serves as a memory barrier to publish updates to the
176      * remoteTransactionActors list so they will be visible to the thread accessing the
177      * PhantomReference.
178      */
179     private List<ActorSelection> remoteTransactionActors;
180     private volatile AtomicBoolean remoteTransactionActorsMB;
181
182     /**
183      * Stores the create transaction results per shard.
184      */
185     private final Map<String, TransactionFutureCallback> txFutureCallbackMap = new HashMap<>();
186
187     private final TransactionType transactionType;
188     private final ActorContext actorContext;
189     private final TransactionIdentifier identifier;
190     private final String transactionChainId;
191     private final SchemaContext schemaContext;
192     private boolean inReadyState;
193
194     private volatile boolean initialized;
195     private Semaphore operationLimiter;
196     private OperationCompleter operationCompleter;
197
198     public TransactionProxy(ActorContext actorContext, TransactionType transactionType) {
199         this(actorContext, transactionType, "");
200     }
201
202     public TransactionProxy(ActorContext actorContext, TransactionType transactionType,
203             String transactionChainId) {
204         this.actorContext = Preconditions.checkNotNull(actorContext,
205             "actorContext should not be null");
206         this.transactionType = Preconditions.checkNotNull(transactionType,
207             "transactionType should not be null");
208         this.schemaContext = Preconditions.checkNotNull(actorContext.getSchemaContext(),
209             "schemaContext should not be null");
210         this.transactionChainId = transactionChainId;
211
212         String memberName = actorContext.getCurrentMemberName();
213         if(memberName == null){
214             memberName = "UNKNOWN-MEMBER";
215         }
216
217         this.identifier = new TransactionIdentifier(memberName, counter.getAndIncrement());
218
219         LOG.debug("Created txn {} of type {} on chain {}", identifier, transactionType, transactionChainId);
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                 recordedOperationFutures.addAll(transactionContext.getRecordedOperationFutures());
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 {}", identifier, 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 {}", identifier, 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 {}", identifier, 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 {}", identifier, 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 {}", identifier, 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", identifier,
392                     txFutureCallbackMap.size());
393
394         if(txFutureCallbackMap.size() == 0) {
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 {}", identifier,
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                 identifier.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 Object getIdentifier() {
444         return this.identifier;
445     }
446
447     @Override
448     public void close() {
449         for (TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
450             txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
451                 @Override
452                 public void invoke(TransactionContext transactionContext) {
453                     transactionContext.closeTransaction();
454                 }
455             });
456         }
457
458         txFutureCallbackMap.clear();
459
460         if(remoteTransactionActorsMB != null) {
461             remoteTransactionActors.clear();
462             remoteTransactionActorsMB.set(true);
463         }
464     }
465
466     private String shardNameFromIdentifier(YangInstanceIdentifier path){
467         return ShardStrategyFactory.getStrategy(path).findShard(path);
468     }
469
470     protected Future<ActorSelection> sendFindPrimaryShardAsync(String shardName) {
471         return actorContext.findPrimaryShardAsync(shardName);
472     }
473
474     private TransactionFutureCallback getOrCreateTxFutureCallback(YangInstanceIdentifier path) {
475         String shardName = shardNameFromIdentifier(path);
476         TransactionFutureCallback txFutureCallback = txFutureCallbackMap.get(shardName);
477         if(txFutureCallback == null) {
478             Future<ActorSelection> findPrimaryFuture = sendFindPrimaryShardAsync(shardName);
479
480             final TransactionFutureCallback newTxFutureCallback = new TransactionFutureCallback(shardName);
481
482             txFutureCallback = newTxFutureCallback;
483             txFutureCallbackMap.put(shardName, txFutureCallback);
484
485             findPrimaryFuture.onComplete(new OnComplete<ActorSelection>() {
486                 @Override
487                 public void onComplete(Throwable failure, ActorSelection primaryShard) {
488                     if(failure != null) {
489                         newTxFutureCallback.createTransactionContext(failure, null);
490                     } else {
491                         newTxFutureCallback.setPrimaryShard(primaryShard);
492                     }
493                 }
494             }, actorContext.getClientDispatcher());
495         }
496
497         return txFutureCallback;
498     }
499
500     public String getTransactionChainId() {
501         return transactionChainId;
502     }
503
504     protected ActorContext getActorContext() {
505         return actorContext;
506     }
507
508     /**
509      * Interfaces for transaction operations to be invoked later.
510      */
511     private static interface TransactionOperation {
512         void invoke(TransactionContext transactionContext);
513     }
514
515     /**
516      * Implements a Future OnComplete callback for a CreateTransaction message. This class handles
517      * retries, up to a limit, if the shard doesn't have a leader yet. This is done by scheduling a
518      * retry task after a short delay.
519      * <p>
520      * The end result from a completed CreateTransaction message is a TransactionContext that is
521      * used to perform transaction operations. Transaction operations that occur before the
522      * CreateTransaction completes are cache and executed once the CreateTransaction completes,
523      * successfully or not.
524      */
525     private class TransactionFutureCallback extends OnComplete<Object> {
526
527         /**
528          * The list of transaction operations to execute once the CreateTransaction completes.
529          */
530         @GuardedBy("txOperationsOnComplete")
531         private final List<TransactionOperation> txOperationsOnComplete = Lists.newArrayList();
532
533         /**
534          * The TransactionContext resulting from the CreateTransaction reply.
535          */
536         private volatile TransactionContext transactionContext;
537
538         /**
539          * The target primary shard.
540          */
541         private volatile ActorSelection primaryShard;
542
543         private volatile int createTxTries = (int) (actorContext.getDatastoreContext().
544                 getShardLeaderElectionTimeout().duration().toMillis() /
545                 CREATE_TX_TRY_INTERVAL.toMillis());
546
547         private final String shardName;
548
549         TransactionFutureCallback(String shardName) {
550             this.shardName = shardName;
551         }
552
553         String getShardName() {
554             return shardName;
555         }
556
557         TransactionContext getTransactionContext() {
558             return transactionContext;
559         }
560
561
562         /**
563          * Sets the target primary shard and initiates a CreateTransaction try.
564          */
565         void setPrimaryShard(ActorSelection primaryShard) {
566             this.primaryShard = primaryShard;
567
568             if(transactionType == TransactionType.WRITE_ONLY &&
569                     actorContext.getDatastoreContext().isWriteOnlyTransactionOptimizationsEnabled()) {
570                 LOG.debug("Tx {} Primary shard {} found - creating WRITE_ONLY transaction context",
571                         identifier, primaryShard);
572
573                 // For write-only Tx's we prepare the transaction modifications directly on the shard actor
574                 // to avoid the overhead of creating a separate transaction actor.
575                 // FIXME: can't assume the shard version is LITHIUM_VERSION - need to obtain it somehow.
576                 executeTxOperatonsOnComplete(createValidTransactionContext(this.primaryShard,
577                         this.primaryShard.path().toString(), DataStoreVersions.LITHIUM_VERSION));
578             } else {
579                 tryCreateTransaction();
580             }
581         }
582
583         /**
584          * Adds a TransactionOperation to be executed after the CreateTransaction completes.
585          */
586         void addTxOperationOnComplete(TransactionOperation operation) {
587             boolean invokeOperation = true;
588             synchronized(txOperationsOnComplete) {
589                 if(transactionContext == null) {
590                     LOG.debug("Tx {} Adding operation on complete", identifier);
591
592                     invokeOperation = false;
593                     txOperationsOnComplete.add(operation);
594                 }
595             }
596
597             if(invokeOperation) {
598                 operation.invoke(transactionContext);
599             }
600         }
601
602         void enqueueTransactionOperation(final TransactionOperation op) {
603
604             if (transactionContext != null) {
605                 op.invoke(transactionContext);
606             } else {
607                 // The shard Tx hasn't been created yet so add the Tx operation to the Tx Future
608                 // callback to be executed after the Tx is created.
609                 addTxOperationOnComplete(op);
610             }
611         }
612
613         /**
614          * Performs a CreateTransaction try async.
615          */
616         private void tryCreateTransaction() {
617             if(LOG.isDebugEnabled()) {
618                 LOG.debug("Tx {} Primary shard {} found - trying create transaction", identifier, primaryShard);
619             }
620
621             Object serializedCreateMessage = new CreateTransaction(identifier.toString(),
622                     TransactionProxy.this.transactionType.ordinal(),
623                     getTransactionChainId()).toSerializable();
624
625             Future<Object> createTxFuture = actorContext.executeOperationAsync(primaryShard, serializedCreateMessage);
626
627             createTxFuture.onComplete(this, actorContext.getClientDispatcher());
628         }
629
630         @Override
631         public void onComplete(Throwable failure, Object response) {
632             if(failure instanceof NoShardLeaderException) {
633                 // There's no leader for the shard yet - schedule and try again, unless we're out
634                 // of retries. Note: createTxTries is volatile as it may be written by different
635                 // threads however not concurrently, therefore decrementing it non-atomically here
636                 // is ok.
637                 if(--createTxTries > 0) {
638                     LOG.debug("Tx {} Shard {} has no leader yet - scheduling create Tx retry",
639                             identifier, shardName);
640
641                     actorContext.getActorSystem().scheduler().scheduleOnce(CREATE_TX_TRY_INTERVAL,
642                             new Runnable() {
643                                 @Override
644                                 public void run() {
645                                     tryCreateTransaction();
646                                 }
647                             }, actorContext.getClientDispatcher());
648                     return;
649                 }
650             }
651
652             createTransactionContext(failure, response);
653         }
654
655         private void createTransactionContext(Throwable failure, Object response) {
656             // Mainly checking for state violation here to perform a volatile read of "initialized" to
657             // ensure updates to operationLimter et al are visible to this thread (ie we're doing
658             // "piggy-back" synchronization here).
659             Preconditions.checkState(initialized, "Tx was not propertly initialized.");
660
661             // Create the TransactionContext from the response or failure. Store the new
662             // TransactionContext locally until we've completed invoking the
663             // TransactionOperations. This avoids thread timing issues which could cause
664             // out-of-order TransactionOperations. Eg, on a modification operation, if the
665             // TransactionContext is non-null, then we directly call the TransactionContext.
666             // However, at the same time, the code may be executing the cached
667             // TransactionOperations. So to avoid thus timing, we don't publish the
668             // TransactionContext until after we've executed all cached TransactionOperations.
669             TransactionContext localTransactionContext;
670             if(failure != null) {
671                 LOG.debug("Tx {} Creating NoOpTransaction because of error", identifier, failure);
672
673                 localTransactionContext = new NoOpTransactionContext(failure, identifier, operationLimiter);
674             } else if (response.getClass().equals(CreateTransactionReply.SERIALIZABLE_CLASS)) {
675                 localTransactionContext = createValidTransactionContext(
676                         CreateTransactionReply.fromSerializable(response));
677             } else {
678                 IllegalArgumentException exception = new IllegalArgumentException(String.format(
679                         "Invalid reply type %s for CreateTransaction", response.getClass()));
680
681                 localTransactionContext = new NoOpTransactionContext(exception, identifier, operationLimiter);
682             }
683
684             executeTxOperatonsOnComplete(localTransactionContext);
685         }
686
687         private void executeTxOperatonsOnComplete(TransactionContext localTransactionContext) {
688             while(true) {
689                 // Access to txOperationsOnComplete and transactionContext must be protected and atomic
690                 // (ie synchronized) with respect to #addTxOperationOnComplete to handle timing
691                 // issues and ensure no TransactionOperation is missed and that they are processed
692                 // in the order they occurred.
693
694                 // We'll make a local copy of the txOperationsOnComplete list to handle re-entrancy
695                 // in case a TransactionOperation results in another transaction operation being
696                 // queued (eg a put operation from a client read Future callback that is notified
697                 // synchronously).
698                 Collection<TransactionOperation> operationsBatch = null;
699                 synchronized(txOperationsOnComplete) {
700                     if(txOperationsOnComplete.isEmpty()) {
701                         // We're done invoking the TransactionOperations so we can now publish the
702                         // TransactionContext.
703                         transactionContext = localTransactionContext;
704                         break;
705                     }
706
707                     operationsBatch = new ArrayList<>(txOperationsOnComplete);
708                     txOperationsOnComplete.clear();
709                 }
710
711                 // Invoke TransactionOperations outside the sync block to avoid unnecessary blocking.
712                 // A slight down-side is that we need to re-acquire the lock below but this should
713                 // be negligible.
714                 for(TransactionOperation oper: operationsBatch) {
715                     oper.invoke(localTransactionContext);
716                 }
717             }
718         }
719
720         private TransactionContext createValidTransactionContext(CreateTransactionReply reply) {
721             LOG.debug("Tx {} Received {}", identifier, reply);
722
723             return createValidTransactionContext(actorContext.actorSelection(reply.getTransactionPath()),
724                     reply.getTransactionPath(), reply.getVersion());
725         }
726
727         private TransactionContext createValidTransactionContext(ActorSelection transactionActor,
728                 String transactionPath, short remoteTransactionVersion) {
729
730             if (transactionType == TransactionType.READ_ONLY) {
731                 // Read-only Tx's aren't explicitly closed by the client so we create a PhantomReference
732                 // to close the remote Tx's when this instance is no longer in use and is garbage
733                 // collected.
734
735                 if(remoteTransactionActorsMB == null) {
736                     remoteTransactionActors = Lists.newArrayList();
737                     remoteTransactionActorsMB = new AtomicBoolean();
738
739                     TransactionProxyCleanupPhantomReference cleanup =
740                             new TransactionProxyCleanupPhantomReference(TransactionProxy.this);
741                     phantomReferenceCache.put(cleanup, cleanup);
742                 }
743
744                 // Add the actor to the remoteTransactionActors list for access by the
745                 // cleanup PhantonReference.
746                 remoteTransactionActors.add(transactionActor);
747
748                 // Write to the memory barrier volatile to publish the above update to the
749                 // remoteTransactionActors list for thread visibility.
750                 remoteTransactionActorsMB.set(true);
751             }
752
753             // TxActor is always created where the leader of the shard is.
754             // Check if TxActor is created in the same node
755             boolean isTxActorLocal = actorContext.isPathLocal(transactionPath);
756
757             if(remoteTransactionVersion < DataStoreVersions.LITHIUM_VERSION) {
758                 return new PreLithiumTransactionContextImpl(transactionPath, transactionActor, identifier,
759                         transactionChainId, actorContext, schemaContext, isTxActorLocal, remoteTransactionVersion,
760                         operationCompleter);
761             } else if (transactionType == TransactionType.WRITE_ONLY &&
762                     actorContext.getDatastoreContext().isWriteOnlyTransactionOptimizationsEnabled()) {
763                 return new WriteOnlyTransactionContextImpl(transactionActor, identifier, transactionChainId,
764                     actorContext, schemaContext, isTxActorLocal, remoteTransactionVersion, operationCompleter);
765             } else {
766                 return new TransactionContextImpl(transactionActor, identifier, transactionChainId,
767                         actorContext, schemaContext, isTxActorLocal, remoteTransactionVersion, operationCompleter);
768             }
769         }
770     }
771
772     private static class NoOpDOMStoreThreePhaseCommitCohort implements DOMStoreThreePhaseCommitCohort {
773         static NoOpDOMStoreThreePhaseCommitCohort INSTANCE = new NoOpDOMStoreThreePhaseCommitCohort();
774
775         private static final ListenableFuture<Void> IMMEDIATE_VOID_SUCCESS =
776                 com.google.common.util.concurrent.Futures.immediateFuture(null);
777         private static final ListenableFuture<Boolean> IMMEDIATE_BOOLEAN_SUCCESS =
778                 com.google.common.util.concurrent.Futures.immediateFuture(Boolean.TRUE);
779
780         private NoOpDOMStoreThreePhaseCommitCohort() {
781         }
782
783         @Override
784         public ListenableFuture<Boolean> canCommit() {
785             return IMMEDIATE_BOOLEAN_SUCCESS;
786         }
787
788         @Override
789         public ListenableFuture<Void> preCommit() {
790             return IMMEDIATE_VOID_SUCCESS;
791         }
792
793         @Override
794         public ListenableFuture<Void> abort() {
795             return IMMEDIATE_VOID_SUCCESS;
796         }
797
798         @Override
799         public ListenableFuture<Void> commit() {
800             return IMMEDIATE_VOID_SUCCESS;
801         }
802     }
803 }