X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-distributed-datastore%2Fsrc%2Fmain%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fdatabroker%2Factors%2Fdds%2FProxyHistory.java;h=70b5960a056d943ef8f927094e3b433fb85f3021;hb=refs%2Fchanges%2F49%2F85749%2F63;hp=e26e00fa13a98c42d4236cce29050a55f1f020a9;hpb=c09801280b4c44f4ec26766e4d13b1a5d1f3ed59;p=controller.git diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/databroker/actors/dds/ProxyHistory.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/databroker/actors/dds/ProxyHistory.java index e26e00fa13..70b5960a05 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/databroker/actors/dds/ProxyHistory.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/databroker/actors/dds/ProxyHistory.java @@ -7,20 +7,29 @@ */ package org.opendaylight.controller.cluster.databroker.actors.dds; +import static com.google.common.base.Preconditions.checkState; +import static com.google.common.base.Verify.verify; +import static com.google.common.base.Verify.verifyNotNull; +import static java.util.Objects.requireNonNull; + import akka.actor.ActorRef; -import com.google.common.base.Preconditions; -import com.google.common.base.Verify; +import com.google.common.collect.ImmutableList; +import com.google.common.primitives.UnsignedLong; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.util.ArrayList; import java.util.Collection; import java.util.Iterator; import java.util.LinkedHashMap; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.function.Consumer; -import javax.annotation.concurrent.GuardedBy; +import org.checkerframework.checker.lock.qual.GuardedBy; +import org.checkerframework.checker.lock.qual.Holding; +import org.eclipse.jdt.annotation.NonNull; import org.opendaylight.controller.cluster.access.client.AbstractClientConnection; import org.opendaylight.controller.cluster.access.client.ClientActorContext; import org.opendaylight.controller.cluster.access.client.ConnectedClientConnection; @@ -29,6 +38,7 @@ import org.opendaylight.controller.cluster.access.commands.CreateLocalHistoryReq import org.opendaylight.controller.cluster.access.commands.DestroyLocalHistoryRequest; import org.opendaylight.controller.cluster.access.commands.LocalHistoryRequest; import org.opendaylight.controller.cluster.access.commands.PurgeLocalHistoryRequest; +import org.opendaylight.controller.cluster.access.commands.SkipTransactionsRequest; import org.opendaylight.controller.cluster.access.commands.TransactionRequest; import org.opendaylight.controller.cluster.access.concepts.LocalHistoryIdentifier; import org.opendaylight.controller.cluster.access.concepts.Request; @@ -36,8 +46,8 @@ import org.opendaylight.controller.cluster.access.concepts.RequestException; import org.opendaylight.controller.cluster.access.concepts.Response; import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier; import org.opendaylight.yangtools.concepts.Identifiable; -import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTree; import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeSnapshot; +import org.opendaylight.yangtools.yang.data.api.schema.tree.ReadOnlyDataTree; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,12 +58,12 @@ import org.slf4j.LoggerFactory; */ abstract class ProxyHistory implements Identifiable { private abstract static class AbstractLocal extends ProxyHistory { - private final DataTree dataTree; + private final ReadOnlyDataTree dataTree; AbstractLocal(final AbstractClientHistory parent, final AbstractClientConnection connection, - final LocalHistoryIdentifier identifier, final DataTree dataTree) { + final LocalHistoryIdentifier identifier, final ReadOnlyDataTree dataTree) { super(parent, connection, identifier); - this.dataTree = Preconditions.checkNotNull(dataTree); + this.dataTree = requireNonNull(dataTree); } final DataTreeSnapshot takeSnapshot() { @@ -79,14 +89,14 @@ abstract class ProxyHistory implements Identifiable { private volatile LocalReadWriteProxyTransaction lastSealed; Local(final AbstractClientHistory parent, final AbstractClientConnection connection, - final LocalHistoryIdentifier identifier, final DataTree dataTree) { + final LocalHistoryIdentifier identifier, final ReadOnlyDataTree dataTree) { super(parent, connection, identifier, dataTree); } @Override AbstractProxyTransaction doCreateTransactionProxy(final AbstractClientConnection connection, final TransactionIdentifier txId, final boolean snapshotOnly, final boolean isDone) { - Preconditions.checkState(lastOpen == null, "Proxy %s has %s currently open", this, lastOpen); + checkState(lastOpen == null, "Proxy %s has %s currently open", this, lastOpen); if (isDone) { // Done transactions do not register on our radar on should not have any state associated. @@ -126,17 +136,16 @@ abstract class ProxyHistory implements Identifiable { @Override void onTransactionCompleted(final AbstractProxyTransaction tx) { - Verify.verify(tx instanceof LocalProxyTransaction); - if (tx instanceof LocalReadWriteProxyTransaction) { - if (LAST_SEALED_UPDATER.compareAndSet(this, (LocalReadWriteProxyTransaction) tx, null)) { - LOG.debug("Completed last sealed transaction {}", tx); - } + verify(tx instanceof LocalProxyTransaction, "Unexpected transaction %s", tx); + if (tx instanceof LocalReadWriteProxyTransaction + && LAST_SEALED_UPDATER.compareAndSet(this, (LocalReadWriteProxyTransaction) tx, null)) { + LOG.debug("Completed last sealed transaction {}", tx); } } @Override void onTransactionSealed(final AbstractProxyTransaction tx) { - Preconditions.checkState(tx.equals(lastOpen)); + checkState(tx.equals(lastOpen)); lastSealed = lastOpen; lastOpen = null; } @@ -144,7 +153,7 @@ abstract class ProxyHistory implements Identifiable { private static final class LocalSingle extends AbstractLocal { LocalSingle(final AbstractClientHistory parent, final AbstractClientConnection connection, - final LocalHistoryIdentifier identifier, final DataTree dataTree) { + final LocalHistoryIdentifier identifier, final ReadOnlyDataTree dataTree) { super(parent, connection, identifier, dataTree); } @@ -217,7 +226,7 @@ abstract class ProxyHistory implements Identifiable { return identifier; } - @GuardedBy("lock") + @Holding("lock") @Override void replayRequests(final Collection previousEntries) { // First look for our Create message @@ -226,7 +235,7 @@ abstract class ProxyHistory implements Identifiable { final ConnectionEntry e = it.next(); final Request req = e.getRequest(); if (identifier.equals(req.getTarget())) { - Verify.verify(req instanceof LocalHistoryRequest); + verify(req instanceof LocalHistoryRequest, "Unexpected request %s", req); if (req instanceof CreateLocalHistoryRequest) { successor.connection.enqueueRequest(req, e.getCallback(), e.getEnqueuedTicks()); it.remove(); @@ -240,13 +249,21 @@ abstract class ProxyHistory implements Identifiable { t.replayMessages(successor, previousEntries); } + // Forward any skipped transactions + final var local = skippedTransactions; + if (local != null) { + LOG.debug("{} forwarding skipped transactions towards successor {}", identifier, successor); + successor.skipTransactions(local); + skippedTransactions = null; + } + // Now look for any finalizing messages it = previousEntries.iterator(); while (it.hasNext()) { final ConnectionEntry e = it.next(); final Request req = e.getRequest(); if (identifier.equals(req.getTarget())) { - Verify.verify(req instanceof LocalHistoryRequest); + verify(req instanceof LocalHistoryRequest, "Unexpected request %s", req); if (req instanceof DestroyLocalHistoryRequest) { successor.connection.enqueueRequest(req, e.getCallback(), e.getEnqueuedTicks()); it.remove(); @@ -256,10 +273,10 @@ abstract class ProxyHistory implements Identifiable { } } - @GuardedBy("lock") + @Holding("lock") @Override ProxyHistory finishReconnect() { - final ProxyHistory ret = Verify.verifyNotNull(successor); + final ProxyHistory ret = verifyNotNull(successor); for (AbstractProxyTransaction t : proxies.values()) { t.finishReconnect(); @@ -317,25 +334,45 @@ abstract class ProxyHistory implements Identifiable { private static final Logger LOG = LoggerFactory.getLogger(ProxyHistory.class); private final Lock lock = new ReentrantLock(); - private final LocalHistoryIdentifier identifier; - private final AbstractClientConnection connection; - private final AbstractClientHistory parent; + private final @NonNull LocalHistoryIdentifier identifier; + private final @NonNull AbstractClientConnection connection; + private final @NonNull AbstractClientHistory parent; @GuardedBy("lock") private final Map proxies = new LinkedHashMap<>(); @GuardedBy("lock") private ProxyHistory successor; + // List of transaction identifiers which were allocated by our parent history, but did not touch our shard. Each of + // these represents a hole in otherwise-contiguous allocation of transactionIds. These holes are problematic, as + // each of them prevents LeaderFrontendState.purgedHistories from coalescing, leading to a gradual heap exhaustion. + // + //

+ // We keep these in an ArrayList for fast insertion, as that happens when we are otherwise idle. We translate these + // into purge requests when: + // - we are about to allocate a new transaction + // - we get a successor proxy + // - the list grows unreasonably long + // + // TODO: we are tracking entire TransactionIdentifiers, but really only need to track the longs. Do that once we + // have a {@code List}. + // FIXME: this is not tuneable, but perhaps should be + // FIXME: default value deserves some explanation -- this affects depth of an RB Tree on the receiving end. + private static final int PURGE_SKIPPED_TXID_THRESHOLD = 256; + + @GuardedBy("lock") + private volatile List skippedTransactions; + private ProxyHistory(final AbstractClientHistory parent, final AbstractClientConnection connection, final LocalHistoryIdentifier identifier) { - this.parent = Preconditions.checkNotNull(parent); - this.connection = Preconditions.checkNotNull(connection); - this.identifier = Preconditions.checkNotNull(identifier); + this.parent = requireNonNull(parent); + this.connection = requireNonNull(connection); + this.identifier = requireNonNull(identifier); } static ProxyHistory createClient(final AbstractClientHistory parent, final AbstractClientConnection connection, final LocalHistoryIdentifier identifier) { - final Optional dataTree = connection.getBackendInfo().flatMap(ShardBackendInfo::getDataTree); + final Optional dataTree = connection.getBackendInfo().flatMap(ShardBackendInfo::getDataTree); return dataTree.isPresent() ? new Local(parent, connection, identifier, dataTree.get()) : new Remote(parent, connection, identifier); } @@ -343,12 +380,13 @@ abstract class ProxyHistory implements Identifiable { static ProxyHistory createSingle(final AbstractClientHistory parent, final AbstractClientConnection connection, final LocalHistoryIdentifier identifier) { - final Optional dataTree = connection.getBackendInfo().flatMap(ShardBackendInfo::getDataTree); + final Optional dataTree = connection.getBackendInfo().flatMap(ShardBackendInfo::getDataTree); return dataTree.isPresent() ? new LocalSingle(parent, connection, identifier, dataTree.get()) : new RemoteSingle(parent, connection, identifier); } @Override + // Non-final for mocking public LocalHistoryIdentifier getIdentifier() { return identifier; } @@ -374,6 +412,7 @@ abstract class ProxyHistory implements Identifiable { return createTransactionProxy(txId, snapshotOnly, false); } + // Non-final for mocking AbstractProxyTransaction createTransactionProxy(final TransactionIdentifier txId, final boolean snapshotOnly, final boolean isDone) { lock.lock(); @@ -392,6 +431,86 @@ abstract class ProxyHistory implements Identifiable { } } + final void skipTransaction(final TransactionIdentifier txId) { + lock.lock(); + try { + if (successor != null) { + successor.skipTransaction(txId); + return; + } + + var local = skippedTransactions; + if (local == null) { + skippedTransactions = local = new ArrayList<>(); + } + local.add(txId); + LOG.debug("Recorded skipped transaction {}", txId); + skipIfNeeded(local); + } finally { + lock.unlock(); + } + } + + @Holding("lock") + private void skipIfNeeded(final List current) { + if (current.size() >= PURGE_SKIPPED_TXID_THRESHOLD) { + skippedTransactions = null; + doSkipTransactions(current); + } + } + + private void skipTransactions(final List toSkip) { + lock.lock(); + try { + if (successor != null) { + successor.skipTransactions(toSkip); + return; + } + + var local = skippedTransactions; + if (local != null) { + local.addAll(toSkip); + } else { + skippedTransactions = local = toSkip; + } + skipIfNeeded(local); + } finally { + lock.unlock(); + } + } + + private void skipTransactions() { + var local = skippedTransactions; + if (local != null) { + lock.lock(); + try { + local = skippedTransactions; + if (local != null && successor == null) { + skippedTransactions = null; + doSkipTransactions(local); + } + } finally { + lock.unlock(); + } + } + } + + @Holding("lock") + private void doSkipTransactions(final List toSkip) { + final var txIds = toSkip.stream() + .mapToLong(TransactionIdentifier::getTransactionId) + .distinct() + .sorted() + .mapToObj(UnsignedLong::fromLongBits) + .collect(ImmutableList.toImmutableList()); + + LOG.debug("Proxy {} skipping transactions {}", this, txIds); + connection.enqueueRequest(new SkipTransactionsRequest(new TransactionIdentifier(identifier, + txIds.get(0).longValue()), 0, localActor(),txIds.subList(1, txIds.size())), resp -> { + LOG.debug("Proxy {} confirmed transaction skip", this); + }, connection.currentTime()); + } + final void abortTransaction(final AbstractProxyTransaction tx) { lock.lock(); try { @@ -414,7 +533,7 @@ abstract class ProxyHistory implements Identifiable { } } - void purgeTransaction(final AbstractProxyTransaction tx) { + final void purgeTransaction(final AbstractProxyTransaction tx) { lock.lock(); try { proxies.remove(tx.getIdentifier()); @@ -442,21 +561,26 @@ abstract class ProxyHistory implements Identifiable { final void enqueueRequest(final TransactionRequest request, final Consumer> callback, final long enqueuedTicks) { + skipTransactions(); connection.enqueueRequest(request, callback, enqueuedTicks); } final void sendRequest(final TransactionRequest request, final Consumer> callback) { + skipTransactions(); connection.sendRequest(request, callback); } - @GuardedBy("lock") + @Holding("lock") + @SuppressWarnings("checkstyle:hiddenField") abstract AbstractProxyTransaction doCreateTransactionProxy(AbstractClientConnection connection, TransactionIdentifier txId, boolean snapshotOnly, boolean isDone); + @Holding("lock") + @SuppressWarnings("checkstyle:hiddenField") abstract ProxyHistory createSuccessor(AbstractClientConnection connection); @SuppressFBWarnings(value = "UL_UNRELEASED_LOCK", justification = "Lock is released asynchronously via the cohort") - ProxyReconnectCohort startReconnect(final ConnectedClientConnection newConnection) { + final ProxyReconnectCohort startReconnect(final ConnectedClientConnection newConnection) { lock.lock(); if (successor != null) { lock.unlock(); @@ -490,16 +614,17 @@ abstract class ProxyHistory implements Identifiable { LOG.debug("Proxy {} purge completed with {}", this, response); } - @GuardedBy("lock") + @Holding("lock") void onTransactionAborted(final AbstractProxyTransaction tx) { // No-op for most implementations } - @GuardedBy("lock") + @Holding("lock") void onTransactionCompleted(final AbstractProxyTransaction tx) { // No-op for most implementations } + @Holding("lock") void onTransactionSealed(final AbstractProxyTransaction tx) { // No-op on most implementations }