Slice front-end request messages
[controller.git] / opendaylight / md-sal / cds-access-client / src / main / java / org / opendaylight / controller / cluster / access / client / ClientActorBehavior.java
index 554ffe97c77ae7c3011daca2a28c22f660a20e78..3f6515cbb8b20ce22183b857eccacd6acb50223c 100644 (file)
@@ -13,6 +13,7 @@ import com.google.common.base.Stopwatch;
 import com.google.common.base.Verify;
 import java.util.Collection;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
@@ -31,8 +32,11 @@ import org.opendaylight.controller.cluster.access.concepts.RetiredGenerationExce
 import org.opendaylight.controller.cluster.access.concepts.RuntimeRequestException;
 import org.opendaylight.controller.cluster.access.concepts.SuccessEnvelope;
 import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
+import org.opendaylight.controller.cluster.common.actor.Dispatchers.DispatcherType;
+import org.opendaylight.controller.cluster.io.FileBackedOutputStreamFactory;
+import org.opendaylight.controller.cluster.messaging.MessageAssembler;
 import org.opendaylight.yangtools.concepts.Identifiable;
-import org.opendaylight.yangtools.concepts.WritableIdentifier;
+import org.opendaylight.yangtools.concepts.Identifier;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import scala.concurrent.duration.FiniteDuration;
@@ -60,7 +64,7 @@ public abstract class ClientActorBehavior<T extends BackendInfo> extends
     }
 
     private static final Logger LOG = LoggerFactory.getLogger(ClientActorBehavior.class);
-    private static final FiniteDuration RESOLVE_RETRY_DURATION = FiniteDuration.apply(5, TimeUnit.SECONDS);
+    private static final FiniteDuration RESOLVE_RETRY_DURATION = FiniteDuration.apply(1, TimeUnit.SECONDS);
 
     /**
      * Map of connections to the backend. This map is concurrent to allow lookups, but given complex operations
@@ -78,11 +82,18 @@ public abstract class ClientActorBehavior<T extends BackendInfo> extends
     private final Map<Long, AbstractClientConnection<T>> connections = new ConcurrentHashMap<>();
     private final InversibleLock connectionsLock = new InversibleLock();
     private final BackendInfoResolver<T> resolver;
+    private final MessageAssembler responseMessageAssembler;
 
     protected ClientActorBehavior(@Nonnull final ClientActorContext context,
             @Nonnull final BackendInfoResolver<T> resolver) {
         super(context);
         this.resolver = Preconditions.checkNotNull(resolver);
+
+        final ClientActorConfig config = context.config();
+        responseMessageAssembler = MessageAssembler.builder().logContext(persistenceId())
+                .fileBackedStreamFactory(new FileBackedOutputStreamFactory(config.getFileBackedStreamingThreshold(),
+                        config.getTempFileDirectory()))
+                .assembledMessageCallback((message, sender) -> context.self().tell(message, sender)).build();
     }
 
     @Override
@@ -91,6 +102,11 @@ public abstract class ClientActorBehavior<T extends BackendInfo> extends
         return context().getIdentifier();
     }
 
+    @Override
+    public void close() {
+        responseMessageAssembler.close();
+    }
+
     /**
      * Get a connection to a shard.
      *
@@ -120,17 +136,29 @@ public abstract class ClientActorBehavior<T extends BackendInfo> extends
         if (command instanceof InternalCommand) {
             return ((InternalCommand<T>) command).execute(this);
         }
+
         if (command instanceof SuccessEnvelope) {
             return onRequestSuccess((SuccessEnvelope) command);
         }
+
         if (command instanceof FailureEnvelope) {
             return internalOnRequestFailure((FailureEnvelope) command);
         }
 
+        if (MessageAssembler.isHandledMessage(command)) {
+            context().dispatchers().getDispatcher(DispatcherType.Serialization).execute(
+                () -> responseMessageAssembler.handleMessage(command, context().self()));
+            return this;
+        }
+
+        if (context().messageSlicer().handleMessage(command)) {
+            return this;
+        }
+
         return onCommand(command);
     }
 
-    private static long extractCookie(final WritableIdentifier id) {
+    private static long extractCookie(final Identifier id) {
         if (id instanceof TransactionIdentifier) {
             return ((TransactionIdentifier) id).getHistoryId().getCookie();
         } else if (id instanceof LocalHistoryIdentifier) {
@@ -160,6 +188,25 @@ public abstract class ClientActorBehavior<T extends BackendInfo> extends
     }
 
     private ClientActorBehavior<T> internalOnRequestFailure(final FailureEnvelope command) {
+        final AbstractClientConnection<T> conn = getConnection(command);
+        if (conn != null) {
+            /*
+             * We are talking to multiple actors, which may be lagging behind our state significantly. This has
+             * the effect that we may be receiving responses from a previous connection after we have created a new
+             * one to a different actor.
+             *
+             * Since we are already replaying requests to the new actor, we want to ignore errors reported on the old
+             * connection -- for example NotLeaderException, which must not cause a new reconnect. Check the envelope's
+             * sessionId and if it does not match our current connection just ignore it.
+             */
+            final Optional<T> optBackend = conn.getBackendInfo();
+            if (optBackend.isPresent() && optBackend.get().getSessionId() != command.getSessionId()) {
+                LOG.debug("{}: Mismatched current connection {} and envelope {}, ignoring response", persistenceId(),
+                    conn, command);
+                return this;
+            }
+        }
+
         final RequestFailure<?, ?> failure = command.getMessage();
         final RequestException cause = failure.getCause();
         if (cause instanceof RetiredGenerationException) {
@@ -169,7 +216,6 @@ public abstract class ClientActorBehavior<T extends BackendInfo> extends
             return null;
         }
         if (cause instanceof NotLeaderException) {
-            final AbstractClientConnection<T> conn = getConnection(command);
             if (conn instanceof ReconnectingClientConnection) {
                 // Already reconnecting, do not churn the logs
                 return this;
@@ -179,13 +225,12 @@ public abstract class ClientActorBehavior<T extends BackendInfo> extends
             }
         }
         if (cause instanceof OutOfSequenceEnvelopeException) {
-            final AbstractClientConnection<T> conn = getConnection(command);
             if (conn instanceof ReconnectingClientConnection) {
                 // Already reconnecting, do not churn the logs
                 return this;
             } else if (conn != null) {
-                LOG.info("{}: connection {} indicated no sequencing mismatch on {} sequence {}, reconnecting it",
-                    persistenceId(), conn, failure.getTarget(), failure.getSequence(), cause);
+                LOG.info("{}: connection {} indicated sequencing mismatch on {} sequence {} ({}), reconnecting it",
+                    persistenceId(), conn, failure.getTarget(), failure.getSequence(), command.getTxSequence(), cause);
                 return conn.reconnect(this, cause);
             }
         }
@@ -204,6 +249,8 @@ public abstract class ClientActorBehavior<T extends BackendInfo> extends
         } finally {
             connectionsLock.unlockWrite(stamp);
         }
+
+        context().messageSlicer().close();
     }
 
     /**
@@ -243,22 +290,22 @@ public abstract class ClientActorBehavior<T extends BackendInfo> extends
     @GuardedBy("connectionsLock")
     @Nonnull protected abstract ConnectionConnectCohort connectionUp(@Nonnull ConnectedClientConnection<T> newConn);
 
-    private void backendConnectFinished(final Long shard, final AbstractClientConnection<T> conn,
+    private void backendConnectFinished(final Long shard, final AbstractClientConnection<T> oldConn,
             final T backend, final Throwable failure) {
         if (failure != null) {
             if (failure instanceof TimeoutException) {
-                if (!conn.equals(connections.get(shard))) {
+                if (!oldConn.equals(connections.get(shard))) {
                     // AbstractClientConnection will remove itself when it decides there is no point in continuing,
                     // at which point we want to stop retrying
-                    LOG.info("{}: stopping resolution of shard {} on stale connection {}", persistenceId(), shard, conn,
-                        failure);
+                    LOG.info("{}: stopping resolution of shard {} on stale connection {}", persistenceId(), shard,
+                        oldConn, failure);
                     return;
                 }
 
                 LOG.debug("{}: timed out resolving shard {}, scheduling retry in {}", persistenceId(), shard,
                     RESOLVE_RETRY_DURATION, failure);
                 context().executeInActor(b -> {
-                    resolveConnection(shard, conn);
+                    resolveConnection(shard, oldConn);
                     return b;
                 }, RESOLVE_RETRY_DURATION);
                 return;
@@ -272,7 +319,7 @@ public abstract class ClientActorBehavior<T extends BackendInfo> extends
                 cause = new RuntimeRequestException("Failed to resolve shard " + shard, failure);
             }
 
-            conn.poison(cause);
+            oldConn.poison(cause);
             return;
         }
 
@@ -282,29 +329,31 @@ public abstract class ClientActorBehavior<T extends BackendInfo> extends
             final Stopwatch sw = Stopwatch.createStarted();
 
             // Create a new connected connection
-            final ConnectedClientConnection<T> newConn = new ConnectedClientConnection<>(conn.context(),
-                    conn.cookie(), backend);
-            LOG.info("{}: resolving connection {} to {}", persistenceId(), conn, newConn);
+            final ConnectedClientConnection<T> newConn = new ConnectedClientConnection<>(oldConn, backend);
+            LOG.info("{}: resolving connection {} to {}", persistenceId(), oldConn, newConn);
 
             // Start reconnecting without the old connection lock held
             final ConnectionConnectCohort cohort = Verify.verifyNotNull(connectionUp(newConn));
 
             // Lock the old connection and get a reference to its entries
-            final Collection<ConnectionEntry> replayIterable = conn.startReplay();
+            final Collection<ConnectionEntry> replayIterable = oldConn.startReplay();
 
             // Finish the connection attempt
             final ReconnectForwarder forwarder = Verify.verifyNotNull(cohort.finishReconnect(replayIterable));
 
+            // Cancel sleep debt after entries were replayed, before new connection starts receiving.
+            newConn.cancelDebt();
+
             // Install the forwarder, unlocking the old connection
-            conn.finishReplay(forwarder);
+            oldConn.finishReplay(forwarder);
 
             // Make sure new lookups pick up the new connection
-            if (!connections.replace(shard, conn, newConn)) {
-                final AbstractClientConnection<T> existing = connections.get(conn.cookie());
+            if (!connections.replace(shard, oldConn, newConn)) {
+                final AbstractClientConnection<T> existing = connections.get(oldConn.cookie());
                 LOG.warn("{}: old connection {} does not match existing {}, new connection {} in limbo",
-                    persistenceId(), conn, existing, newConn);
+                    persistenceId(), oldConn, existing, newConn);
             } else {
-                LOG.info("{}: replaced connection {} with {} in {}", persistenceId(), conn, newConn, sw);
+                LOG.info("{}: replaced connection {} with {} in {}", persistenceId(), oldConn, newConn, sw);
             }
         } finally {
             connectionsLock.unlockWrite(stamp);
@@ -324,6 +373,7 @@ public abstract class ClientActorBehavior<T extends BackendInfo> extends
                 }
             } else {
                 LOG.info("{}: removed connection {}", persistenceId(), conn);
+                cancelSlicing(conn.cookie());
             }
         } finally {
             connectionsLock.unlockWrite(stamp);
@@ -347,6 +397,8 @@ public abstract class ClientActorBehavior<T extends BackendInfo> extends
                 } else {
                     LOG.warn("{}: failed to replace connection {}, as it was not tracked", persistenceId(), conn);
                 }
+            } else {
+                cancelSlicing(oldConn.cookie());
             }
         } finally {
             connectionsLock.unlockWrite(stamp);
@@ -361,6 +413,17 @@ public abstract class ClientActorBehavior<T extends BackendInfo> extends
             }));
     }
 
+    private void cancelSlicing(final Long cookie) {
+        context().messageSlicer().cancelSlicing(id -> {
+            try {
+                return cookie.equals(extractCookie(id));
+            } catch (IllegalArgumentException e) {
+                LOG.debug("extractCookie failed while cancelling slicing for cookie {}: {}", cookie, e);
+                return false;
+            }
+        });
+    }
+
     private ConnectingClientConnection<T> createConnection(final Long shard) {
         final ConnectingClientConnection<T> conn = new ConnectingClientConnection<>(context(), shard);
         resolveConnection(shard, conn);