Encapsulate ShardedDOMDataTreeProducer layout
[mdsal.git] / dom / mdsal-dom-broker / src / main / java / org / opendaylight / mdsal / dom / broker / ShardedDOMDataTreeProducer.java
index 837871c03156addd19e4d5dac364fe38ff794ac8..b7c2e58e523a653a1135bf5be87ceb9f36b44c16 100644 (file)
@@ -9,18 +9,11 @@ package org.opendaylight.mdsal.dom.broker;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Verify;
-import com.google.common.collect.ArrayListMultimap;
-import com.google.common.collect.BiMap;
-import com.google.common.collect.ImmutableBiMap;
-import com.google.common.collect.ImmutableBiMap.Builder;
-import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Multimap;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.Map;
-import java.util.Map.Entry;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 import javax.annotation.concurrent.GuardedBy;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeCursorAwareTransaction;
@@ -29,8 +22,6 @@ import org.opendaylight.mdsal.dom.api.DOMDataTreeProducer;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeProducerBusyException;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeProducerException;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeShard;
-import org.opendaylight.mdsal.dom.store.inmemory.DOMDataTreeShardProducer;
-import org.opendaylight.mdsal.dom.store.inmemory.WriteableDOMDataTreeShard;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -40,13 +31,9 @@ class ShardedDOMDataTreeProducer implements DOMDataTreeProducer {
     private final Set<DOMDataTreeIdentifier> subtrees;
     private final ShardedDOMDataTree dataTree;
 
-    private BiMap<DOMDataTreeIdentifier, DOMDataTreeShardProducer> idToProducer = ImmutableBiMap.of();
-    private Map<DOMDataTreeIdentifier, DOMDataTreeShard> idToShard;
-
     private static final AtomicReferenceFieldUpdater<ShardedDOMDataTreeProducer, ShardedDOMDataTreeWriteTransaction>
         CURRENT_UPDATER = AtomicReferenceFieldUpdater.newUpdater(ShardedDOMDataTreeProducer.class,
             ShardedDOMDataTreeWriteTransaction.class, "currentTx");
-    @SuppressWarnings("unused")
     private volatile ShardedDOMDataTreeWriteTransaction currentTx;
 
     private static final AtomicReferenceFieldUpdater<ShardedDOMDataTreeProducer, ShardedDOMDataTreeWriteTransaction>
@@ -59,177 +46,153 @@ class ShardedDOMDataTreeProducer implements DOMDataTreeProducer {
             ShardedDOMDataTreeWriteTransaction.class, "lastTx");
     private volatile ShardedDOMDataTreeWriteTransaction lastTx;
 
-    @GuardedBy("this")
-    private Map<DOMDataTreeIdentifier, DOMDataTreeProducer> children = Collections.emptyMap();
-    @GuardedBy("this")
-    private boolean closed;
+    private static final AtomicIntegerFieldUpdater<ShardedDOMDataTreeProducer> CLOSED_UPDATER =
+            AtomicIntegerFieldUpdater.newUpdater(ShardedDOMDataTreeProducer.class, "closed");
+    private volatile int closed;
 
-    @GuardedBy("this")
-    private ShardedDOMDataTreeListenerContext<?> attachedListener;
+    private volatile ShardedDOMDataTreeListenerContext<?> attachedListener;
+    private volatile ProducerLayout layout;
 
-    ShardedDOMDataTreeProducer(final ShardedDOMDataTree dataTree,
+    private ShardedDOMDataTreeProducer(final ShardedDOMDataTree dataTree,
                                final Collection<DOMDataTreeIdentifier> subtrees,
-                               final Map<DOMDataTreeIdentifier, DOMDataTreeShard> shardMap,
-                               final Multimap<DOMDataTreeShard, DOMDataTreeIdentifier> shardToId) {
+                               final Map<DOMDataTreeIdentifier, DOMDataTreeShard> shardMap) {
         this.dataTree = Preconditions.checkNotNull(dataTree);
-        if (!shardToId.isEmpty()) {
-            this.idToProducer = mapIdsToProducer(shardToId);
-        }
-        idToShard = ImmutableMap.copyOf(shardMap);
         this.subtrees = ImmutableSet.copyOf(subtrees);
+        this.layout = ProducerLayout.create(shardMap);
     }
 
     static DOMDataTreeProducer create(final ShardedDOMDataTree dataTree,
                                       final Collection<DOMDataTreeIdentifier> subtrees,
                                       final Map<DOMDataTreeIdentifier, DOMDataTreeShard> shardMap) {
-        final Multimap<DOMDataTreeShard, DOMDataTreeIdentifier> shardToIdentifiers = ArrayListMultimap.create();
-        // map which identifier belongs to which shard
-        for (final Entry<DOMDataTreeIdentifier, DOMDataTreeShard> entry : shardMap.entrySet()) {
-            shardToIdentifiers.put(entry.getValue(), entry.getKey());
-        }
+        return new ShardedDOMDataTreeProducer(dataTree, subtrees, shardMap);
+    }
 
-        return new ShardedDOMDataTreeProducer(dataTree, subtrees, shardMap, shardToIdentifiers);
+    private void checkNotClosed() {
+        Preconditions.checkState(closed == 0, "Producer is already closed");
     }
 
-    void subshardAdded(final Map<DOMDataTreeIdentifier, DOMDataTreeShard> shardMap) {
+    private void checkIdle() {
         Preconditions.checkState(openTx == null, "Transaction %s is still open", openTx);
-        final Multimap<DOMDataTreeShard, DOMDataTreeIdentifier> shardToIdentifiers = ArrayListMultimap.create();
-        // map which identifier belongs to which shard
-        for (final Entry<DOMDataTreeIdentifier, DOMDataTreeShard> entry : shardMap.entrySet()) {
-            shardToIdentifiers.put(entry.getValue(), entry.getKey());
-        }
-        this.idToProducer = mapIdsToProducer(shardToIdentifiers);
-        idToShard = ImmutableMap.copyOf(shardMap);
     }
 
-    private static BiMap<DOMDataTreeIdentifier, DOMDataTreeShardProducer> mapIdsToProducer(
-            final Multimap<DOMDataTreeShard, DOMDataTreeIdentifier> shardToId) {
-        final Builder<DOMDataTreeIdentifier, DOMDataTreeShardProducer> idToProducerBuilder = ImmutableBiMap.builder();
-        for (final Entry<DOMDataTreeShard, Collection<DOMDataTreeIdentifier>> entry : shardToId.asMap().entrySet()) {
-            if (entry.getKey() instanceof WriteableDOMDataTreeShard) {
-                //create a single producer for all prefixes in a single shard
-                final DOMDataTreeShardProducer producer = ((WriteableDOMDataTreeShard) entry.getKey())
-                        .createProducer(entry.getValue());
-                // id mapped to producers
-                for (final DOMDataTreeIdentifier id : entry.getValue()) {
-                    idToProducerBuilder.put(id, producer);
-                }
-            } else {
-                LOG.error("Unable to create a producer for shard that's not a WriteableDOMDataTreeShard");
-            }
-        }
+    void subshardAdded(final Map<DOMDataTreeIdentifier, DOMDataTreeShard> shardMap) {
+        checkIdle();
 
-        return idToProducerBuilder.build();
+        layout = layout.reshard(shardMap);
     }
 
     @Override
-    public synchronized DOMDataTreeCursorAwareTransaction createTransaction(final boolean isolated) {
-        Preconditions.checkState(!closed, "Producer is already closed");
-        Preconditions.checkState(openTx == null, "Transaction %s is still open", openTx);
+    public DOMDataTreeCursorAwareTransaction createTransaction(final boolean isolated) {
+        checkNotClosed();
+        checkIdle();
+
+        LOG.debug("Creating transaction from producer {}", this);
 
-        LOG.debug("Creating transaction from producer");
         final ShardedDOMDataTreeWriteTransaction current = CURRENT_UPDATER.getAndSet(this, null);
         final ShardedDOMDataTreeWriteTransaction ret;
         if (isolated) {
-            // Isolated case. If we have a previous transaction, submit it before returning this one.
-            if (current != null) {
-                submitTransaction(current);
-            }
-            ret = new ShardedDOMDataTreeWriteTransaction(this, idToProducer, children, true);
+            ret = createIsolatedTransaction(layout, current);
         } else {
-            // Non-isolated case, see if we can reuse the transaction
-            if (current != null) {
-                LOG.debug("Reusing previous transaction {} since there is still a transaction inflight",
-                        current.getIdentifier());
-                ret = current;
-            } else {
-                ret = new ShardedDOMDataTreeWriteTransaction(this, idToProducer, children, false);
-            }
+            ret = createReusedTransaction(layout, current);
         }
 
         final boolean success = OPEN_UPDATER.compareAndSet(this, null, ret);
-        Verify.verify(success);
+        Preconditions.checkState(success, "Illegal concurrent access to producer %s detected", this);
         return ret;
     }
 
-    private void submitTransaction(final ShardedDOMDataTreeWriteTransaction current) {
-        lastTx = current;
-        current.doSubmit(this::transactionSuccessful, this::transactionFailed);
+    // This may look weird, but this has side-effects on local's producers, hence it needs to be properly synchronized
+    // so that it happens-after submitTransaction() which may have been stolen by a callback.
+    @GuardedBy("this")
+    private ShardedDOMDataTreeWriteTransaction createTransaction(final ProducerLayout local) {
+        return new ShardedDOMDataTreeWriteTransaction(this, local.createTransactions(), local);
+
     }
 
-    @GuardedBy("this")
-    private boolean haveSubtree(final DOMDataTreeIdentifier subtree) {
-        for (final DOMDataTreeIdentifier i : idToShard.keySet()) {
-            if (i.contains(subtree)) {
-                return true;
-            }
+    // Isolated case. If we have a previous transaction, submit it before returning this one.
+    private synchronized ShardedDOMDataTreeWriteTransaction createIsolatedTransaction(
+            final ProducerLayout local, final ShardedDOMDataTreeWriteTransaction current) {
+        if (current != null) {
+            submitTransaction(current);
         }
 
-        return false;
+        return createTransaction(local);
     }
 
-    @GuardedBy("this")
-    private DOMDataTreeProducer lookupChild(final DOMDataTreeIdentifier domDataTreeIdentifier) {
-        for (final Entry<DOMDataTreeIdentifier, DOMDataTreeProducer> e : children.entrySet()) {
-            if (e.getKey().contains(domDataTreeIdentifier)) {
-                return e.getValue();
+    private ShardedDOMDataTreeWriteTransaction createReusedTransaction(final ProducerLayout local,
+            final ShardedDOMDataTreeWriteTransaction current) {
+        if (current != null) {
+            // Lock-free fast path
+            if (local.equals(current.getLayout())) {
+                LOG.debug("Reusing previous transaction {} since there is still a transaction inflight",
+                    current.getIdentifier());
+                return current;
+            }
+
+            synchronized (this) {
+                submitTransaction(current);
+                return createTransaction(local);
             }
         }
 
-        return null;
+        // Null indicates we have not seen a previous transaction -- which does not mean it is ready, as it may have
+        // been stolen and in is process of being submitted.
+        synchronized (this) {
+            return createTransaction(local);
+        }
+    }
+
+    @GuardedBy("this")
+    private void submitTransaction(final ShardedDOMDataTreeWriteTransaction tx) {
+        lastTx = tx;
+        tx.doSubmit(this::transactionSuccessful, this::transactionFailed);
     }
 
     @Override
-    public synchronized DOMDataTreeProducer createProducer(final Collection<DOMDataTreeIdentifier> subtrees) {
-        Preconditions.checkState(!closed, "Producer is already closed");
-        Preconditions.checkState(openTx == null, "Transaction %s is still open", openTx);
+    public DOMDataTreeProducer createProducer(final Collection<DOMDataTreeIdentifier> subtrees) {
+        checkNotClosed();
+        checkIdle();
+
+        final ProducerLayout local = layout;
 
         for (final DOMDataTreeIdentifier s : subtrees) {
             // Check if the subtree was visible at any time
-            Preconditions.checkArgument(haveSubtree(s), "Subtree %s was never available in producer %s", s, this);
+            Preconditions.checkArgument(local.haveSubtree(s), "Subtree %s was never available in producer %s", s, this);
             // Check if the subtree has not been delegated to a child
-            final DOMDataTreeProducer child = lookupChild(s);
+            final DOMDataTreeProducer child = local.lookupChild(s);
             Preconditions.checkArgument(child == null, "Subtree %s is delegated to child producer %s", s, child);
 
             // Check if part of the requested subtree is not delegated to a child.
-            for (final DOMDataTreeIdentifier c : children.keySet()) {
-                if (s.contains(c)) {
-                    throw new IllegalArgumentException(String.format("Subtree %s cannot be delegated as it is"
-                            + " superset of already-delegated %s", s, c));
-                }
+            for (final DOMDataTreeIdentifier c : local.getChildTrees()) {
+                Preconditions.checkArgument(!s.contains(c),
+                    "Subtree %s cannot be delegated as it is a superset of already-delegated %s", s, c);
             }
         }
 
-        final DOMDataTreeProducer ret = dataTree.createProducer(this, subtrees);
-        final ImmutableMap.Builder<DOMDataTreeIdentifier, DOMDataTreeProducer> cb = ImmutableMap.builder();
-        cb.putAll(children);
-        for (final DOMDataTreeIdentifier s : subtrees) {
-            cb.put(s, ret);
+
+        final DOMDataTreeProducer ret;
+        synchronized (this) {
+            ret = dataTree.createProducer(this, subtrees);
         }
 
-        children = cb.build();
+        layout = local.addChild(ret, subtrees);
         return ret;
     }
 
     boolean isDelegatedToChild(final DOMDataTreeIdentifier path) {
-        for (final DOMDataTreeIdentifier c : children.keySet()) {
-            if (c.contains(path)) {
-                return true;
-            }
-        }
-        return false;
+        return layout.lookupChild(path) != null;
     }
 
-
     @Override
-    public synchronized void close() throws DOMDataTreeProducerException {
-        if (!closed) {
-            if (openTx != null) {
-                throw new DOMDataTreeProducerBusyException(String.format("Transaction %s is still open", openTx));
-            }
+    public void close() throws DOMDataTreeProducerException {
+        if (openTx != null) {
+            throw new DOMDataTreeProducerBusyException(String.format("Transaction %s is still open", openTx));
+        }
 
-            closed = true;
-            dataTree.destroyProducer(this);
+        if (CLOSED_UPDATER.compareAndSet(this, 0, 1)) {
+            synchronized (this) {
+                dataTree.destroyProducer(this);
+            }
         }
     }
 
@@ -246,54 +209,74 @@ class ShardedDOMDataTreeProducer implements DOMDataTreeProducer {
         }
     }
 
-    void processTransaction(final ShardedDOMDataTreeWriteTransaction transaction) {
+    // Called when the user submits a transaction
+    void transactionSubmitted(final ShardedDOMDataTreeWriteTransaction transaction) {
         final boolean wasOpen = OPEN_UPDATER.compareAndSet(this, transaction, null);
-        Verify.verify(wasOpen);
-
-        if (lastTx != null) {
-            final boolean success = CURRENT_UPDATER.compareAndSet(this, null, transaction);
-            Verify.verify(success);
-            if (lastTx == null) {
-                // Dispatch after requeue
-                processCurrentTransaction();
+        Preconditions.checkState(wasOpen, "Attempted to submit non-open transaction %s", transaction);
+
+        if (lastTx == null) {
+            // No transaction outstanding, we need to submit it now
+            synchronized (this) {
+                submitTransaction(transaction);
             }
-        } else {
-            submitTransaction(transaction);
+
+            return;
+        }
+
+        // There is a potentially-racing submitted transaction. Publish the new one, which may end up being
+        // picked up by processNextTransaction.
+        final boolean success = CURRENT_UPDATER.compareAndSet(this, null, transaction);
+        Verify.verify(success);
+
+        // Now a quick check: if the racing transaction completed in between, it may have missed the current
+        // transaction, hence we need to re-check
+        if (lastTx == null) {
+            submitCurrentTransaction();
         }
     }
 
-    void transactionSuccessful(final ShardedDOMDataTreeWriteTransaction tx, final Void result) {
+    private void submitCurrentTransaction() {
+        final ShardedDOMDataTreeWriteTransaction current = currentTx;
+        if (current != null) {
+            synchronized (this) {
+                if (CURRENT_UPDATER.compareAndSet(this, current, null)) {
+                    submitTransaction(current);
+                }
+            }
+        }
+    }
+
+    private void transactionSuccessful(final ShardedDOMDataTreeWriteTransaction tx) {
         LOG.debug("Transaction {} completed successfully", tx.getIdentifier());
 
-        tx.onTransactionSuccess(result);
-        processNextTransaction(tx);
+        tx.onTransactionSuccess(null);
+        transactionCompleted(tx);
     }
 
-    void transactionFailed(final ShardedDOMDataTreeWriteTransaction tx, final Throwable throwable) {
+    private void transactionFailed(final ShardedDOMDataTreeWriteTransaction tx, final Throwable throwable) {
         LOG.debug("Transaction {} failed", tx.getIdentifier(), throwable);
 
         tx.onTransactionFailure(throwable);
-        processNextTransaction(tx);
+        // FIXME: transaction failure should result in a hard error
+        transactionCompleted(tx);
     }
 
-    private void processCurrentTransaction() {
-        final ShardedDOMDataTreeWriteTransaction current = CURRENT_UPDATER.getAndSet(this, null);
-        if (current != null) {
-            submitTransaction(current);
-        }
-    }
-
-    private void processNextTransaction(final ShardedDOMDataTreeWriteTransaction tx) {
+    private void transactionCompleted(final ShardedDOMDataTreeWriteTransaction tx) {
         final boolean wasLast = LAST_UPDATER.compareAndSet(this, tx, null);
         if (wasLast) {
-            processCurrentTransaction();
+            submitCurrentTransaction();
         }
     }
 
-    synchronized void boundToListener(final ShardedDOMDataTreeListenerContext<?> listener) {
-        // FIXME: Add option to detach
-        Preconditions.checkState(this.attachedListener == null, "Producer %s is already attached to other listener.",
-                listener.getListener());
+    void bindToListener(final ShardedDOMDataTreeListenerContext<?> listener) {
+        Preconditions.checkNotNull(listener);
+
+        final ShardedDOMDataTreeListenerContext<?> local = attachedListener;
+        if (local != null) {
+            throw new IllegalStateException(String.format("Producer %s is already attached to listener %s", this,
+                local.getListener()));
+        }
+
         this.attachedListener = listener;
     }
 }