Speed up enumeration lookups
[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         // Cache all values
77         private static final TransactionType[] VALUES = values();
78
79         public static TransactionType fromInt(final int type) {
80             try {
81                 return VALUES[type];
82             } catch (IndexOutOfBoundsException e) {
83                 throw new IllegalArgumentException("In TransactionType enum value " + type, e);
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.getIdentifier();
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 String transactionChainId;
190     private final SchemaContext schemaContext;
191     private boolean inReadyState;
192
193     private volatile boolean initialized;
194     private Semaphore operationLimiter;
195     private OperationCompleter operationCompleter;
196
197     public TransactionProxy(ActorContext actorContext, TransactionType transactionType) {
198         this(actorContext, transactionType, "");
199     }
200
201     public TransactionProxy(ActorContext actorContext, TransactionType transactionType, String transactionChainId) {
202         super(createIdentifier(actorContext));
203         this.actorContext = Preconditions.checkNotNull(actorContext,
204             "actorContext should not be null");
205         this.transactionType = Preconditions.checkNotNull(transactionType,
206             "transactionType should not be null");
207         this.schemaContext = Preconditions.checkNotNull(actorContext.getSchemaContext(),
208             "schemaContext should not be null");
209         this.transactionChainId = transactionChainId;
210
211         LOG.debug("Created txn {} of type {} on chain {}", getIdentifier(), transactionType, transactionChainId);
212     }
213
214     private static TransactionIdentifier createIdentifier(ActorContext actorContext) {
215         String memberName = actorContext.getCurrentMemberName();
216         if (memberName == null) {
217             memberName = "UNKNOWN-MEMBER";
218         }
219
220         return new TransactionIdentifier(memberName, counter.getAndIncrement());
221     }
222
223     @VisibleForTesting
224     List<Future<Object>> getRecordedOperationFutures() {
225         List<Future<Object>> recordedOperationFutures = Lists.newArrayList();
226         for(TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
227             TransactionContext transactionContext = txFutureCallback.getTransactionContext();
228             if(transactionContext != null) {
229                 recordedOperationFutures.addAll(transactionContext.getRecordedOperationFutures());
230             }
231         }
232
233         return recordedOperationFutures;
234     }
235
236     @VisibleForTesting
237     boolean hasTransactionContext() {
238         for(TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
239             TransactionContext transactionContext = txFutureCallback.getTransactionContext();
240             if(transactionContext != null) {
241                 return true;
242             }
243         }
244
245         return false;
246     }
247
248     @Override
249     public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> read(final YangInstanceIdentifier path) {
250
251         Preconditions.checkState(transactionType != TransactionType.WRITE_ONLY,
252                 "Read operation on write-only transaction is not allowed");
253
254         LOG.debug("Tx {} read {}", getIdentifier(), path);
255
256         throttleOperation();
257
258         final SettableFuture<Optional<NormalizedNode<?, ?>>> proxyFuture = SettableFuture.create();
259
260         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
261         txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
262             @Override
263             public void invoke(TransactionContext transactionContext) {
264                 transactionContext.readData(path, proxyFuture);
265             }
266         });
267
268         return MappingCheckedFuture.create(proxyFuture, ReadFailedException.MAPPER);
269     }
270
271     @Override
272     public CheckedFuture<Boolean, ReadFailedException> exists(final YangInstanceIdentifier path) {
273
274         Preconditions.checkState(transactionType != TransactionType.WRITE_ONLY,
275                 "Exists operation on write-only transaction is not allowed");
276
277         LOG.debug("Tx {} exists {}", getIdentifier(), path);
278
279         throttleOperation();
280
281         final SettableFuture<Boolean> proxyFuture = SettableFuture.create();
282
283         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
284         txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
285             @Override
286             public void invoke(TransactionContext transactionContext) {
287                 transactionContext.dataExists(path, proxyFuture);
288             }
289         });
290
291         return MappingCheckedFuture.create(proxyFuture, ReadFailedException.MAPPER);
292     }
293
294     private void checkModificationState() {
295         Preconditions.checkState(transactionType != TransactionType.READ_ONLY,
296                 "Modification operation on read-only transaction is not allowed");
297         Preconditions.checkState(!inReadyState,
298                 "Transaction is sealed - further modifications are not allowed");
299     }
300
301     private void throttleOperation() {
302         throttleOperation(1);
303     }
304
305     private void throttleOperation(int acquirePermits) {
306         if(!initialized) {
307             // Note : Currently mailbox-capacity comes from akka.conf and not from the config-subsystem
308             operationLimiter = new Semaphore(actorContext.getTransactionOutstandingOperationLimit());
309             operationCompleter = new OperationCompleter(operationLimiter);
310
311             // Make sure we write this last because it's volatile and will also publish the non-volatile writes
312             // above as well so they'll be visible to other threads.
313             initialized = true;
314         }
315
316         try {
317             if(!operationLimiter.tryAcquire(acquirePermits,
318                     actorContext.getDatastoreContext().getOperationTimeoutInSeconds(), TimeUnit.SECONDS)){
319                 LOG.warn("Failed to acquire operation permit for transaction {}", getIdentifier());
320             }
321         } catch (InterruptedException e) {
322             if(LOG.isDebugEnabled()) {
323                 LOG.debug("Interrupted when trying to acquire operation permit for transaction " + getIdentifier().toString(), e);
324             } else {
325                 LOG.warn("Interrupted when trying to acquire operation permit for transaction {}", getIdentifier());
326             }
327         }
328     }
329
330
331     @Override
332     public void write(final YangInstanceIdentifier path, final NormalizedNode<?, ?> data) {
333
334         checkModificationState();
335
336         LOG.debug("Tx {} write {}", getIdentifier(), path);
337
338         throttleOperation();
339
340         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
341         txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
342             @Override
343             public void invoke(TransactionContext transactionContext) {
344                 transactionContext.writeData(path, data);
345             }
346         });
347     }
348
349     @Override
350     public void merge(final YangInstanceIdentifier path, final NormalizedNode<?, ?> data) {
351
352         checkModificationState();
353
354         LOG.debug("Tx {} merge {}", getIdentifier(), path);
355
356         throttleOperation();
357
358         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
359         txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
360             @Override
361             public void invoke(TransactionContext transactionContext) {
362                 transactionContext.mergeData(path, data);
363             }
364         });
365     }
366
367     @Override
368     public void delete(final YangInstanceIdentifier path) {
369
370         checkModificationState();
371
372         LOG.debug("Tx {} delete {}", getIdentifier(), path);
373
374         throttleOperation();
375
376         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
377         txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
378             @Override
379             public void invoke(TransactionContext transactionContext) {
380                 transactionContext.deleteData(path);
381             }
382         });
383     }
384
385     @Override
386     public DOMStoreThreePhaseCommitCohort ready() {
387
388         checkModificationState();
389
390         inReadyState = true;
391
392         LOG.debug("Tx {} Readying {} transactions for commit", getIdentifier(),
393                     txFutureCallbackMap.size());
394
395         if(txFutureCallbackMap.size() == 0) {
396             onTransactionReady(Collections.<Future<ActorSelection>>emptyList());
397             TransactionRateLimitingCallback.adjustRateLimitForUnusedTransaction(actorContext);
398             return NoOpDOMStoreThreePhaseCommitCohort.INSTANCE;
399         }
400
401         throttleOperation(txFutureCallbackMap.size());
402
403         List<Future<ActorSelection>> cohortFutures = Lists.newArrayList();
404
405         for(TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
406
407             LOG.debug("Tx {} Readying transaction for shard {} chain {}", getIdentifier(),
408                         txFutureCallback.getShardName(), transactionChainId);
409
410             final TransactionContext transactionContext = txFutureCallback.getTransactionContext();
411             final Future<ActorSelection> future;
412             if (transactionContext != null) {
413                 // avoid the creation of a promise and a TransactionOperation
414                 future = transactionContext.readyTransaction();
415             } else {
416                 final Promise<ActorSelection> promise = akka.dispatch.Futures.promise();
417                 txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
418                     @Override
419                     public void invoke(TransactionContext transactionContext) {
420                         promise.completeWith(transactionContext.readyTransaction());
421                     }
422                 });
423                 future = promise.future();
424             }
425
426             cohortFutures.add(future);
427         }
428
429         onTransactionReady(cohortFutures);
430
431         return new ThreePhaseCommitCohortProxy(actorContext, cohortFutures,
432             getIdentifier().toString());
433     }
434
435     /**
436      * Method for derived classes to be notified when the transaction has been readied.
437      *
438      * @param cohortFutures the cohort Futures for each shard transaction.
439      */
440     protected void onTransactionReady(List<Future<ActorSelection>> cohortFutures) {
441     }
442
443     @Override
444     public void close() {
445         for (TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
446             txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
447                 @Override
448                 public void invoke(TransactionContext transactionContext) {
449                     transactionContext.closeTransaction();
450                 }
451             });
452         }
453
454         txFutureCallbackMap.clear();
455
456         if(remoteTransactionActorsMB != null) {
457             remoteTransactionActors.clear();
458             remoteTransactionActorsMB.set(true);
459         }
460     }
461
462     private String shardNameFromIdentifier(YangInstanceIdentifier path){
463         return ShardStrategyFactory.getStrategy(path).findShard(path);
464     }
465
466     protected Future<ActorSelection> sendFindPrimaryShardAsync(String shardName) {
467         return actorContext.findPrimaryShardAsync(shardName);
468     }
469
470     private TransactionFutureCallback getOrCreateTxFutureCallback(YangInstanceIdentifier path) {
471         String shardName = shardNameFromIdentifier(path);
472         TransactionFutureCallback txFutureCallback = txFutureCallbackMap.get(shardName);
473         if(txFutureCallback == null) {
474             Future<ActorSelection> findPrimaryFuture = sendFindPrimaryShardAsync(shardName);
475
476             final TransactionFutureCallback newTxFutureCallback = new TransactionFutureCallback(shardName);
477
478             txFutureCallback = newTxFutureCallback;
479             txFutureCallbackMap.put(shardName, txFutureCallback);
480
481             findPrimaryFuture.onComplete(new OnComplete<ActorSelection>() {
482                 @Override
483                 public void onComplete(Throwable failure, ActorSelection primaryShard) {
484                     if(failure != null) {
485                         newTxFutureCallback.createTransactionContext(failure, null);
486                     } else {
487                         newTxFutureCallback.setPrimaryShard(primaryShard);
488                     }
489                 }
490             }, actorContext.getClientDispatcher());
491         }
492
493         return txFutureCallback;
494     }
495
496     public String getTransactionChainId() {
497         return transactionChainId;
498     }
499
500     protected ActorContext getActorContext() {
501         return actorContext;
502     }
503
504     /**
505      * Interfaces for transaction operations to be invoked later.
506      */
507     private static interface TransactionOperation {
508         void invoke(TransactionContext transactionContext);
509     }
510
511     /**
512      * Implements a Future OnComplete callback for a CreateTransaction message. This class handles
513      * retries, up to a limit, if the shard doesn't have a leader yet. This is done by scheduling a
514      * retry task after a short delay.
515      * <p>
516      * The end result from a completed CreateTransaction message is a TransactionContext that is
517      * used to perform transaction operations. Transaction operations that occur before the
518      * CreateTransaction completes are cache and executed once the CreateTransaction completes,
519      * successfully or not.
520      */
521     private class TransactionFutureCallback extends OnComplete<Object> {
522
523         /**
524          * The list of transaction operations to execute once the CreateTransaction completes.
525          */
526         @GuardedBy("txOperationsOnComplete")
527         private final List<TransactionOperation> txOperationsOnComplete = Lists.newArrayList();
528
529         /**
530          * The TransactionContext resulting from the CreateTransaction reply.
531          */
532         private volatile TransactionContext transactionContext;
533
534         /**
535          * The target primary shard.
536          */
537         private volatile ActorSelection primaryShard;
538
539         private volatile int createTxTries = (int) (actorContext.getDatastoreContext().
540                 getShardLeaderElectionTimeout().duration().toMillis() /
541                 CREATE_TX_TRY_INTERVAL.toMillis());
542
543         private final String shardName;
544
545         TransactionFutureCallback(String shardName) {
546             this.shardName = shardName;
547         }
548
549         String getShardName() {
550             return shardName;
551         }
552
553         TransactionContext getTransactionContext() {
554             return transactionContext;
555         }
556
557
558         /**
559          * Sets the target primary shard and initiates a CreateTransaction try.
560          */
561         void setPrimaryShard(ActorSelection primaryShard) {
562             this.primaryShard = primaryShard;
563
564             if(transactionType == TransactionType.WRITE_ONLY &&
565                     actorContext.getDatastoreContext().isWriteOnlyTransactionOptimizationsEnabled()) {
566                 LOG.debug("Tx {} Primary shard {} found - creating WRITE_ONLY transaction context",
567                     getIdentifier(), primaryShard);
568
569                 // For write-only Tx's we prepare the transaction modifications directly on the shard actor
570                 // to avoid the overhead of creating a separate transaction actor.
571                 // FIXME: can't assume the shard version is LITHIUM_VERSION - need to obtain it somehow.
572                 executeTxOperatonsOnComplete(createValidTransactionContext(this.primaryShard,
573                         this.primaryShard.path().toString(), DataStoreVersions.LITHIUM_VERSION));
574             } else {
575                 tryCreateTransaction();
576             }
577         }
578
579         /**
580          * Adds a TransactionOperation to be executed after the CreateTransaction completes.
581          */
582         void addTxOperationOnComplete(TransactionOperation operation) {
583             boolean invokeOperation = true;
584             synchronized(txOperationsOnComplete) {
585                 if(transactionContext == null) {
586                     LOG.debug("Tx {} Adding operation on complete", getIdentifier());
587
588                     invokeOperation = false;
589                     txOperationsOnComplete.add(operation);
590                 }
591             }
592
593             if(invokeOperation) {
594                 operation.invoke(transactionContext);
595             }
596         }
597
598         void enqueueTransactionOperation(final TransactionOperation op) {
599
600             if (transactionContext != null) {
601                 op.invoke(transactionContext);
602             } else {
603                 // The shard Tx hasn't been created yet so add the Tx operation to the Tx Future
604                 // callback to be executed after the Tx is created.
605                 addTxOperationOnComplete(op);
606             }
607         }
608
609         /**
610          * Performs a CreateTransaction try async.
611          */
612         private void tryCreateTransaction() {
613             if(LOG.isDebugEnabled()) {
614                 LOG.debug("Tx {} Primary shard {} found - trying create transaction", getIdentifier(), primaryShard);
615             }
616
617             Object serializedCreateMessage = new CreateTransaction(getIdentifier().toString(),
618                     TransactionProxy.this.transactionType.ordinal(),
619                     getTransactionChainId()).toSerializable();
620
621             Future<Object> createTxFuture = actorContext.executeOperationAsync(primaryShard, serializedCreateMessage);
622
623             createTxFuture.onComplete(this, actorContext.getClientDispatcher());
624         }
625
626         @Override
627         public void onComplete(Throwable failure, Object response) {
628             if(failure instanceof NoShardLeaderException) {
629                 // There's no leader for the shard yet - schedule and try again, unless we're out
630                 // of retries. Note: createTxTries is volatile as it may be written by different
631                 // threads however not concurrently, therefore decrementing it non-atomically here
632                 // is ok.
633                 if(--createTxTries > 0) {
634                     LOG.debug("Tx {} Shard {} has no leader yet - scheduling create Tx retry",
635                         getIdentifier(), shardName);
636
637                     actorContext.getActorSystem().scheduler().scheduleOnce(CREATE_TX_TRY_INTERVAL,
638                             new Runnable() {
639                                 @Override
640                                 public void run() {
641                                     tryCreateTransaction();
642                                 }
643                             }, actorContext.getClientDispatcher());
644                     return;
645                 }
646             }
647
648             createTransactionContext(failure, response);
649         }
650
651         private void createTransactionContext(Throwable failure, Object response) {
652             // Mainly checking for state violation here to perform a volatile read of "initialized" to
653             // ensure updates to operationLimter et al are visible to this thread (ie we're doing
654             // "piggy-back" synchronization here).
655             Preconditions.checkState(initialized, "Tx was not propertly initialized.");
656
657             // Create the TransactionContext from the response or failure. Store the new
658             // TransactionContext locally until we've completed invoking the
659             // TransactionOperations. This avoids thread timing issues which could cause
660             // out-of-order TransactionOperations. Eg, on a modification operation, if the
661             // TransactionContext is non-null, then we directly call the TransactionContext.
662             // However, at the same time, the code may be executing the cached
663             // TransactionOperations. So to avoid thus timing, we don't publish the
664             // TransactionContext until after we've executed all cached TransactionOperations.
665             TransactionContext localTransactionContext;
666             if(failure != null) {
667                 LOG.debug("Tx {} Creating NoOpTransaction because of error", getIdentifier(), failure);
668
669                 localTransactionContext = new NoOpTransactionContext(failure, getIdentifier(), operationLimiter);
670             } else if (response.getClass().equals(CreateTransactionReply.SERIALIZABLE_CLASS)) {
671                 localTransactionContext = createValidTransactionContext(
672                         CreateTransactionReply.fromSerializable(response));
673             } else {
674                 IllegalArgumentException exception = new IllegalArgumentException(String.format(
675                         "Invalid reply type %s for CreateTransaction", response.getClass()));
676
677                 localTransactionContext = new NoOpTransactionContext(exception, getIdentifier(), operationLimiter);
678             }
679
680             executeTxOperatonsOnComplete(localTransactionContext);
681         }
682
683         private void executeTxOperatonsOnComplete(TransactionContext localTransactionContext) {
684             while(true) {
685                 // Access to txOperationsOnComplete and transactionContext must be protected and atomic
686                 // (ie synchronized) with respect to #addTxOperationOnComplete to handle timing
687                 // issues and ensure no TransactionOperation is missed and that they are processed
688                 // in the order they occurred.
689
690                 // We'll make a local copy of the txOperationsOnComplete list to handle re-entrancy
691                 // in case a TransactionOperation results in another transaction operation being
692                 // queued (eg a put operation from a client read Future callback that is notified
693                 // synchronously).
694                 Collection<TransactionOperation> operationsBatch = null;
695                 synchronized(txOperationsOnComplete) {
696                     if(txOperationsOnComplete.isEmpty()) {
697                         // We're done invoking the TransactionOperations so we can now publish the
698                         // TransactionContext.
699                         transactionContext = localTransactionContext;
700                         break;
701                     }
702
703                     operationsBatch = new ArrayList<>(txOperationsOnComplete);
704                     txOperationsOnComplete.clear();
705                 }
706
707                 // Invoke TransactionOperations outside the sync block to avoid unnecessary blocking.
708                 // A slight down-side is that we need to re-acquire the lock below but this should
709                 // be negligible.
710                 for(TransactionOperation oper: operationsBatch) {
711                     oper.invoke(localTransactionContext);
712                 }
713             }
714         }
715
716         private TransactionContext createValidTransactionContext(CreateTransactionReply reply) {
717             LOG.debug("Tx {} Received {}", getIdentifier(), reply);
718
719             return createValidTransactionContext(actorContext.actorSelection(reply.getTransactionPath()),
720                     reply.getTransactionPath(), reply.getVersion());
721         }
722
723         private TransactionContext createValidTransactionContext(ActorSelection transactionActor,
724                 String transactionPath, short remoteTransactionVersion) {
725
726             if (transactionType == TransactionType.READ_ONLY) {
727                 // Read-only Tx's aren't explicitly closed by the client so we create a PhantomReference
728                 // to close the remote Tx's when this instance is no longer in use and is garbage
729                 // collected.
730
731                 if(remoteTransactionActorsMB == null) {
732                     remoteTransactionActors = Lists.newArrayList();
733                     remoteTransactionActorsMB = new AtomicBoolean();
734
735                     TransactionProxyCleanupPhantomReference cleanup =
736                             new TransactionProxyCleanupPhantomReference(TransactionProxy.this);
737                     phantomReferenceCache.put(cleanup, cleanup);
738                 }
739
740                 // Add the actor to the remoteTransactionActors list for access by the
741                 // cleanup PhantonReference.
742                 remoteTransactionActors.add(transactionActor);
743
744                 // Write to the memory barrier volatile to publish the above update to the
745                 // remoteTransactionActors list for thread visibility.
746                 remoteTransactionActorsMB.set(true);
747             }
748
749             // TxActor is always created where the leader of the shard is.
750             // Check if TxActor is created in the same node
751             boolean isTxActorLocal = actorContext.isPathLocal(transactionPath);
752
753             if(remoteTransactionVersion < DataStoreVersions.LITHIUM_VERSION) {
754                 return new PreLithiumTransactionContextImpl(transactionPath, transactionActor, getIdentifier(),
755                         transactionChainId, actorContext, schemaContext, isTxActorLocal, remoteTransactionVersion,
756                         operationCompleter);
757             } else if (transactionType == TransactionType.WRITE_ONLY &&
758                     actorContext.getDatastoreContext().isWriteOnlyTransactionOptimizationsEnabled()) {
759                 return new WriteOnlyTransactionContextImpl(transactionActor, getIdentifier(), transactionChainId,
760                     actorContext, schemaContext, isTxActorLocal, remoteTransactionVersion, operationCompleter);
761             } else {
762                 return new TransactionContextImpl(transactionActor, getIdentifier(), transactionChainId,
763                         actorContext, schemaContext, isTxActorLocal, remoteTransactionVersion, operationCompleter);
764             }
765         }
766     }
767
768     private static class NoOpDOMStoreThreePhaseCommitCohort implements DOMStoreThreePhaseCommitCohort {
769         static NoOpDOMStoreThreePhaseCommitCohort INSTANCE = new NoOpDOMStoreThreePhaseCommitCohort();
770
771         private static final ListenableFuture<Void> IMMEDIATE_VOID_SUCCESS =
772                 com.google.common.util.concurrent.Futures.immediateFuture(null);
773         private static final ListenableFuture<Boolean> IMMEDIATE_BOOLEAN_SUCCESS =
774                 com.google.common.util.concurrent.Futures.immediateFuture(Boolean.TRUE);
775
776         private NoOpDOMStoreThreePhaseCommitCohort() {
777         }
778
779         @Override
780         public ListenableFuture<Boolean> canCommit() {
781             return IMMEDIATE_BOOLEAN_SUCCESS;
782         }
783
784         @Override
785         public ListenableFuture<Void> preCommit() {
786             return IMMEDIATE_VOID_SUCCESS;
787         }
788
789         @Override
790         public ListenableFuture<Void> abort() {
791             return IMMEDIATE_VOID_SUCCESS;
792         }
793
794         @Override
795         public ListenableFuture<Void> commit() {
796             return IMMEDIATE_VOID_SUCCESS;
797         }
798     }
799 }