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:
Renato Cavalcanti 2018-02-20 15:47:09 +01:00 committed by Patrik Nordwall
parent a4e9881a6f
commit c83e4adfea
19 changed files with 4832 additions and 123 deletions

View file

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