Fixes for leaked connection
- Also introduces reason in the Disassociate message - Reliable delivery now transitions from idle to active if there are pending system msgs - Minor fix in merging receive buffers (reduces resends) - Tweaked WireFormat - Removed busy-wait in startup - throwing the proper exception type in EndpointReader - InvalidAssociationException extends NoStackTrace
This commit is contained in:
parent
6d8e13c760
commit
e6e5be859b
20 changed files with 220 additions and 120 deletions
|
|
@ -3,27 +3,29 @@
|
|||
*/
|
||||
package akka.remote
|
||||
|
||||
import akka.actor.OneForOneStrategy
|
||||
import akka.actor.SupervisorStrategy._
|
||||
import akka.actor.Terminated
|
||||
import akka.actor._
|
||||
import akka.dispatch.sysmsg.SystemMessage
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.pattern.pipe
|
||||
import akka.remote.EndpointManager.Link
|
||||
import akka.remote.EndpointManager.Send
|
||||
import akka.remote.EndpointWriter.{ StoppedReading, FlushAndStop }
|
||||
import akka.remote.WireFormats.SerializedMessage
|
||||
import akka.remote.transport.AkkaPduCodec._
|
||||
import akka.remote.transport.AssociationHandle._
|
||||
import akka.remote.transport.AkkaPduCodec.Message
|
||||
import akka.remote.transport.AssociationHandle.{ DisassociateInfo, ActorHandleEventListener, Disassociated, InboundPayload }
|
||||
import akka.remote.transport.Transport.InvalidAssociationException
|
||||
import akka.remote.transport.{ AkkaPduProtobufCodec, AkkaPduCodec, Transport, AkkaProtocolHandle }
|
||||
import akka.remote.transport._
|
||||
import akka.serialization.Serialization
|
||||
import akka.util.ByteString
|
||||
import akka.{ OnlyCauseStackTrace, AkkaException }
|
||||
import java.io.NotSerializableException
|
||||
import java.util.concurrent.ConcurrentHashMap
|
||||
import scala.annotation.tailrec
|
||||
import scala.concurrent.duration.{ Duration, Deadline }
|
||||
import scala.util.control.NonFatal
|
||||
import scala.annotation.tailrec
|
||||
import akka.remote.EndpointWriter.FlushAndStop
|
||||
import akka.actor.SupervisorStrategy._
|
||||
import akka.remote.EndpointManager.Link
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -145,6 +147,7 @@ private[remote] class OversizedPayloadException(msg: String) extends EndpointExc
|
|||
*/
|
||||
private[remote] object ReliableDeliverySupervisor {
|
||||
case object Ungate
|
||||
case object AttemptSysMsgRedelivery
|
||||
case class GotUid(uid: Int)
|
||||
|
||||
def props(
|
||||
|
|
@ -171,6 +174,7 @@ private[remote] class ReliableDeliverySupervisor(
|
|||
val codec: AkkaPduCodec,
|
||||
val receiveBuffers: ConcurrentHashMap[Link, AckedReceiveBuffer[Message]]) extends Actor {
|
||||
import ReliableDeliverySupervisor._
|
||||
import context.dispatcher
|
||||
|
||||
def retryGateEnabled = settings.RetryGateClosedFor > Duration.Zero
|
||||
|
||||
|
|
@ -178,11 +182,11 @@ private[remote] class ReliableDeliverySupervisor(
|
|||
case e @ (_: InvalidAssociation | _: HopelessAssociation | _: QuarantinedUidException) ⇒ Escalate
|
||||
case NonFatal(e) ⇒
|
||||
if (retryGateEnabled) {
|
||||
import context.dispatcher
|
||||
context.become(gated)
|
||||
context.system.scheduler.scheduleOnce(settings.RetryGateClosedFor, self, Ungate)
|
||||
context.unwatch(writer)
|
||||
currentHandle = None
|
||||
context.parent ! StoppedReading(self)
|
||||
Stop
|
||||
} else {
|
||||
Restart
|
||||
|
|
@ -242,6 +246,9 @@ private[remote] class ReliableDeliverySupervisor(
|
|||
resendNacked()
|
||||
case Terminated(_) ⇒
|
||||
currentHandle = None
|
||||
context.parent ! StoppedReading(self)
|
||||
if (resendBuffer.nonAcked.nonEmpty || resendBuffer.nacked.nonEmpty)
|
||||
context.system.scheduler.scheduleOnce(settings.SysResendTimeout, self, AttemptSysMsgRedelivery)
|
||||
context.become(idle)
|
||||
case GotUid(u) ⇒ uid = Some(u)
|
||||
case s: EndpointWriter.StopReading ⇒ writer forward s
|
||||
|
|
@ -267,6 +274,10 @@ private[remote] class ReliableDeliverySupervisor(
|
|||
resendAll()
|
||||
handleSend(s)
|
||||
context.become(receive)
|
||||
case AttemptSysMsgRedelivery ⇒
|
||||
writer = createWriter()
|
||||
resendAll()
|
||||
context.become(receive)
|
||||
case EndpointWriter.FlushAndStop ⇒ context.stop(self)
|
||||
case EndpointWriter.StopReading(w) ⇒ sender ! EndpointWriter.StoppedReading(w)
|
||||
}
|
||||
|
|
@ -415,6 +426,7 @@ private[remote] class EndpointWriter(
|
|||
val msgDispatch = new DefaultMessageDispatcher(extendedSystem, provider, log)
|
||||
|
||||
var inbound = handle.isDefined
|
||||
var stopReason: DisassociateInfo = AssociationHandle.Unknown
|
||||
|
||||
private def publishAndThrow(reason: Throwable): Nothing = {
|
||||
publishError(reason)
|
||||
|
|
@ -521,6 +533,7 @@ private[remote] class EndpointWriter(
|
|||
case Event(FlushAndStop, _) ⇒
|
||||
// Try to send a last Ack message
|
||||
trySendPureAck()
|
||||
stopReason = AssociationHandle.Shutdown
|
||||
stop()
|
||||
|
||||
case Event(AckIdleCheckTimer, _) if ackDeadline.isOverdue() ⇒
|
||||
|
|
@ -558,6 +571,7 @@ private[remote] class EndpointWriter(
|
|||
handle = Some(newHandle)
|
||||
goto(Handoff)
|
||||
case Event(FlushAndStop, _) ⇒
|
||||
stopReason = AssociationHandle.Shutdown
|
||||
stop()
|
||||
case Event(OutboundAck(ack), _) ⇒
|
||||
lastAck = Some(ack)
|
||||
|
|
@ -583,7 +597,7 @@ private[remote] class EndpointWriter(
|
|||
// It is important to call unstashAll() for the stash to work properly and maintain messages during restart.
|
||||
// As the FSM trait does not call super.postStop(), this call is needed
|
||||
unstashAll()
|
||||
handle foreach { _.disassociate() }
|
||||
handle foreach { _.disassociate(stopReason) }
|
||||
eventPublisher.notifyListeners(DisassociatedEvent(localAddress, remoteAddress, inbound))
|
||||
}
|
||||
|
||||
|
|
@ -675,8 +689,7 @@ private[remote] class EndpointReader(
|
|||
}
|
||||
|
||||
override def receive: Receive = {
|
||||
case Disassociated ⇒
|
||||
context.stop(self)
|
||||
case Disassociated(info) ⇒ handleDisassociated(info)
|
||||
|
||||
case InboundPayload(p) if p.size <= transport.maximumPayloadBytes ⇒
|
||||
val (ackOption, msgOption) = tryDecodeMessageAndAck(p)
|
||||
|
|
@ -705,10 +718,10 @@ private[remote] class EndpointReader(
|
|||
}
|
||||
|
||||
def notReading: Receive = {
|
||||
case Disassociated ⇒ context.stop(self)
|
||||
case Disassociated(info) ⇒ handleDisassociated(info)
|
||||
|
||||
case StopReading(newHandle) ⇒
|
||||
sender ! StoppedReading(newHandle)
|
||||
case StopReading(writer) ⇒
|
||||
sender ! StoppedReading(writer)
|
||||
|
||||
case InboundPayload(p) ⇒
|
||||
val (ackOption, _) = tryDecodeMessageAndAck(p)
|
||||
|
|
@ -717,6 +730,22 @@ private[remote] class EndpointReader(
|
|||
case _ ⇒
|
||||
}
|
||||
|
||||
private def handleDisassociated(info: DisassociateInfo): Unit = info match {
|
||||
case AssociationHandle.Unknown ⇒
|
||||
context.stop(self)
|
||||
case AssociationHandle.Shutdown ⇒
|
||||
throw InvalidAssociation(
|
||||
localAddress,
|
||||
remoteAddress,
|
||||
InvalidAssociationException("The remote system terminated the association because it is shutting down."))
|
||||
case AssociationHandle.Quarantined ⇒
|
||||
throw InvalidAssociation(
|
||||
localAddress,
|
||||
remoteAddress,
|
||||
InvalidAssociationException("The remote system has quarantined this system. No further associations " +
|
||||
"to the remote system are possible until this system is restarted."))
|
||||
}
|
||||
|
||||
private def deliverAndAck(): Unit = {
|
||||
val (updatedBuffer, deliver, ack) = ackedReceiveBuffer.extractDeliverable
|
||||
ackedReceiveBuffer = updatedBuffer
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue