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