Use long uid in artery remoting and cluster #20644

This commit is contained in:
Johan Andrén 2016-09-26 15:34:59 +02:00 committed by GitHub
parent e9c4393f7b
commit 8ae0c9a888
37 changed files with 932 additions and 151 deletions

View file

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