2013-01-15 18:08:45 +01:00
|
|
|
/**
|
2017-01-04 17:37:10 +01:00
|
|
|
* Copyright (C) 2009-2017 Lightbend Inc. <http://www.lightbend.com>
|
2013-01-15 18:08:45 +01:00
|
|
|
*/
|
|
|
|
|
|
|
|
|
|
package akka.io
|
|
|
|
|
|
2017-05-16 15:22:11 +02:00
|
|
|
import java.net.{ InetSocketAddress, SocketException }
|
2013-05-06 17:01:01 +02:00
|
|
|
import java.nio.channels.SelectionKey._
|
2013-04-09 14:22:20 +02:00
|
|
|
import java.io.{ FileInputStream, IOException }
|
|
|
|
|
import java.nio.channels.{ FileChannel, SocketChannel }
|
2013-01-22 14:10:36 +01:00
|
|
|
import java.nio.ByteBuffer
|
2017-05-16 15:22:11 +02:00
|
|
|
|
2013-01-22 14:10:36 +01:00
|
|
|
import scala.annotation.tailrec
|
2013-01-22 16:03:22 +01:00
|
|
|
import scala.collection.immutable
|
2017-05-16 15:22:11 +02:00
|
|
|
import scala.util.control.{ NoStackTrace, NonFatal }
|
2013-01-15 18:08:45 +01:00
|
|
|
import scala.concurrent.duration._
|
|
|
|
|
import akka.actor._
|
|
|
|
|
import akka.util.ByteString
|
2013-02-05 11:48:47 +01:00
|
|
|
import akka.io.Inet.SocketOption
|
|
|
|
|
import akka.io.Tcp._
|
2013-02-01 13:11:17 +01:00
|
|
|
import akka.io.SelectionHandler._
|
2017-05-16 15:22:11 +02:00
|
|
|
import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
|
2017-04-19 19:05:19 +02:00
|
|
|
import java.nio.file.Paths
|
2013-01-15 18:08:45 +01:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Base class for TcpIncomingConnection and TcpOutgoingConnection.
|
2013-02-15 11:59:01 +01:00
|
|
|
*
|
|
|
|
|
* INTERNAL API
|
2013-01-15 18:08:45 +01:00
|
|
|
*/
|
2014-01-07 15:50:36 +01:00
|
|
|
private[io] abstract class TcpConnection(val tcp: TcpExt, val channel: SocketChannel, val pullMode: Boolean)
|
2013-04-26 12:18:01 +02:00
|
|
|
extends Actor with ActorLogging with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
|
2013-05-06 17:01:01 +02:00
|
|
|
|
2013-01-17 17:29:44 +01:00
|
|
|
import tcp.Settings._
|
2013-02-04 16:24:34 +01:00
|
|
|
import tcp.bufferPool
|
2013-01-24 15:08:42 +01:00
|
|
|
import TcpConnection._
|
2013-01-17 14:45:50 +01:00
|
|
|
|
2013-09-05 16:40:06 +02:00
|
|
|
private[this] var pendingWrite: PendingWrite = EmptyPendingWrite
|
2013-04-11 14:28:14 +02:00
|
|
|
private[this] var peerClosed = false
|
2013-05-06 17:01:01 +02:00
|
|
|
private[this] var writingSuspended = false
|
2014-01-07 15:50:36 +01:00
|
|
|
private[this] var readingSuspended = pullMode
|
2013-05-06 17:01:01 +02:00
|
|
|
private[this] var interestedInResume: Option[ActorRef] = None
|
|
|
|
|
var closedMessage: CloseInformation = _ // for ConnectionClosed message in postStop
|
2015-07-06 14:09:06 +02:00
|
|
|
private var watchedActor: ActorRef = context.system.deadLetters
|
2017-06-01 00:32:58 -07:00
|
|
|
private var registration: Option[ChannelRegistration] = None
|
2015-07-06 14:09:06 +02:00
|
|
|
|
|
|
|
|
def signDeathPact(actor: ActorRef): Unit = {
|
|
|
|
|
unsignDeathPact()
|
|
|
|
|
watchedActor = actor
|
|
|
|
|
context.watch(watchedActor)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def unsignDeathPact(): Unit =
|
|
|
|
|
if (watchedActor ne context.system.deadLetters) context.unwatch(watchedActor)
|
2013-04-07 17:05:22 +02:00
|
|
|
|
2013-09-05 16:40:06 +02:00
|
|
|
def writePending = pendingWrite ne EmptyPendingWrite
|
2013-01-15 18:08:45 +01:00
|
|
|
|
|
|
|
|
// STATES
|
|
|
|
|
|
|
|
|
|
/** connection established, waiting for registration from user handler */
|
2013-05-06 17:01:01 +02:00
|
|
|
def waitingForRegistration(registration: ChannelRegistration, commander: ActorRef): Receive = {
|
2013-04-16 22:31:09 +02:00
|
|
|
case Register(handler, keepOpenOnPeerClosed, useResumeWriting) ⇒
|
2013-04-11 14:31:01 +02:00
|
|
|
// up to this point we've been watching the commander,
|
|
|
|
|
// but since registration is now complete we only need to watch the handler from here on
|
|
|
|
|
if (handler != commander) {
|
|
|
|
|
context.unwatch(commander)
|
|
|
|
|
context.watch(handler)
|
|
|
|
|
}
|
2013-02-10 13:52:52 +01:00
|
|
|
if (TraceLogging) log.debug("[{}] registered as connection handler", handler)
|
2013-04-07 17:05:22 +02:00
|
|
|
|
2013-05-06 17:01:01 +02:00
|
|
|
val info = ConnectionInfo(registration, handler, keepOpenOnPeerClosed, useResumeWriting)
|
2014-09-15 11:19:07 +02:00
|
|
|
|
2014-05-28 16:50:24 +02:00
|
|
|
// if we have resumed reading from pullMode while waiting for Register then register OP_READ interest
|
|
|
|
|
if (pullMode && !readingSuspended) resumeReading(info)
|
2014-05-23 11:33:06 +02:00
|
|
|
doRead(info, None) // immediately try reading, pullMode is handled by readingSuspended
|
2013-01-15 18:08:45 +01:00
|
|
|
context.setReceiveTimeout(Duration.Undefined)
|
2013-05-06 17:01:01 +02:00
|
|
|
context.become(connected(info))
|
2013-01-15 18:08:45 +01:00
|
|
|
|
2014-05-23 11:33:06 +02:00
|
|
|
case ResumeReading ⇒
|
|
|
|
|
readingSuspended = false
|
|
|
|
|
|
|
|
|
|
case SuspendReading ⇒
|
|
|
|
|
readingSuspended = true
|
|
|
|
|
|
2013-01-15 18:08:45 +01:00
|
|
|
case cmd: CloseCommand ⇒
|
2013-05-06 17:01:01 +02:00
|
|
|
val info = ConnectionInfo(registration, commander, keepOpenOnPeerClosed = false, useResumeWriting = false)
|
2014-01-16 15:16:35 +01:00
|
|
|
handleClose(info, Some(sender()), cmd.event)
|
2013-01-15 18:08:45 +01:00
|
|
|
|
|
|
|
|
case ReceiveTimeout ⇒
|
2013-01-17 14:45:50 +01:00
|
|
|
// after sending `Register` user should watch this actor to make sure
|
|
|
|
|
// it didn't die because of the timeout
|
2013-05-31 09:52:51 +02:00
|
|
|
log.debug("Configured registration timeout of [{}] expired, stopping", RegisterTimeout)
|
2013-01-15 18:08:45 +01:00
|
|
|
context.stop(self)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/** normal connected state */
|
2013-05-06 17:01:01 +02:00
|
|
|
def connected(info: ConnectionInfo): Receive =
|
|
|
|
|
handleWriteMessages(info) orElse {
|
2013-10-21 15:33:31 +02:00
|
|
|
case SuspendReading ⇒ suspendReading(info)
|
|
|
|
|
case ResumeReading ⇒ resumeReading(info)
|
2013-05-06 17:01:01 +02:00
|
|
|
case ChannelReadable ⇒ doRead(info, None)
|
2014-01-16 15:16:35 +01:00
|
|
|
case cmd: CloseCommand ⇒ handleClose(info, Some(sender()), cmd.event)
|
2013-05-06 17:01:01 +02:00
|
|
|
}
|
2013-01-15 18:08:45 +01:00
|
|
|
|
2013-04-02 15:17:48 +02:00
|
|
|
/** the peer sent EOF first, but we may still want to send */
|
2013-05-06 17:01:01 +02:00
|
|
|
def peerSentEOF(info: ConnectionInfo): Receive =
|
|
|
|
|
handleWriteMessages(info) orElse {
|
2014-01-16 15:16:35 +01:00
|
|
|
case cmd: CloseCommand ⇒ handleClose(info, Some(sender()), cmd.event)
|
2017-03-15 16:48:20 +01:00
|
|
|
case ResumeReading ⇒ // ignore, no more data to read
|
2013-05-06 17:01:01 +02:00
|
|
|
}
|
2013-01-15 18:08:45 +01:00
|
|
|
|
|
|
|
|
/** connection is closing but a write has to be finished first */
|
2013-05-06 17:01:01 +02:00
|
|
|
def closingWithPendingWrite(info: ConnectionInfo, closeCommander: Option[ActorRef],
|
|
|
|
|
closedEvent: ConnectionClosed): Receive = {
|
2013-10-21 15:33:31 +02:00
|
|
|
case SuspendReading ⇒ suspendReading(info)
|
|
|
|
|
case ResumeReading ⇒ resumeReading(info)
|
2013-05-06 17:01:01 +02:00
|
|
|
case ChannelReadable ⇒ doRead(info, closeCommander)
|
2013-01-15 18:08:45 +01:00
|
|
|
|
|
|
|
|
case ChannelWritable ⇒
|
2013-05-06 17:01:01 +02:00
|
|
|
doWrite(info)
|
2013-01-15 18:08:45 +01:00
|
|
|
if (!writePending) // writing is now finished
|
2013-05-06 17:01:01 +02:00
|
|
|
handleClose(info, closeCommander, closedEvent)
|
2013-01-15 18:08:45 +01:00
|
|
|
|
2014-09-30 10:40:48 +02:00
|
|
|
case UpdatePendingWriteAndThen(remaining, work) ⇒
|
2013-09-05 16:40:06 +02:00
|
|
|
pendingWrite = remaining
|
2014-09-30 10:40:48 +02:00
|
|
|
work()
|
2013-09-05 16:40:06 +02:00
|
|
|
if (writePending) info.registration.enableInterest(OP_WRITE)
|
|
|
|
|
else handleClose(info, closeCommander, closedEvent)
|
|
|
|
|
|
|
|
|
|
case WriteFileFailed(e) ⇒ handleError(info.handler, e) // rethrow exception from dispatcher task
|
|
|
|
|
|
2014-01-16 15:16:35 +01:00
|
|
|
case Abort ⇒ handleClose(info, Some(sender()), Aborted)
|
2013-01-15 18:08:45 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/** connection is closed on our side and we're waiting from confirmation from the other side */
|
2013-05-06 17:01:01 +02:00
|
|
|
def closing(info: ConnectionInfo, closeCommander: Option[ActorRef]): Receive = {
|
2013-10-21 15:33:31 +02:00
|
|
|
case SuspendReading ⇒ suspendReading(info)
|
|
|
|
|
case ResumeReading ⇒ resumeReading(info)
|
2013-05-06 17:01:01 +02:00
|
|
|
case ChannelReadable ⇒ doRead(info, closeCommander)
|
2014-01-16 15:16:35 +01:00
|
|
|
case Abort ⇒ handleClose(info, Some(sender()), Aborted)
|
2013-01-15 18:08:45 +01:00
|
|
|
}
|
|
|
|
|
|
2013-05-06 17:01:01 +02:00
|
|
|
def handleWriteMessages(info: ConnectionInfo): Receive = {
|
2013-04-16 22:31:09 +02:00
|
|
|
case ChannelWritable ⇒
|
|
|
|
|
if (writePending) {
|
2013-05-06 17:01:01 +02:00
|
|
|
doWrite(info)
|
2013-04-16 22:31:09 +02:00
|
|
|
if (!writePending && interestedInResume.nonEmpty) {
|
|
|
|
|
interestedInResume.get ! WritingResumed
|
|
|
|
|
interestedInResume = None
|
|
|
|
|
}
|
|
|
|
|
}
|
2013-04-02 15:17:48 +02:00
|
|
|
|
2013-04-09 12:25:43 +02:00
|
|
|
case write: WriteCommand ⇒
|
2013-04-16 22:31:09 +02:00
|
|
|
if (writingSuspended) {
|
|
|
|
|
if (TraceLogging) log.debug("Dropping write because writing is suspended")
|
2017-05-16 15:22:11 +02:00
|
|
|
sender() ! write.failureMessage.withCause(DroppingWriteBecauseWritingIsSuspendedException)
|
2013-04-16 22:31:09 +02:00
|
|
|
|
|
|
|
|
} else if (writePending) {
|
|
|
|
|
if (TraceLogging) log.debug("Dropping write because queue is full")
|
2017-05-16 15:22:11 +02:00
|
|
|
sender() ! write.failureMessage.withCause(DroppingWriteBecauseQueueIsFullException)
|
2013-05-06 17:01:01 +02:00
|
|
|
if (info.useResumeWriting) writingSuspended = true
|
2013-04-16 22:31:09 +02:00
|
|
|
|
2013-09-05 16:40:06 +02:00
|
|
|
} else {
|
2014-01-16 15:16:35 +01:00
|
|
|
pendingWrite = PendingWrite(sender(), write)
|
2013-09-05 16:40:06 +02:00
|
|
|
if (writePending) doWrite(info)
|
2013-04-16 22:31:09 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
case ResumeWriting ⇒
|
|
|
|
|
/*
|
2013-05-31 09:52:51 +02:00
|
|
|
* If more than one actor sends Writes then the first to send this
|
2013-04-16 22:31:09 +02:00
|
|
|
* message might resume too early for the second, leading to a Write of
|
|
|
|
|
* the second to go through although it has not been resumed yet; there
|
2013-05-31 09:52:51 +02:00
|
|
|
* is nothing we can do about this apart from all actors needing to
|
2013-04-16 22:31:09 +02:00
|
|
|
* register themselves and us keeping track of them, which sounds bad.
|
|
|
|
|
*
|
|
|
|
|
* Thus it is documented that useResumeWriting is incompatible with
|
|
|
|
|
* multiple writers. But we fail as gracefully as we can.
|
|
|
|
|
*/
|
|
|
|
|
writingSuspended = false
|
|
|
|
|
if (writePending) {
|
2014-01-16 15:16:35 +01:00
|
|
|
if (interestedInResume.isEmpty) interestedInResume = Some(sender())
|
|
|
|
|
else sender() ! CommandFailed(ResumeWriting)
|
|
|
|
|
} else sender() ! WritingResumed
|
2013-04-09 14:22:20 +02:00
|
|
|
|
2014-09-30 10:40:48 +02:00
|
|
|
case UpdatePendingWriteAndThen(remaining, work) ⇒
|
2013-09-05 16:40:06 +02:00
|
|
|
pendingWrite = remaining
|
2014-09-30 10:40:48 +02:00
|
|
|
work()
|
2013-09-05 16:40:06 +02:00
|
|
|
if (writePending) info.registration.enableInterest(OP_WRITE)
|
|
|
|
|
|
|
|
|
|
case WriteFileFailed(e) ⇒ handleError(info.handler, e) // rethrow exception from dispatcher task
|
2013-04-02 15:17:48 +02:00
|
|
|
}
|
|
|
|
|
|
2013-01-15 18:08:45 +01:00
|
|
|
// AUXILIARIES and IMPLEMENTATION
|
|
|
|
|
|
2013-01-21 17:02:20 +01:00
|
|
|
/** used in subclasses to start the common machinery above once a channel is connected */
|
2013-05-06 17:01:01 +02:00
|
|
|
def completeConnect(registration: ChannelRegistration, commander: ActorRef,
|
|
|
|
|
options: immutable.Traversable[SocketOption]): Unit = {
|
2017-06-01 00:32:58 -07:00
|
|
|
this.registration = Some(registration)
|
|
|
|
|
|
2013-03-05 14:22:21 +01:00
|
|
|
// Turn off Nagle's algorithm by default
|
2015-01-23 15:12:13 +01:00
|
|
|
try channel.socket.setTcpNoDelay(true) catch {
|
|
|
|
|
case e: SocketException ⇒
|
|
|
|
|
// as reported in #16653 some versions of netcat (`nc -z`) doesn't allow setTcpNoDelay
|
|
|
|
|
// continue anyway
|
|
|
|
|
log.debug("Could not enable TcpNoDelay: {}", e.getMessage)
|
|
|
|
|
}
|
2015-04-30 09:23:18 +02:00
|
|
|
options.foreach(_.afterConnect(channel.socket))
|
2013-01-15 18:08:45 +01:00
|
|
|
|
|
|
|
|
commander ! Connected(
|
|
|
|
|
channel.socket.getRemoteSocketAddress.asInstanceOf[InetSocketAddress],
|
|
|
|
|
channel.socket.getLocalSocketAddress.asInstanceOf[InetSocketAddress])
|
|
|
|
|
|
2013-01-17 17:29:44 +01:00
|
|
|
context.setReceiveTimeout(RegisterTimeout)
|
2014-09-15 11:19:07 +02:00
|
|
|
|
|
|
|
|
// !!WARNING!! The line below is needed to make Windows notify us about aborted connections, see #15766
|
|
|
|
|
if (WindowsConnectionAbortWorkaroundEnabled) registration.enableInterest(OP_CONNECT)
|
|
|
|
|
|
2013-05-06 17:01:01 +02:00
|
|
|
context.become(waitingForRegistration(registration, commander))
|
2013-01-15 18:08:45 +01:00
|
|
|
}
|
|
|
|
|
|
2013-10-21 15:33:31 +02:00
|
|
|
def suspendReading(info: ConnectionInfo): Unit = {
|
|
|
|
|
readingSuspended = true
|
|
|
|
|
info.registration.disableInterest(OP_READ)
|
|
|
|
|
}
|
|
|
|
|
def resumeReading(info: ConnectionInfo): Unit = {
|
|
|
|
|
readingSuspended = false
|
|
|
|
|
info.registration.enableInterest(OP_READ)
|
2013-01-15 18:08:45 +01:00
|
|
|
}
|
|
|
|
|
|
2013-10-21 15:33:31 +02:00
|
|
|
def doRead(info: ConnectionInfo, closeCommander: Option[ActorRef]): Unit =
|
|
|
|
|
if (!readingSuspended) {
|
|
|
|
|
@tailrec def innerRead(buffer: ByteBuffer, remainingLimit: Int): ReadResult =
|
|
|
|
|
if (remainingLimit > 0) {
|
|
|
|
|
// never read more than the configured limit
|
|
|
|
|
buffer.clear()
|
|
|
|
|
val maxBufferSpace = math.min(DirectBufferSize, remainingLimit)
|
|
|
|
|
buffer.limit(maxBufferSpace)
|
|
|
|
|
val readBytes = channel.read(buffer)
|
|
|
|
|
buffer.flip()
|
|
|
|
|
|
|
|
|
|
if (TraceLogging) log.debug("Read [{}] bytes.", readBytes)
|
|
|
|
|
if (readBytes > 0) info.handler ! Received(ByteString(buffer))
|
|
|
|
|
|
|
|
|
|
readBytes match {
|
2014-07-22 16:17:45 +02:00
|
|
|
case `maxBufferSpace` ⇒ if (pullMode) MoreDataWaiting else innerRead(buffer, remainingLimit - maxBufferSpace)
|
2013-10-21 15:33:31 +02:00
|
|
|
case x if x >= 0 ⇒ AllRead
|
|
|
|
|
case -1 ⇒ EndOfStream
|
|
|
|
|
case _ ⇒
|
|
|
|
|
throw new IllegalStateException("Unexpected value returned from read: " + readBytes)
|
|
|
|
|
}
|
|
|
|
|
} else MoreDataWaiting
|
|
|
|
|
|
|
|
|
|
val buffer = bufferPool.acquire()
|
|
|
|
|
try innerRead(buffer, ReceivedMessageSizeLimit) match {
|
2014-01-07 15:50:36 +01:00
|
|
|
case AllRead ⇒
|
|
|
|
|
if (!pullMode) info.registration.enableInterest(OP_READ)
|
|
|
|
|
case MoreDataWaiting ⇒
|
|
|
|
|
if (!pullMode) self ! ChannelReadable
|
2013-10-21 15:33:31 +02:00
|
|
|
case EndOfStream if channel.socket.isOutputShutdown ⇒
|
|
|
|
|
if (TraceLogging) log.debug("Read returned end-of-stream, our side already closed")
|
|
|
|
|
doCloseConnection(info.handler, closeCommander, ConfirmedClosed)
|
|
|
|
|
case EndOfStream ⇒
|
|
|
|
|
if (TraceLogging) log.debug("Read returned end-of-stream, our side not yet closed")
|
|
|
|
|
handleClose(info, closeCommander, PeerClosed)
|
|
|
|
|
} catch {
|
|
|
|
|
case e: IOException ⇒ handleError(info.handler, e)
|
|
|
|
|
} finally bufferPool.release(buffer)
|
|
|
|
|
}
|
|
|
|
|
|
2013-05-06 17:01:01 +02:00
|
|
|
def doWrite(info: ConnectionInfo): Unit = pendingWrite = pendingWrite.doWrite(info)
|
2013-01-15 18:08:45 +01:00
|
|
|
|
|
|
|
|
def closeReason =
|
|
|
|
|
if (channel.socket.isOutputShutdown) ConfirmedClosed
|
|
|
|
|
else PeerClosed
|
|
|
|
|
|
2013-05-06 17:01:01 +02:00
|
|
|
def handleClose(info: ConnectionInfo, closeCommander: Option[ActorRef],
|
|
|
|
|
closedEvent: ConnectionClosed): Unit = closedEvent match {
|
2013-04-02 15:22:11 +02:00
|
|
|
case Aborted ⇒
|
2013-01-17 17:29:44 +01:00
|
|
|
if (TraceLogging) log.debug("Got Abort command. RESETing connection.")
|
2013-05-06 17:01:01 +02:00
|
|
|
doCloseConnection(info.handler, closeCommander, closedEvent)
|
|
|
|
|
case PeerClosed if info.keepOpenOnPeerClosed ⇒
|
2013-04-02 15:17:48 +02:00
|
|
|
// report that peer closed the connection
|
2013-05-06 17:01:01 +02:00
|
|
|
info.handler ! PeerClosed
|
2013-04-02 15:17:48 +02:00
|
|
|
// used to check if peer already closed its side later
|
2013-04-11 14:28:14 +02:00
|
|
|
peerClosed = true
|
2013-05-06 17:01:01 +02:00
|
|
|
context.become(peerSentEOF(info))
|
2013-04-02 15:22:11 +02:00
|
|
|
case _ if writePending ⇒ // finish writing first
|
2015-07-06 14:09:06 +02:00
|
|
|
// Our registered actor is now free to terminate cleanly
|
|
|
|
|
unsignDeathPact()
|
2013-01-17 17:29:44 +01:00
|
|
|
if (TraceLogging) log.debug("Got Close command but write is still pending.")
|
2013-05-06 17:01:01 +02:00
|
|
|
context.become(closingWithPendingWrite(info, closeCommander, closedEvent))
|
2013-04-02 15:22:11 +02:00
|
|
|
case ConfirmedClosed ⇒ // shutdown output and wait for confirmation
|
2013-01-17 17:29:44 +01:00
|
|
|
if (TraceLogging) log.debug("Got ConfirmedClose command, sending FIN.")
|
2013-01-15 18:08:45 +01:00
|
|
|
|
2013-10-22 17:20:24 +02:00
|
|
|
// If peer closed first, the socket is now fully closed.
|
|
|
|
|
// Also, if shutdownOutput threw an exception we expect this to be an indication
|
|
|
|
|
// that the peer closed first or concurrently with this code running.
|
|
|
|
|
// also see http://bugs.sun.com/view_bug.do?bug_id=4516760
|
|
|
|
|
if (peerClosed || !safeShutdownOutput())
|
2013-05-06 17:01:01 +02:00
|
|
|
doCloseConnection(info.handler, closeCommander, closedEvent)
|
|
|
|
|
else context.become(closing(info, closeCommander))
|
2013-04-02 15:22:11 +02:00
|
|
|
case _ ⇒ // close now
|
2013-01-17 17:29:44 +01:00
|
|
|
if (TraceLogging) log.debug("Got Close command, closing connection.")
|
2013-05-06 17:01:01 +02:00
|
|
|
doCloseConnection(info.handler, closeCommander, closedEvent)
|
2013-04-02 15:22:11 +02:00
|
|
|
}
|
2013-01-15 18:08:45 +01:00
|
|
|
|
2013-01-21 14:45:19 +01:00
|
|
|
def doCloseConnection(handler: ActorRef, closeCommander: Option[ActorRef], closedEvent: ConnectionClosed): Unit = {
|
2013-01-15 18:08:45 +01:00
|
|
|
if (closedEvent == Aborted) abort()
|
|
|
|
|
else channel.close()
|
2013-06-20 10:22:47 +02:00
|
|
|
stopWith(CloseInformation(Set(handler) ++ closeCommander, closedEvent))
|
2013-01-15 18:08:45 +01:00
|
|
|
}
|
|
|
|
|
|
2013-06-20 10:22:47 +02:00
|
|
|
def handleError(handler: ActorRef, exception: IOException): Unit = {
|
|
|
|
|
log.debug("Closing connection due to IO error {}", exception)
|
|
|
|
|
stopWith(CloseInformation(Set(handler), ErrorClosed(extractMsg(exception))))
|
2013-01-15 18:08:45 +01:00
|
|
|
}
|
2013-10-22 17:20:24 +02:00
|
|
|
def safeShutdownOutput(): Boolean =
|
|
|
|
|
try {
|
|
|
|
|
channel.socket().shutdownOutput()
|
|
|
|
|
true
|
|
|
|
|
} catch {
|
|
|
|
|
case _: SocketException ⇒ false
|
|
|
|
|
}
|
2013-05-06 17:01:01 +02:00
|
|
|
|
2013-01-17 14:45:50 +01:00
|
|
|
@tailrec private[this] def extractMsg(t: Throwable): String =
|
|
|
|
|
if (t == null) "unknown"
|
|
|
|
|
else {
|
|
|
|
|
t.getMessage match {
|
|
|
|
|
case null | "" ⇒ extractMsg(t.getCause)
|
|
|
|
|
case msg ⇒ msg
|
|
|
|
|
}
|
|
|
|
|
}
|
2013-01-15 18:08:45 +01:00
|
|
|
|
|
|
|
|
def abort(): Unit = {
|
|
|
|
|
try channel.socket.setSoLinger(true, 0) // causes the following close() to send TCP RST
|
|
|
|
|
catch {
|
|
|
|
|
case NonFatal(e) ⇒
|
|
|
|
|
// setSoLinger can fail due to http://bugs.sun.com/view_bug.do?bug_id=6799574
|
|
|
|
|
// (also affected: OS/X Java 1.6.0_37)
|
2013-02-10 13:52:52 +01:00
|
|
|
if (TraceLogging) log.debug("setSoLinger(true, 0) failed with [{}]", e)
|
2013-01-15 18:08:45 +01:00
|
|
|
}
|
|
|
|
|
channel.close()
|
2017-06-01 00:32:58 -07:00
|
|
|
|
|
|
|
|
// On linux, closing the channel directly triggers a RST as a side effect of `preClose`
|
|
|
|
|
// called from `sun.nio.ch.SocketChannelImpl#implCloseSelectableChannel`.
|
|
|
|
|
|
|
|
|
|
// On windows, however, the connection is merely added to the `cancelledKeys` of the `java.nio.channels.spi.AbstractSelector`,
|
|
|
|
|
// and `sun.nio.ch.SelectorImpl` will kill those from `processDeregisterQueue` after the select poll has returned.
|
|
|
|
|
|
|
|
|
|
// We don't want to have to wait for that, hence explicitly triggering the cancellation:
|
|
|
|
|
registration.foreach(_.cancel())
|
2013-01-15 18:08:45 +01:00
|
|
|
}
|
|
|
|
|
|
2013-06-20 10:22:47 +02:00
|
|
|
def stopWith(closeInfo: CloseInformation): Unit = {
|
|
|
|
|
closedMessage = closeInfo
|
|
|
|
|
context.stop(self)
|
|
|
|
|
}
|
|
|
|
|
|
2013-01-17 14:45:50 +01:00
|
|
|
override def postStop(): Unit = {
|
2013-01-22 17:32:46 +01:00
|
|
|
if (channel.isOpen)
|
|
|
|
|
abort()
|
|
|
|
|
|
2013-04-09 12:25:04 +02:00
|
|
|
if (writePending) pendingWrite.release()
|
2013-01-22 17:32:46 +01:00
|
|
|
|
2013-01-17 14:45:50 +01:00
|
|
|
if (closedMessage != null) {
|
2013-01-21 14:45:19 +01:00
|
|
|
val interestedInClose =
|
|
|
|
|
if (writePending) closedMessage.notificationsTo + pendingWrite.commander
|
|
|
|
|
else closedMessage.notificationsTo
|
2013-01-17 14:45:50 +01:00
|
|
|
|
2013-01-21 14:45:19 +01:00
|
|
|
interestedInClose.foreach(_ ! closedMessage.closedEvent)
|
2013-01-17 14:45:50 +01:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def postRestart(reason: Throwable): Unit =
|
|
|
|
|
throw new IllegalStateException("Restarting not supported for connection actors.")
|
2013-01-15 18:08:45 +01:00
|
|
|
|
2013-09-05 16:40:06 +02:00
|
|
|
def PendingWrite(commander: ActorRef, write: WriteCommand): PendingWrite = {
|
|
|
|
|
@tailrec def create(head: WriteCommand, tail: WriteCommand = Write.empty): PendingWrite =
|
|
|
|
|
head match {
|
|
|
|
|
case Write.empty ⇒ if (tail eq Write.empty) EmptyPendingWrite else create(tail)
|
|
|
|
|
case Write(data, ack) if data.nonEmpty ⇒ PendingBufferWrite(commander, data, ack, tail)
|
|
|
|
|
case WriteFile(path, offset, count, ack) ⇒ PendingWriteFile(commander, path, offset, count, ack, tail)
|
|
|
|
|
case CompoundWrite(h, t) ⇒ create(h, t)
|
|
|
|
|
case x @ Write(_, ack) ⇒ // empty write with either an ACK or a non-standard NoACK
|
|
|
|
|
if (x.wantsAck) commander ! ack
|
|
|
|
|
create(tail)
|
2013-04-15 14:54:46 +02:00
|
|
|
}
|
2013-09-05 16:40:06 +02:00
|
|
|
create(write)
|
2013-04-09 14:22:20 +02:00
|
|
|
}
|
|
|
|
|
|
2013-09-05 16:40:06 +02:00
|
|
|
def PendingBufferWrite(commander: ActorRef, data: ByteString, ack: Event, tail: WriteCommand): PendingBufferWrite = {
|
|
|
|
|
val buffer = bufferPool.acquire()
|
|
|
|
|
try {
|
|
|
|
|
val copied = data.copyToBuffer(buffer)
|
|
|
|
|
buffer.flip()
|
|
|
|
|
new PendingBufferWrite(commander, data.drop(copied), ack, buffer, tail)
|
|
|
|
|
} catch {
|
|
|
|
|
case NonFatal(e) ⇒
|
|
|
|
|
bufferPool.release(buffer)
|
|
|
|
|
throw e
|
|
|
|
|
}
|
|
|
|
|
}
|
2013-04-09 12:25:04 +02:00
|
|
|
|
2013-09-05 16:40:06 +02:00
|
|
|
class PendingBufferWrite(
|
|
|
|
|
val commander: ActorRef,
|
|
|
|
|
remainingData: ByteString,
|
2016-06-02 14:06:57 +02:00
|
|
|
ack: Any,
|
|
|
|
|
buffer: ByteBuffer,
|
|
|
|
|
tail: WriteCommand) extends PendingWrite {
|
2013-04-09 12:25:04 +02:00
|
|
|
|
2013-05-06 17:01:01 +02:00
|
|
|
def doWrite(info: ConnectionInfo): PendingWrite = {
|
2013-09-05 16:40:06 +02:00
|
|
|
@tailrec def writeToChannel(data: ByteString): PendingWrite = {
|
|
|
|
|
val writtenBytes = channel.write(buffer) // at first we try to drain the remaining bytes from the buffer
|
2013-04-09 12:25:04 +02:00
|
|
|
if (TraceLogging) log.debug("Wrote [{}] bytes to channel", writtenBytes)
|
2013-09-05 16:40:06 +02:00
|
|
|
if (buffer.hasRemaining) {
|
|
|
|
|
// we weren't able to write all bytes from the buffer, so we need to try again later
|
|
|
|
|
if (data eq remainingData) this
|
|
|
|
|
else new PendingBufferWrite(commander, data, ack, buffer, tail) // copy with updated remainingData
|
|
|
|
|
|
|
|
|
|
} else if (data.nonEmpty) {
|
|
|
|
|
buffer.clear()
|
2013-10-17 14:10:00 +02:00
|
|
|
val copied = data.copyToBuffer(buffer)
|
2013-09-05 16:40:06 +02:00
|
|
|
buffer.flip()
|
2013-10-17 14:10:00 +02:00
|
|
|
writeToChannel(data drop copied)
|
2013-09-05 16:40:06 +02:00
|
|
|
|
|
|
|
|
} else {
|
|
|
|
|
if (!ack.isInstanceOf[NoAck]) commander ! ack
|
|
|
|
|
release()
|
|
|
|
|
PendingWrite(commander, tail)
|
2013-04-09 12:25:04 +02:00
|
|
|
}
|
|
|
|
|
}
|
2013-09-05 16:40:06 +02:00
|
|
|
try {
|
|
|
|
|
val next = writeToChannel(remainingData)
|
|
|
|
|
if (next ne EmptyPendingWrite) info.registration.enableInterest(OP_WRITE)
|
|
|
|
|
next
|
|
|
|
|
} catch { case e: IOException ⇒ handleError(info.handler, e); this }
|
2013-04-09 12:25:04 +02:00
|
|
|
}
|
2013-09-05 16:40:06 +02:00
|
|
|
|
|
|
|
|
def release(): Unit = bufferPool.release(buffer)
|
2013-01-17 14:31:35 +01:00
|
|
|
}
|
|
|
|
|
|
2013-09-05 16:40:06 +02:00
|
|
|
def PendingWriteFile(commander: ActorRef, filePath: String, offset: Long, count: Long, ack: Event,
|
|
|
|
|
tail: WriteCommand): PendingWriteFile =
|
2017-04-19 19:05:19 +02:00
|
|
|
new PendingWriteFile(commander, FileChannel.open(Paths.get(filePath)), offset, count, ack, tail)
|
2013-09-05 16:40:06 +02:00
|
|
|
|
|
|
|
|
class PendingWriteFile(
|
|
|
|
|
val commander: ActorRef,
|
2016-06-02 14:06:57 +02:00
|
|
|
fileChannel: FileChannel,
|
|
|
|
|
offset: Long,
|
|
|
|
|
remaining: Long,
|
|
|
|
|
ack: Event,
|
|
|
|
|
tail: WriteCommand) extends PendingWrite with Runnable {
|
2013-04-10 14:56:51 +02:00
|
|
|
|
2013-05-06 17:01:01 +02:00
|
|
|
def doWrite(info: ConnectionInfo): PendingWrite = {
|
2013-09-05 16:40:06 +02:00
|
|
|
tcp.fileIoDispatcher.execute(this)
|
2013-04-09 14:22:20 +02:00
|
|
|
this
|
|
|
|
|
}
|
2013-01-17 14:31:35 +01:00
|
|
|
|
2013-09-05 16:40:06 +02:00
|
|
|
def release(): Unit = fileChannel.close()
|
2013-04-09 14:22:20 +02:00
|
|
|
|
2013-09-05 16:40:06 +02:00
|
|
|
def run(): Unit =
|
|
|
|
|
try {
|
|
|
|
|
val toWrite = math.min(remaining, tcp.Settings.TransferToLimit)
|
|
|
|
|
val written = fileChannel.transferTo(offset, toWrite, channel)
|
2013-04-09 14:22:20 +02:00
|
|
|
|
2013-09-05 16:40:06 +02:00
|
|
|
if (written < remaining) {
|
|
|
|
|
val updated = new PendingWriteFile(commander, fileChannel, offset + written, remaining - written, ack, tail)
|
2014-09-30 10:40:48 +02:00
|
|
|
self ! UpdatePendingWriteAndThen(updated, TcpConnection.doNothing)
|
2013-09-05 16:40:06 +02:00
|
|
|
} else {
|
|
|
|
|
release()
|
2014-09-30 10:40:48 +02:00
|
|
|
val andThen = if (!ack.isInstanceOf[NoAck]) () ⇒ commander ! ack else doNothing
|
|
|
|
|
self ! UpdatePendingWriteAndThen(PendingWrite(commander, tail), andThen)
|
2013-04-09 14:22:20 +02:00
|
|
|
}
|
2013-04-15 15:03:02 +02:00
|
|
|
} catch {
|
|
|
|
|
case e: IOException ⇒ self ! WriteFileFailed(e)
|
2013-04-09 14:22:20 +02:00
|
|
|
}
|
2013-09-05 16:40:06 +02:00
|
|
|
}
|
2013-01-24 15:08:42 +01:00
|
|
|
}
|
|
|
|
|
|
2013-02-15 11:59:01 +01:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
2013-01-24 15:08:42 +01:00
|
|
|
private[io] object TcpConnection {
|
|
|
|
|
sealed trait ReadResult
|
|
|
|
|
object EndOfStream extends ReadResult
|
2013-04-03 16:10:49 +02:00
|
|
|
object AllRead extends ReadResult
|
|
|
|
|
object MoreDataWaiting extends ReadResult
|
2013-01-21 14:45:19 +01:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Used to transport information to the postStop method to notify
|
|
|
|
|
* interested party about a connection close.
|
|
|
|
|
*/
|
2014-03-07 13:20:01 +01:00
|
|
|
final case class CloseInformation(notificationsTo: Set[ActorRef], closedEvent: Event)
|
2013-05-06 17:01:01 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Groups required connection-related data that are only available once the connection has been fully established.
|
|
|
|
|
*/
|
2016-06-02 14:06:57 +02:00
|
|
|
final case class ConnectionInfo(
|
|
|
|
|
registration: ChannelRegistration,
|
|
|
|
|
handler: ActorRef,
|
|
|
|
|
keepOpenOnPeerClosed: Boolean,
|
|
|
|
|
useResumeWriting: Boolean)
|
2013-04-09 12:25:04 +02:00
|
|
|
|
2013-04-09 14:22:20 +02:00
|
|
|
// INTERNAL MESSAGES
|
|
|
|
|
|
2014-09-30 10:40:48 +02:00
|
|
|
final case class UpdatePendingWriteAndThen(remainingWrite: PendingWrite, work: () ⇒ Unit) extends NoSerializationVerificationNeeded
|
2014-03-07 13:20:01 +01:00
|
|
|
final case class WriteFileFailed(e: IOException)
|
2013-04-09 14:22:20 +02:00
|
|
|
|
2013-09-05 16:40:06 +02:00
|
|
|
sealed abstract class PendingWrite {
|
2013-04-09 12:25:04 +02:00
|
|
|
def commander: ActorRef
|
2013-05-06 17:01:01 +02:00
|
|
|
def doWrite(info: ConnectionInfo): PendingWrite
|
2013-09-05 16:40:06 +02:00
|
|
|
def release(): Unit // free any occupied resources
|
|
|
|
|
}
|
2013-04-09 12:25:04 +02:00
|
|
|
|
2013-09-05 16:40:06 +02:00
|
|
|
object EmptyPendingWrite extends PendingWrite {
|
|
|
|
|
def commander: ActorRef = throw new IllegalStateException
|
|
|
|
|
def doWrite(info: ConnectionInfo): PendingWrite = throw new IllegalStateException
|
|
|
|
|
def release(): Unit = throw new IllegalStateException
|
2013-04-09 12:25:04 +02:00
|
|
|
}
|
2014-09-30 10:40:48 +02:00
|
|
|
|
|
|
|
|
val doNothing: () ⇒ Unit = () ⇒ ()
|
2017-05-16 15:22:11 +02:00
|
|
|
|
|
|
|
|
val DroppingWriteBecauseWritingIsSuspendedException =
|
|
|
|
|
new IOException("Dropping write because writing is suspended") with NoStackTrace
|
|
|
|
|
|
|
|
|
|
val DroppingWriteBecauseQueueIsFullException =
|
|
|
|
|
new IOException("Dropping write because queue is full") with NoStackTrace
|
2013-01-22 15:51:21 +01:00
|
|
|
}
|