Merge "Split out ChainedTransactionProxy"
[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                 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 {}", 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.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 {}", 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      * Interfaces for transaction operations to be invoked later.
505      */
506     private static interface TransactionOperation {
507         void invoke(TransactionContext transactionContext);
508     }
509
510     /**
511      * Implements a Future OnComplete callback for a CreateTransaction message. This class handles
512      * retries, up to a limit, if the shard doesn't have a leader yet. This is done by scheduling a
513      * retry task after a short delay.
514      * <p>
515      * The end result from a completed CreateTransaction message is a TransactionContext that is
516      * used to perform transaction operations. Transaction operations that occur before the
517      * CreateTransaction completes are cache and executed once the CreateTransaction completes,
518      * successfully or not.
519      */
520     private class TransactionFutureCallback extends OnComplete<Object> {
521
522         /**
523          * The list of transaction operations to execute once the CreateTransaction completes.
524          */
525         @GuardedBy("txOperationsOnComplete")
526         private final List<TransactionOperation> txOperationsOnComplete = Lists.newArrayList();
527
528         /**
529          * The TransactionContext resulting from the CreateTransaction reply.
530          */
531         private volatile TransactionContext transactionContext;
532
533         /**
534          * The target primary shard.
535          */
536         private volatile ActorSelection primaryShard;
537
538         private volatile int createTxTries = (int) (actorContext.getDatastoreContext().
539                 getShardLeaderElectionTimeout().duration().toMillis() /
540                 CREATE_TX_TRY_INTERVAL.toMillis());
541
542         private final String shardName;
543
544         TransactionFutureCallback(String shardName) {
545             this.shardName = shardName;
546         }
547
548         String getShardName() {
549             return shardName;
550         }
551
552         TransactionContext getTransactionContext() {
553             return transactionContext;
554         }
555
556
557         /**
558          * Sets the target primary shard and initiates a CreateTransaction try.
559          */
560         void setPrimaryShard(ActorSelection primaryShard) {
561             this.primaryShard = primaryShard;
562
563             if(transactionType == TransactionType.WRITE_ONLY &&
564                     actorContext.getDatastoreContext().isWriteOnlyTransactionOptimizationsEnabled()) {
565                 LOG.debug("Tx {} Primary shard {} found - creating WRITE_ONLY transaction context",
566                     getIdentifier(), primaryShard);
567
568                 // For write-only Tx's we prepare the transaction modifications directly on the shard actor
569                 // to avoid the overhead of creating a separate transaction actor.
570                 // FIXME: can't assume the shard version is LITHIUM_VERSION - need to obtain it somehow.
571                 executeTxOperatonsOnComplete(createValidTransactionContext(this.primaryShard,
572                         this.primaryShard.path().toString(), DataStoreVersions.LITHIUM_VERSION));
573             } else {
574                 tryCreateTransaction();
575             }
576         }
577
578         /**
579          * Adds a TransactionOperation to be executed after the CreateTransaction completes.
580          */
581         void addTxOperationOnComplete(TransactionOperation operation) {
582             boolean invokeOperation = true;
583             synchronized(txOperationsOnComplete) {
584                 if(transactionContext == null) {
585                     LOG.debug("Tx {} Adding operation on complete", getIdentifier());
586
587                     invokeOperation = false;
588                     txOperationsOnComplete.add(operation);
589                 }
590             }
591
592             if(invokeOperation) {
593                 operation.invoke(transactionContext);
594             }
595         }
596
597         void enqueueTransactionOperation(final TransactionOperation op) {
598
599             if (transactionContext != null) {
600                 op.invoke(transactionContext);
601             } else {
602                 // The shard Tx hasn't been created yet so add the Tx operation to the Tx Future
603                 // callback to be executed after the Tx is created.
604                 addTxOperationOnComplete(op);
605             }
606         }
607
608         /**
609          * Performs a CreateTransaction try async.
610          */
611         private void tryCreateTransaction() {
612             if(LOG.isDebugEnabled()) {
613                 LOG.debug("Tx {} Primary shard {} found - trying create transaction", getIdentifier(), primaryShard);
614             }
615
616             Object serializedCreateMessage = new CreateTransaction(getIdentifier().toString(),
617                     TransactionProxy.this.transactionType.ordinal(),
618                     getTransactionChainId()).toSerializable();
619
620             Future<Object> createTxFuture = actorContext.executeOperationAsync(primaryShard, serializedCreateMessage);
621
622             createTxFuture.onComplete(this, actorContext.getClientDispatcher());
623         }
624
625         @Override
626         public void onComplete(Throwable failure, Object response) {
627             if(failure instanceof NoShardLeaderException) {
628                 // There's no leader for the shard yet - schedule and try again, unless we're out
629                 // of retries. Note: createTxTries is volatile as it may be written by different
630                 // threads however not concurrently, therefore decrementing it non-atomically here
631                 // is ok.
632                 if(--createTxTries > 0) {
633                     LOG.debug("Tx {} Shard {} has no leader yet - scheduling create Tx retry",
634                         getIdentifier(), shardName);
635
636                     actorContext.getActorSystem().scheduler().scheduleOnce(CREATE_TX_TRY_INTERVAL,
637                             new Runnable() {
638                                 @Override
639                                 public void run() {
640                                     tryCreateTransaction();
641                                 }
642                             }, actorContext.getClientDispatcher());
643                     return;
644                 }
645             }
646
647             createTransactionContext(failure, response);
648         }
649
650         private void createTransactionContext(Throwable failure, Object response) {
651             // Mainly checking for state violation here to perform a volatile read of "initialized" to
652             // ensure updates to operationLimter et al are visible to this thread (ie we're doing
653             // "piggy-back" synchronization here).
654             Preconditions.checkState(initialized, "Tx was not propertly initialized.");
655
656             // Create the TransactionContext from the response or failure. Store the new
657             // TransactionContext locally until we've completed invoking the
658             // TransactionOperations. This avoids thread timing issues which could cause
659             // out-of-order TransactionOperations. Eg, on a modification operation, if the
660             // TransactionContext is non-null, then we directly call the TransactionContext.
661             // However, at the same time, the code may be executing the cached
662             // TransactionOperations. So to avoid thus timing, we don't publish the
663             // TransactionContext until after we've executed all cached TransactionOperations.
664             TransactionContext localTransactionContext;
665             if(failure != null) {
666                 LOG.debug("Tx {} Creating NoOpTransaction because of error", getIdentifier(), failure);
667
668                 localTransactionContext = new NoOpTransactionContext(failure, getIdentifier(), operationLimiter);
669             } else if (CreateTransactionReply.SERIALIZABLE_CLASS.equals(response.getClass())) {
670                 localTransactionContext = createValidTransactionContext(
671                         CreateTransactionReply.fromSerializable(response));
672             } else {
673                 IllegalArgumentException exception = new IllegalArgumentException(String.format(
674                         "Invalid reply type %s for CreateTransaction", response.getClass()));
675
676                 localTransactionContext = new NoOpTransactionContext(exception, getIdentifier(), operationLimiter);
677             }
678
679             executeTxOperatonsOnComplete(localTransactionContext);
680         }
681
682         private void executeTxOperatonsOnComplete(TransactionContext localTransactionContext) {
683             while(true) {
684                 // Access to txOperationsOnComplete and transactionContext must be protected and atomic
685                 // (ie synchronized) with respect to #addTxOperationOnComplete to handle timing
686                 // issues and ensure no TransactionOperation is missed and that they are processed
687                 // in the order they occurred.
688
689                 // We'll make a local copy of the txOperationsOnComplete list to handle re-entrancy
690                 // in case a TransactionOperation results in another transaction operation being
691                 // queued (eg a put operation from a client read Future callback that is notified
692                 // synchronously).
693                 Collection<TransactionOperation> operationsBatch = null;
694                 synchronized(txOperationsOnComplete) {
695                     if(txOperationsOnComplete.isEmpty()) {
696                         // We're done invoking the TransactionOperations so we can now publish the
697                         // TransactionContext.
698                         transactionContext = localTransactionContext;
699                         break;
700                     }
701
702                     operationsBatch = new ArrayList<>(txOperationsOnComplete);
703                     txOperationsOnComplete.clear();
704                 }
705
706                 // Invoke TransactionOperations outside the sync block to avoid unnecessary blocking.
707                 // A slight down-side is that we need to re-acquire the lock below but this should
708                 // be negligible.
709                 for(TransactionOperation oper: operationsBatch) {
710                     oper.invoke(localTransactionContext);
711                 }
712             }
713         }
714
715         private TransactionContext createValidTransactionContext(CreateTransactionReply reply) {
716             LOG.debug("Tx {} Received {}", getIdentifier(), reply);
717
718             return createValidTransactionContext(actorContext.actorSelection(reply.getTransactionPath()),
719                     reply.getTransactionPath(), reply.getVersion());
720         }
721
722         private TransactionContext createValidTransactionContext(ActorSelection transactionActor,
723                 String transactionPath, short remoteTransactionVersion) {
724
725             if (transactionType == TransactionType.READ_ONLY) {
726                 // Read-only Tx's aren't explicitly closed by the client so we create a PhantomReference
727                 // to close the remote Tx's when this instance is no longer in use and is garbage
728                 // collected.
729
730                 if(remoteTransactionActorsMB == null) {
731                     remoteTransactionActors = Lists.newArrayList();
732                     remoteTransactionActorsMB = new AtomicBoolean();
733
734                     TransactionProxyCleanupPhantomReference cleanup =
735                             new TransactionProxyCleanupPhantomReference(TransactionProxy.this);
736                     phantomReferenceCache.put(cleanup, cleanup);
737                 }
738
739                 // Add the actor to the remoteTransactionActors list for access by the
740                 // cleanup PhantonReference.
741                 remoteTransactionActors.add(transactionActor);
742
743                 // Write to the memory barrier volatile to publish the above update to the
744                 // remoteTransactionActors list for thread visibility.
745                 remoteTransactionActorsMB.set(true);
746             }
747
748             // TxActor is always created where the leader of the shard is.
749             // Check if TxActor is created in the same node
750             boolean isTxActorLocal = actorContext.isPathLocal(transactionPath);
751
752             if(remoteTransactionVersion < DataStoreVersions.LITHIUM_VERSION) {
753                 return new PreLithiumTransactionContextImpl(transactionPath, transactionActor, getIdentifier(),
754                         transactionChainId, actorContext, schemaContext, isTxActorLocal, remoteTransactionVersion,
755                         operationCompleter);
756             } else if (transactionType == TransactionType.WRITE_ONLY &&
757                     actorContext.getDatastoreContext().isWriteOnlyTransactionOptimizationsEnabled()) {
758                 return new WriteOnlyTransactionContextImpl(transactionActor, getIdentifier(), transactionChainId,
759                     actorContext, schemaContext, isTxActorLocal, remoteTransactionVersion, operationCompleter);
760             } else {
761                 return new TransactionContextImpl(transactionActor, getIdentifier(), transactionChainId,
762                         actorContext, schemaContext, isTxActorLocal, remoteTransactionVersion, operationCompleter);
763             }
764         }
765     }
766 }