Rolling update config checker, #24009
* adds config compatibility check * doc'ed what happens when joining a cluster not supporting this feature * added extra docs over sensitive paths
This commit is contained in:
parent
a4e9881a6f
commit
c83e4adfea
19 changed files with 4832 additions and 123 deletions
|
|
@ -10,9 +10,12 @@ import akka.cluster.MemberStatus._
|
|||
import akka.cluster.ClusterEvent._
|
||||
import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
|
||||
import akka.Done
|
||||
import akka.actor.CoordinatedShutdown.Reason
|
||||
import akka.cluster.ClusterUserAction.JoinTo
|
||||
import akka.pattern.ask
|
||||
import akka.remote.QuarantinedEvent
|
||||
import akka.util.Timeout
|
||||
import com.typesafe.config.{ Config, ConfigFactory }
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration._
|
||||
|
|
@ -32,6 +35,7 @@ trait ClusterMessage extends Serializable
|
|||
* [[akka.cluster.Cluster]] extension
|
||||
* or JMX.
|
||||
*/
|
||||
@InternalApi
|
||||
private[cluster] object ClusterUserAction {
|
||||
|
||||
/**
|
||||
|
|
@ -58,6 +62,7 @@ private[cluster] object ClusterUserAction {
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi
|
||||
private[cluster] object InternalClusterAction {
|
||||
|
||||
/**
|
||||
|
|
@ -92,17 +97,22 @@ private[cluster] object InternalClusterAction {
|
|||
*/
|
||||
case object JoinSeedNode extends DeadLetterSuppression
|
||||
|
||||
/**
|
||||
* see JoinSeedNode
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
case object InitJoin extends ClusterMessage with DeadLetterSuppression
|
||||
sealed trait ConfigCheck
|
||||
case object UncheckedConfig extends ConfigCheck
|
||||
case object IncompatibleConfig extends ConfigCheck
|
||||
final case class CompatibleConfig(clusterConfig: Config) extends ConfigCheck
|
||||
|
||||
/**
|
||||
* see JoinSeedNode
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final case class InitJoinAck(address: Address) extends ClusterMessage with DeadLetterSuppression
|
||||
case class InitJoin(configOfJoiningNode: Config) extends ClusterMessage with DeadLetterSuppression
|
||||
|
||||
/**
|
||||
* see JoinSeedNode
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final case class InitJoinAck(address: Address, configCheck: ConfigCheck) extends ClusterMessage with DeadLetterSuppression
|
||||
|
||||
/**
|
||||
* see JoinSeedNode
|
||||
|
|
@ -162,7 +172,8 @@ private[cluster] object InternalClusterAction {
|
|||
*
|
||||
* Supervisor managing the different Cluster daemons.
|
||||
*/
|
||||
private[cluster] final class ClusterDaemon(settings: ClusterSettings) extends Actor with ActorLogging
|
||||
@InternalApi
|
||||
private[cluster] final class ClusterDaemon(settings: ClusterSettings, joinConfigCompatChecker: JoinConfigCompatChecker) extends Actor with ActorLogging
|
||||
with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
|
||||
import InternalClusterAction._
|
||||
// Important - don't use Cluster(context.system) in constructor because that would
|
||||
|
|
@ -196,7 +207,7 @@ private[cluster] final class ClusterDaemon(settings: ClusterSettings) extends Ac
|
|||
}
|
||||
|
||||
def createChildren(): Unit = {
|
||||
coreSupervisor = Some(context.actorOf(Props[ClusterCoreSupervisor].
|
||||
coreSupervisor = Some(context.actorOf(Props(classOf[ClusterCoreSupervisor], joinConfigCompatChecker).
|
||||
withDispatcher(context.props.dispatcher), name = "core"))
|
||||
context.actorOf(Props[ClusterHeartbeatReceiver].
|
||||
withDispatcher(context.props.dispatcher), name = "heartbeatReceiver")
|
||||
|
|
@ -225,7 +236,8 @@ private[cluster] final class ClusterDaemon(settings: ClusterSettings) extends Ac
|
|||
* ClusterCoreDaemon and ClusterDomainEventPublisher can't be restarted because the state
|
||||
* would be obsolete. Shutdown the member if any those actors crashed.
|
||||
*/
|
||||
private[cluster] final class ClusterCoreSupervisor extends Actor with ActorLogging
|
||||
@InternalApi
|
||||
private[cluster] final class ClusterCoreSupervisor(joinConfigCompatChecker: JoinConfigCompatChecker) extends Actor with ActorLogging
|
||||
with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
|
||||
|
||||
// Important - don't use Cluster(context.system) in constructor because that would
|
||||
|
|
@ -238,7 +250,7 @@ private[cluster] final class ClusterCoreSupervisor extends Actor with ActorLoggi
|
|||
def createChildren(): Unit = {
|
||||
val publisher = context.actorOf(Props[ClusterDomainEventPublisher].
|
||||
withDispatcher(context.props.dispatcher), name = "publisher")
|
||||
coreDaemon = Some(context.watch(context.actorOf(Props(classOf[ClusterCoreDaemon], publisher).
|
||||
coreDaemon = Some(context.watch(context.actorOf(Props(classOf[ClusterCoreDaemon], publisher, joinConfigCompatChecker).
|
||||
withDispatcher(context.props.dispatcher), name = "daemon")))
|
||||
}
|
||||
|
||||
|
|
@ -274,7 +286,7 @@ private[cluster] object ClusterCoreDaemon {
|
|||
* INTERNAL API.
|
||||
*/
|
||||
@InternalApi
|
||||
private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with ActorLogging
|
||||
private[cluster] class ClusterCoreDaemon(publisher: ActorRef, joinConfigCompatChecker: JoinConfigCompatChecker) extends Actor with ActorLogging
|
||||
with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
|
||||
import InternalClusterAction._
|
||||
import ClusterCoreDaemon._
|
||||
|
|
@ -475,9 +487,9 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with
|
|||
case ReapUnreachableTick ⇒ reapUnreachableMembers()
|
||||
case LeaderActionsTick ⇒ leaderActions()
|
||||
case PublishStatsTick ⇒ publishInternalStats()
|
||||
case InitJoin ⇒
|
||||
case InitJoin(joiningNodeConfig) ⇒
|
||||
logInfo("Received InitJoin message from [{}] to [{}]", sender(), selfAddress)
|
||||
initJoin()
|
||||
initJoin(joiningNodeConfig)
|
||||
case Join(node, roles) ⇒ joining(node, roles)
|
||||
case ClusterUserAction.Down(address) ⇒ downing(address)
|
||||
case ClusterUserAction.Leave(address) ⇒ leaving(address)
|
||||
|
|
@ -509,7 +521,7 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with
|
|||
case other ⇒ super.unhandled(other)
|
||||
}
|
||||
|
||||
def initJoin(): Unit = {
|
||||
def initJoin(joiningNodeConfig: Config): Unit = {
|
||||
val selfStatus = latestGossip.member(selfUniqueAddress).status
|
||||
if (removeUnreachableWithMemberStatus.contains(selfStatus)) {
|
||||
// prevents a Down and Exiting node from being used for joining
|
||||
|
|
@ -517,7 +529,30 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with
|
|||
sender() ! InitJoinNack(selfAddress)
|
||||
} else {
|
||||
logInfo("Sending InitJoinAck message from node [{}] to [{}]", selfAddress, sender())
|
||||
sender() ! InitJoinAck(selfAddress)
|
||||
// run config compatibility check using config provided by
|
||||
// joining node and current (full) config on cluster side
|
||||
|
||||
val configWithoutSensitiveKeys = {
|
||||
val allowedConfigPaths = JoinConfigCompatChecker.removeSensitiveKeys(context.system.settings.config, cluster.settings)
|
||||
// build a stripped down config instead where sensitive config paths are removed
|
||||
// we don't want any check to happen on those keys
|
||||
JoinConfigCompatChecker.filterWithKeys(allowedConfigPaths, context.system.settings.config)
|
||||
}
|
||||
|
||||
joinConfigCompatChecker.check(joiningNodeConfig, configWithoutSensitiveKeys) match {
|
||||
case Valid ⇒
|
||||
val nonSensitiveKeys = JoinConfigCompatChecker.removeSensitiveKeys(joiningNodeConfig, cluster.settings)
|
||||
// Send back to joining node a subset of current configuration
|
||||
// containing the keys initially sent by the joining node minus
|
||||
// any sensitive keys as defined by this node configuration
|
||||
val clusterConfig = JoinConfigCompatChecker.filterWithKeys(nonSensitiveKeys, context.system.settings.config)
|
||||
sender() ! InitJoinAck(selfAddress, CompatibleConfig(clusterConfig))
|
||||
case Invalid(messages) ⇒
|
||||
// messages are only logged on the cluster side
|
||||
log.warning("Found incompatible settings when [{}] tried to join: {}", sender().path.address, messages.mkString(", "))
|
||||
sender() ! InitJoinAck(selfAddress, IncompatibleConfig)
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -533,10 +568,10 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with
|
|||
// use unique name of this actor, stopSeedNodeProcess doesn't wait for termination
|
||||
seedNodeProcessCounter += 1
|
||||
if (newSeedNodes.head == selfAddress) {
|
||||
Some(context.actorOf(Props(classOf[FirstSeedNodeProcess], newSeedNodes).
|
||||
Some(context.actorOf(Props(classOf[FirstSeedNodeProcess], newSeedNodes, joinConfigCompatChecker).
|
||||
withDispatcher(UseDispatcher), name = "firstSeedNodeProcess-" + seedNodeProcessCounter))
|
||||
} else {
|
||||
Some(context.actorOf(Props(classOf[JoinSeedNodeProcess], newSeedNodes).
|
||||
Some(context.actorOf(Props(classOf[JoinSeedNodeProcess], newSeedNodes, joinConfigCompatChecker).
|
||||
withDispatcher(UseDispatcher), name = "joinSeedNodeProcess-" + seedNodeProcessCounter))
|
||||
}
|
||||
}
|
||||
|
|
@ -949,7 +984,7 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with
|
|||
if (latestGossip.isMultiDc)
|
||||
latestGossip.overview.seen.count(membershipState.isInSameDc) < latestGossip.members.count(_.dataCenter == cluster.selfDataCenter) / 2
|
||||
else
|
||||
(latestGossip.overview.seen.size < latestGossip.members.size / 2)
|
||||
latestGossip.overview.seen.size < latestGossip.members.size / 2
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -1274,7 +1309,7 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with
|
|||
versionedGossip.clearSeen()
|
||||
else {
|
||||
// Nobody else has seen this gossip but us
|
||||
val seenVersionedGossip = versionedGossip onlySeen (selfUniqueAddress)
|
||||
val seenVersionedGossip = versionedGossip onlySeen selfUniqueAddress
|
||||
// Update the state with the new gossip
|
||||
seenVersionedGossip
|
||||
}
|
||||
|
|
@ -1284,7 +1319,7 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with
|
|||
|
||||
def assertLatestGossip(): Unit =
|
||||
if (Cluster.isAssertInvariantsEnabled && latestGossip.version.versions.size > latestGossip.members.size)
|
||||
throw new IllegalStateException(s"Too many vector clock entries in gossip state ${latestGossip}")
|
||||
throw new IllegalStateException(s"Too many vector clock entries in gossip state $latestGossip")
|
||||
|
||||
def publishMembershipState(): Unit = {
|
||||
if (cluster.settings.Debug.VerboseGossipLogging)
|
||||
|
|
@ -1303,6 +1338,11 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with
|
|||
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*/
|
||||
private[cluster] case object IncompatibleConfigurationDetected extends Reason
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*
|
||||
|
|
@ -1315,11 +1355,13 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with
|
|||
* it will reply with InitJoinAck and then the first seed node will join
|
||||
* that other seed node to join existing cluster.
|
||||
*/
|
||||
private[cluster] final class FirstSeedNodeProcess(seedNodes: immutable.IndexedSeq[Address]) extends Actor with ActorLogging {
|
||||
@InternalApi
|
||||
private[cluster] final class FirstSeedNodeProcess(seedNodes: immutable.IndexedSeq[Address], joinConfigCompatChecker: JoinConfigCompatChecker) extends Actor with ActorLogging {
|
||||
import InternalClusterAction._
|
||||
import ClusterUserAction.JoinTo
|
||||
|
||||
val cluster = Cluster(context.system)
|
||||
import cluster.settings._
|
||||
import cluster.InfoLogger._
|
||||
|
||||
def selfAddress = cluster.selfAddress
|
||||
|
|
@ -1341,8 +1383,11 @@ private[cluster] final class FirstSeedNodeProcess(seedNodes: immutable.IndexedSe
|
|||
def receive = {
|
||||
case JoinSeedNode ⇒
|
||||
if (timeout.hasTimeLeft) {
|
||||
val requiredNonSensitiveKeys = JoinConfigCompatChecker.removeSensitiveKeys(joinConfigCompatChecker.requiredKeys, cluster.settings)
|
||||
// configToValidate only contains the keys that are required according to JoinConfigCompatChecker on this node
|
||||
val configToValidate = JoinConfigCompatChecker.filterWithKeys(requiredNonSensitiveKeys, context.system.settings.config)
|
||||
// send InitJoin to remaining seed nodes (except myself)
|
||||
remainingSeedNodes foreach { a ⇒ context.actorSelection(context.parent.path.toStringWithAddress(a)) ! InitJoin }
|
||||
remainingSeedNodes foreach { a ⇒ context.actorSelection(context.parent.path.toStringWithAddress(a)) ! InitJoin(configToValidate) }
|
||||
} else {
|
||||
// no InitJoinAck received, initialize new cluster by joining myself
|
||||
if (log.isDebugEnabled)
|
||||
|
|
@ -1352,11 +1397,60 @@ private[cluster] final class FirstSeedNodeProcess(seedNodes: immutable.IndexedSe
|
|||
context.parent ! JoinTo(selfAddress)
|
||||
context.stop(self)
|
||||
}
|
||||
case InitJoinAck(address) ⇒
|
||||
|
||||
case InitJoinAck(address, CompatibleConfig(clusterConfig)) ⇒
|
||||
logInfo("Received InitJoinAck message from [{}] to [{}]", sender(), selfAddress)
|
||||
// first InitJoinAck reply, join existing cluster
|
||||
// validates config coming from cluster against this node config
|
||||
joinConfigCompatChecker.check(clusterConfig, context.system.settings.config) match {
|
||||
case Valid ⇒
|
||||
// first InitJoinAck reply
|
||||
context.parent ! JoinTo(address)
|
||||
context.stop(self)
|
||||
|
||||
case Invalid(messages) if ByPassConfigCompatCheck ⇒
|
||||
log.warning("Cluster validated this node config, but sent back incompatible settings: {}. " +
|
||||
"Join will be performed because compatibility check is configured to not be enforced.", messages.mkString(", "))
|
||||
context.parent ! JoinTo(address)
|
||||
context.stop(self)
|
||||
|
||||
case Invalid(messages) ⇒
|
||||
log.error("Cluster validated this node config, but sent back incompatible settings: {}. " +
|
||||
"It's recommended to perform a full cluster shutdown in order to deploy this new version. " +
|
||||
"If a cluster shutdown isn't an option, you may want to disable this protection by setting " +
|
||||
"'akka.cluster.configuration-compatibility-check.enforce-on-join = off'. " +
|
||||
"Note that disabling it will allow the formation of a cluster with nodes having incompatible configuration settings. " +
|
||||
"This node will be shutdown!", messages.mkString(", "))
|
||||
context.stop(self)
|
||||
CoordinatedShutdown(context.system).run(IncompatibleConfigurationDetected)
|
||||
}
|
||||
|
||||
case InitJoinAck(address, UncheckedConfig) ⇒
|
||||
logInfo("Received InitJoinAck message from [{}] to [{}]", sender(), selfAddress)
|
||||
log.warning("Joining a cluster without configuration compatibility check feature.")
|
||||
context.parent ! JoinTo(address)
|
||||
context.stop(self)
|
||||
|
||||
case InitJoinAck(address, IncompatibleConfig) ⇒
|
||||
// first InitJoinAck reply, but incompatible
|
||||
if (ByPassConfigCompatCheck) {
|
||||
// only join if set to ignore config validation
|
||||
logInfo("Received InitJoinAck message from [{}] to [{}]", sender(), selfAddress)
|
||||
log.warning("Joining cluster with incompatible configurations. " +
|
||||
"Join will be performed because compatibility check is configured to not be enforced.")
|
||||
context.parent ! JoinTo(address)
|
||||
context.stop(self)
|
||||
} else {
|
||||
log.error(
|
||||
"Couldn't join seed nodes because of incompatible cluster configuration. " +
|
||||
"It's recommended to perform a full cluster shutdown in order to deploy this new version." +
|
||||
"If a cluster shutdown isn't an option, you may want to disable this protection by setting " +
|
||||
"'akka.cluster.configuration-compatibility-check.enforce-on-join = off'. " +
|
||||
"Note that disabling it will allow the formation of a cluster with nodes having incompatible configuration settings. " +
|
||||
"This node will be shutdown!")
|
||||
context.stop(self)
|
||||
CoordinatedShutdown(context.system).run(IncompatibleConfigurationDetected)
|
||||
}
|
||||
|
||||
case InitJoinNack(address) ⇒
|
||||
logInfo("Received InitJoinNack message from [{}] to [{}]", sender(), selfAddress)
|
||||
remainingSeedNodes -= address
|
||||
|
|
@ -1393,33 +1487,90 @@ private[cluster] final class FirstSeedNodeProcess(seedNodes: immutable.IndexedSe
|
|||
* 5. seed3 retries the join procedure and gets acks from seed2 first, and then joins to seed2
|
||||
*
|
||||
*/
|
||||
private[cluster] final class JoinSeedNodeProcess(seedNodes: immutable.IndexedSeq[Address]) extends Actor with ActorLogging {
|
||||
@InternalApi
|
||||
private[cluster] final class JoinSeedNodeProcess(seedNodes: immutable.IndexedSeq[Address], joinConfigCompatChecker: JoinConfigCompatChecker) extends Actor with ActorLogging {
|
||||
import InternalClusterAction._
|
||||
import ClusterUserAction.JoinTo
|
||||
|
||||
def selfAddress = Cluster(context.system).selfAddress
|
||||
val cluster = Cluster(context.system)
|
||||
import cluster.settings._
|
||||
def selfAddress = cluster.selfAddress
|
||||
|
||||
if (seedNodes.isEmpty || seedNodes.head == selfAddress)
|
||||
throw new IllegalArgumentException("Join seed node should not be done")
|
||||
|
||||
context.setReceiveTimeout(Cluster(context.system).settings.SeedNodeTimeout)
|
||||
context.setReceiveTimeout(SeedNodeTimeout)
|
||||
|
||||
var attempt = 0
|
||||
|
||||
// all seed nodes, except this one
|
||||
val otherSeedNodes = seedNodes.toSet - selfAddress
|
||||
|
||||
override def preStart(): Unit = self ! JoinSeedNode
|
||||
|
||||
def receive = {
|
||||
case JoinSeedNode ⇒
|
||||
val requiredNonSensitiveKeys = JoinConfigCompatChecker.removeSensitiveKeys(joinConfigCompatChecker.requiredKeys, cluster.settings)
|
||||
// configToValidate only contains the keys that are required according to JoinConfigCompatChecker on this node
|
||||
val configToValidate = JoinConfigCompatChecker.filterWithKeys(requiredNonSensitiveKeys, context.system.settings.config)
|
||||
// send InitJoin to all seed nodes (except myself)
|
||||
attempt += 1
|
||||
seedNodes.collect {
|
||||
case a if a != selfAddress ⇒ context.actorSelection(context.parent.path.toStringWithAddress(a))
|
||||
} foreach { _ ! InitJoin }
|
||||
case InitJoinAck(address) ⇒
|
||||
// first InitJoinAck reply
|
||||
otherSeedNodes.foreach { a ⇒ context.actorSelection(context.parent.path.toStringWithAddress(a)) ! InitJoin(configToValidate) }
|
||||
|
||||
case InitJoinAck(address, CompatibleConfig(clusterConfig)) ⇒
|
||||
log.info("Received InitJoinAck message from [{}] to [{}]", sender(), selfAddress)
|
||||
// validates config coming from cluster against this node config
|
||||
joinConfigCompatChecker.check(clusterConfig, context.system.settings.config) match {
|
||||
case Valid ⇒
|
||||
// first InitJoinAck reply
|
||||
context.parent ! JoinTo(address)
|
||||
context.become(done)
|
||||
|
||||
case Invalid(messages) if ByPassConfigCompatCheck ⇒
|
||||
log.warning("Cluster validated this node config, but sent back incompatible settings: {}. " +
|
||||
"Join will be performed because compatibility check is configured to not be enforced.", messages.mkString(", "))
|
||||
context.parent ! JoinTo(address)
|
||||
context.become(done)
|
||||
|
||||
case Invalid(messages) ⇒
|
||||
log.error("Cluster validated this node config, but sent back incompatible settings: {}. " +
|
||||
"It's recommended to perform a full cluster shutdown in order to deploy this new version. " +
|
||||
"If a cluster shutdown isn't an option, you may want to disable this protection by setting " +
|
||||
"'akka.cluster.configuration-compatibility-check.enforce-on-join = off'. " +
|
||||
"Note that disabling it will allow the formation of a cluster with nodes having incompatible configuration settings. " +
|
||||
"This node will be shutdown!", messages.mkString(", "))
|
||||
context.stop(self)
|
||||
CoordinatedShutdown(context.system).run(IncompatibleConfigurationDetected)
|
||||
}
|
||||
|
||||
case InitJoinAck(address, UncheckedConfig) ⇒
|
||||
log.warning("Joining a cluster without configuration compatibility check feature.")
|
||||
context.parent ! JoinTo(address)
|
||||
context.become(done)
|
||||
|
||||
case InitJoinAck(address, IncompatibleConfig) ⇒
|
||||
// first InitJoinAck reply, but incompatible
|
||||
if (ByPassConfigCompatCheck) {
|
||||
log.info("Received InitJoinAck message from [{}] to [{}]", sender(), selfAddress)
|
||||
log.warning("Joining cluster with incompatible configurations. " +
|
||||
"Join will be performed because compatibility check is configured to not be enforced.")
|
||||
// only join if set to ignore config validation
|
||||
context.parent ! JoinTo(address)
|
||||
context.become(done)
|
||||
} else {
|
||||
log.error(
|
||||
"Couldn't join seed nodes because of incompatible cluster configuration. " +
|
||||
"It's recommended to perform a full cluster shutdown in order to deploy this new version." +
|
||||
"If a cluster shutdown isn't an option, you may want to disable this protection by setting " +
|
||||
"'akka.cluster.configuration-compatibility-check.enforce-on-join = off'. " +
|
||||
"Note that disabling it will allow the formation of a cluster with nodes having incompatible configuration settings. " +
|
||||
"This node will be shutdown!")
|
||||
context.stop(self)
|
||||
CoordinatedShutdown(context.system).run(IncompatibleConfigurationDetected)
|
||||
}
|
||||
|
||||
case InitJoinNack(_) ⇒ // that seed was uninitialized
|
||||
|
||||
case ReceiveTimeout ⇒
|
||||
if (attempt >= 2)
|
||||
log.warning(
|
||||
|
|
@ -1430,8 +1581,8 @@ private[cluster] final class JoinSeedNodeProcess(seedNodes: immutable.IndexedSeq
|
|||
}
|
||||
|
||||
def done: Actor.Receive = {
|
||||
case InitJoinAck(_) ⇒ // already received one, skip rest
|
||||
case ReceiveTimeout ⇒ context.stop(self)
|
||||
case InitJoinAck(_, _) ⇒ // already received one, skip rest
|
||||
case ReceiveTimeout ⇒ context.stop(self)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -1440,6 +1591,7 @@ private[cluster] final class JoinSeedNodeProcess(seedNodes: immutable.IndexedSeq
|
|||
*
|
||||
* The supplied callback will be run, once, when current cluster member come up with the same status.
|
||||
*/
|
||||
@InternalApi
|
||||
private[cluster] class OnMemberStatusChangedListener(callback: Runnable, status: MemberStatus) extends Actor with ActorLogging {
|
||||
import ClusterEvent._
|
||||
private val cluster = Cluster(context.system)
|
||||
|
|
@ -1487,6 +1639,7 @@ private[cluster] class OnMemberStatusChangedListener(callback: Runnable, status:
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi
|
||||
@SerialVersionUID(1L)
|
||||
private[cluster] final case class GossipStats(
|
||||
receivedGossipCount: Long = 0L,
|
||||
|
|
@ -1530,8 +1683,8 @@ private[cluster] final case class GossipStats(
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi
|
||||
@SerialVersionUID(1L)
|
||||
private[cluster] final case class VectorClockStats(
|
||||
versionSize: Int = 0,
|
||||
seenLatest: Int = 0)
|
||||
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue