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