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:
Endre Sándor Varga 2013-06-19 14:09:14 +02:00
parent 6d8e13c760
commit e6e5be859b
20 changed files with 220 additions and 120 deletions

View file

@ -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