Clear system messages sequence number for restarted node, #24847
* Notice that the incarnation has changed in SystemMessageDelivery and then reset the sequence number * Take the incarnation number into account in the ClearSystemMessageDelivery message * Trigger quarantine earlier in ClusterRemoteWatcher if node with same host:port joined * Change quarantine-removed-node-after to 5s, shouldn't be necessary to delay it 30s * test reproducer
This commit is contained in:
parent
85026f5f1d
commit
43dc381d59
6 changed files with 165 additions and 15 deletions
|
|
@ -1,3 +1,6 @@
|
|||
# #24553 eliminate test stages when not used
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.artery.OutboundTestStage.this")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.artery.InboundTestStage.this")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.artery.InboundTestStage.this")
|
||||
|
||||
# #24847 ClearSystemMessageDelivery
|
||||
ProblemFilters.exclude[Problem]("akka.remote.artery.SystemMessageDelivery*")
|
||||
|
|
|
|||
|
|
@ -345,10 +345,11 @@ private[remote] class Association(
|
|||
|
||||
val state = associationState
|
||||
val quarantined = state.isQuarantined()
|
||||
val messageIsClearSystemMessageDelivery = message.isInstanceOf[ClearSystemMessageDelivery]
|
||||
|
||||
// allow ActorSelectionMessage to pass through quarantine, to be able to establish interaction with new system
|
||||
if (message.isInstanceOf[ActorSelectionMessage] || !quarantined || message == ClearSystemMessageDelivery) {
|
||||
if (quarantined && message != ClearSystemMessageDelivery) {
|
||||
if (message.isInstanceOf[ActorSelectionMessage] || !quarantined || messageIsClearSystemMessageDelivery) {
|
||||
if (quarantined && !messageIsClearSystemMessageDelivery) {
|
||||
log.debug("Quarantine piercing attempt with message [{}] to [{}]", Logging.messageClassName(message), recipient.getOrElse(""))
|
||||
startQuarantinedIdleTimer()
|
||||
}
|
||||
|
|
@ -360,7 +361,7 @@ private[remote] class Association(
|
|||
quarantine(reason = s"Due to overflow of control queue, size [$controlQueueSize]")
|
||||
dropped(ControlQueueIndex, controlQueueSize, outboundEnvelope)
|
||||
}
|
||||
case ActorSelectionMessage(_: PriorityMessage, _, _) | _: ControlMessage | ClearSystemMessageDelivery ⇒
|
||||
case ActorSelectionMessage(_: PriorityMessage, _, _) | _: ControlMessage | _: ClearSystemMessageDelivery ⇒
|
||||
// ActorSelectionMessage with PriorityMessage is used by cluster and remote failure detector heartbeating
|
||||
if (!controlQueue.offer(outboundEnvelope)) {
|
||||
dropped(ControlQueueIndex, controlQueueSize, outboundEnvelope)
|
||||
|
|
@ -476,7 +477,7 @@ private[remote] class Association(
|
|||
clearOutboundCompression()
|
||||
clearInboundCompression(u)
|
||||
// end delivery of system messages to that incarnation after this point
|
||||
send(ClearSystemMessageDelivery, OptionVal.None, OptionVal.None)
|
||||
send(ClearSystemMessageDelivery(current.incarnation), OptionVal.None, OptionVal.None)
|
||||
// try to tell the other system that we have quarantined it
|
||||
sendControl(Quarantined(localAddress, peer))
|
||||
startQuarantinedIdleTimer()
|
||||
|
|
@ -484,11 +485,12 @@ private[remote] class Association(
|
|||
quarantine(reason, uid) // recursive
|
||||
}
|
||||
case Some(peer) ⇒
|
||||
log.debug(
|
||||
log.info(
|
||||
"Quarantine of [{}] ignored due to non-matching UID, quarantine requested for [{}] but current is [{}]. {}",
|
||||
remoteAddress, u, peer.uid, reason)
|
||||
send(ClearSystemMessageDelivery(current.incarnation - 1), OptionVal.None, OptionVal.None)
|
||||
case None ⇒
|
||||
log.debug(
|
||||
log.info(
|
||||
"Quarantine of [{}] ignored because handshake not completed, quarantine request was for old incarnation. {}",
|
||||
remoteAddress, reason)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -42,7 +42,19 @@ private[remote] object SystemMessageDelivery {
|
|||
final case class Ack(seqNo: Long, from: UniqueAddress) extends Reply
|
||||
final case class Nack(seqNo: Long, from: UniqueAddress) extends Reply
|
||||
|
||||
final case object ClearSystemMessageDelivery
|
||||
/**
|
||||
* Sent when an incarnation of an Association is quarantined. Consumed by the
|
||||
* SystemMessageDelivery stage on the sending side, i.e. not sent to remote system.
|
||||
* The SystemMessageDelivery stage will clear the sequence number and other state associated
|
||||
* with that incarnation.
|
||||
*
|
||||
* The incarnation counter is bumped when the handshake is completed, so a new incarnation
|
||||
* corresponds to a new UID of the remote system.
|
||||
*
|
||||
* The SystemMessageDelivery stage also detects that the incarnation has changed when sending or resending
|
||||
* system messages.
|
||||
*/
|
||||
final case class ClearSystemMessageDelivery(incarnation: Int)
|
||||
|
||||
final class GaveUpSystemMessageException(msg: String) extends RuntimeException(msg) with NoStackTrace
|
||||
|
||||
|
|
@ -75,6 +87,7 @@ private[remote] class SystemMessageDelivery(
|
|||
|
||||
private var replyObserverAttached = false
|
||||
private var seqNo = 0L // sequence number for the first message will be 1
|
||||
private var incarnation = outboundContext.associationState.incarnation
|
||||
private val unacknowledged = new ArrayDeque[OutboundEnvelope]
|
||||
private var resending = new ArrayDeque[OutboundEnvelope]
|
||||
private var resendingFromSeqNo = -1L
|
||||
|
|
@ -85,6 +98,8 @@ private[remote] class SystemMessageDelivery(
|
|||
|
||||
private def localAddress = outboundContext.localAddress
|
||||
private def remoteAddress = outboundContext.remoteAddress
|
||||
private def remoteAddressLogParam: String =
|
||||
outboundContext.associationState.uniqueRemoteAddressValue().getOrElse(remoteAddress).toString
|
||||
|
||||
override protected def logSource: Class[_] = classOf[SystemMessageDelivery]
|
||||
|
||||
|
|
@ -192,6 +207,11 @@ private[remote] class SystemMessageDelivery(
|
|||
}
|
||||
}
|
||||
|
||||
if (incarnation != outboundContext.associationState.incarnation) {
|
||||
log.debug("Noticed new incarnation of [{}] from tryResend, clear state", remoteAddressLogParam)
|
||||
clear()
|
||||
}
|
||||
|
||||
pushCopy(env)
|
||||
}
|
||||
}
|
||||
|
|
@ -207,6 +227,12 @@ private[remote] class SystemMessageDelivery(
|
|||
outboundEnvelope.message match {
|
||||
case msg @ (_: SystemMessage | _: AckedDeliveryMessage) ⇒
|
||||
if (unacknowledged.size < maxBufferSize) {
|
||||
if (seqNo == 0) {
|
||||
incarnation = outboundContext.associationState.incarnation
|
||||
} else if (incarnation != outboundContext.associationState.incarnation) {
|
||||
log.debug("Noticed new incarnation of [{}] from onPush, clear state", remoteAddressLogParam)
|
||||
clear()
|
||||
}
|
||||
seqNo += 1
|
||||
if (unacknowledged.isEmpty)
|
||||
ackTimestamp = System.nanoTime()
|
||||
|
|
@ -231,8 +257,11 @@ private[remote] class SystemMessageDelivery(
|
|||
// pass on HandshakeReq
|
||||
if (isAvailable(out))
|
||||
pushCopy(outboundEnvelope)
|
||||
case ClearSystemMessageDelivery ⇒
|
||||
clear()
|
||||
case ClearSystemMessageDelivery(i) ⇒
|
||||
if (i <= incarnation) {
|
||||
log.debug("Clear system message delivery of [{}]", remoteAddressLogParam)
|
||||
clear()
|
||||
}
|
||||
pull(in)
|
||||
case _ ⇒
|
||||
// e.g. ActorSystemTerminating or ActorSelectionMessage with PriorityMessage, no need for acked delivery
|
||||
|
|
@ -255,6 +284,7 @@ private[remote] class SystemMessageDelivery(
|
|||
private def clear(): Unit = {
|
||||
sendUnacknowledgedToDeadLetters()
|
||||
seqNo = 0L // sequence number for the first message will be 1
|
||||
incarnation = outboundContext.associationState.incarnation
|
||||
unacknowledged.clear()
|
||||
resending.clear()
|
||||
resendingFromSeqNo = -1L
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue