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