Use long uid in artery remoting and cluster #20644
This commit is contained in:
parent
e9c4393f7b
commit
8ae0c9a888
37 changed files with 932 additions and 151 deletions
|
|
@ -8,6 +8,7 @@ import akka.actor._
|
|||
import akka.dispatch.sysmsg.{ DeathWatchNotification, Watch }
|
||||
import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
|
||||
import akka.event.AddressTerminatedTopic
|
||||
import akka.remote.artery.ArteryMessage
|
||||
|
||||
import scala.collection.mutable
|
||||
import scala.concurrent.duration._
|
||||
|
|
@ -34,6 +35,10 @@ private[akka] object RemoteWatcher {
|
|||
@SerialVersionUID(1L) case object Heartbeat extends HeartbeatMessage
|
||||
@SerialVersionUID(1L) final case class HeartbeatRsp(addressUid: Int) extends HeartbeatMessage
|
||||
|
||||
// specific pair of messages for artery to allow for protobuf serialization and long uid
|
||||
case object ArteryHeartbeat extends HeartbeatMessage with ArteryMessage
|
||||
final case class ArteryHeartbeatRsp(uid: Long) extends HeartbeatMessage with ArteryMessage
|
||||
|
||||
// sent to self only
|
||||
case object HeartbeatTick
|
||||
case object ReapUnreachableTick
|
||||
|
|
@ -89,13 +94,12 @@ private[akka] class RemoteWatcher(
|
|||
import context.dispatcher
|
||||
def scheduler = context.system.scheduler
|
||||
|
||||
val remoteProvider: RemoteActorRefProvider = context.system.asInstanceOf[ExtendedActorSystem].provider match {
|
||||
case rarp: RemoteActorRefProvider ⇒ rarp
|
||||
case other ⇒ throw new ConfigurationException(
|
||||
s"ActorSystem [${context.system}] needs to have a 'RemoteActorRefProvider' enabled in the configuration, currently uses [${other.getClass.getName}]")
|
||||
}
|
||||
val remoteProvider: RemoteActorRefProvider = RARP(context.system).provider
|
||||
val artery = remoteProvider.remoteSettings.Artery.Enabled
|
||||
|
||||
val selfHeartbeatRspMsg = HeartbeatRsp(AddressUidExtension(context.system).addressUid)
|
||||
val (heartBeatMsg, selfHeartbeatRspMsg) =
|
||||
if (artery) (ArteryHeartbeat, ArteryHeartbeatRsp(AddressUidExtension(context.system).longAddressUid))
|
||||
else (Heartbeat, HeartbeatRsp(AddressUidExtension(context.system).addressUid))
|
||||
|
||||
// actors that this node is watching, map of watchee -> Set(watchers)
|
||||
val watching = new mutable.HashMap[InternalActorRef, mutable.Set[InternalActorRef]]() with mutable.MultiMap[InternalActorRef, InternalActorRef]
|
||||
|
|
@ -105,7 +109,7 @@ private[akka] class RemoteWatcher(
|
|||
def watchingNodes = watcheeByNodes.keySet
|
||||
|
||||
var unreachable: Set[Address] = Set.empty
|
||||
var addressUids: Map[Address, Int] = Map.empty
|
||||
var addressUids: Map[Address, Long] = Map.empty
|
||||
|
||||
val heartbeatTask = scheduler.schedule(heartbeatInterval, heartbeatInterval, self, HeartbeatTick)
|
||||
val failureDetectorReaperTask = scheduler.schedule(unreachableReaperInterval, unreachableReaperInterval,
|
||||
|
|
@ -119,8 +123,9 @@ private[akka] class RemoteWatcher(
|
|||
|
||||
def receive = {
|
||||
case HeartbeatTick ⇒ sendHeartbeat()
|
||||
case Heartbeat ⇒ receiveHeartbeat()
|
||||
case HeartbeatRsp(uid) ⇒ receiveHeartbeatRsp(uid)
|
||||
case Heartbeat | ArteryHeartbeat ⇒ receiveHeartbeat()
|
||||
case HeartbeatRsp(uid) ⇒ receiveHeartbeatRsp(uid.toLong)
|
||||
case ArteryHeartbeatRsp(uid) ⇒ receiveHeartbeatRsp(uid)
|
||||
case ReapUnreachableTick ⇒ reapUnreachable()
|
||||
case ExpectedFirstHeartbeat(from) ⇒ triggerFirstHeartbeat(from)
|
||||
case WatchRemote(watchee, watcher) ⇒ addWatch(watchee, watcher)
|
||||
|
|
@ -138,7 +143,7 @@ private[akka] class RemoteWatcher(
|
|||
def receiveHeartbeat(): Unit =
|
||||
sender() ! selfHeartbeatRspMsg
|
||||
|
||||
def receiveHeartbeatRsp(uid: Int): Unit = {
|
||||
def receiveHeartbeatRsp(uid: Long): Unit = {
|
||||
val from = sender().path.address
|
||||
|
||||
if (failureDetector.isMonitoring(from))
|
||||
|
|
@ -167,7 +172,7 @@ private[akka] class RemoteWatcher(
|
|||
def publishAddressTerminated(address: Address): Unit =
|
||||
AddressTerminatedTopic(context.system).publish(AddressTerminated(address))
|
||||
|
||||
def quarantine(address: Address, uid: Option[Int], reason: String): Unit =
|
||||
def quarantine(address: Address, uid: Option[Long], reason: String): Unit =
|
||||
remoteProvider.quarantine(address, uid, reason)
|
||||
|
||||
def addWatch(watchee: InternalActorRef, watcher: InternalActorRef): Unit = {
|
||||
|
|
@ -256,7 +261,7 @@ private[akka] class RemoteWatcher(
|
|||
// other side a chance to reply, and also trigger some resends if needed
|
||||
scheduler.scheduleOnce(heartbeatExpectedResponseAfter, self, ExpectedFirstHeartbeat(a))
|
||||
}
|
||||
context.actorSelection(RootActorPath(a) / self.path.elements) ! Heartbeat
|
||||
context.actorSelection(RootActorPath(a) / self.path.elements) ! heartBeatMsg
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue