Merge gossip seen table when versions are the same. See #3115
This commit is contained in:
parent
370d6451c7
commit
fad4289b1b
4 changed files with 162 additions and 30 deletions
|
|
@ -215,6 +215,10 @@ object StressMultiJvmSpec extends MultiNodeConfig {
|
|||
import akka.cluster.Member.addressOrdering
|
||||
immutable.SortedMap.empty[Address, Set[PhiValue]]
|
||||
}
|
||||
var clusterStatsObservedByNode = {
|
||||
import akka.cluster.Member.addressOrdering
|
||||
immutable.SortedMap.empty[Address, ClusterStats]
|
||||
}
|
||||
|
||||
import context.dispatcher
|
||||
val reportMetricsTask = context.system.scheduler.schedule(
|
||||
|
|
@ -231,13 +235,14 @@ object StressMultiJvmSpec extends MultiNodeConfig {
|
|||
def receive = {
|
||||
case ClusterMetricsChanged(clusterMetrics) ⇒ nodeMetrics = clusterMetrics
|
||||
case PhiResult(from, phiValues) ⇒ phiValuesObservedByNode += from -> phiValues
|
||||
case StatsResult(from, stats) => clusterStatsObservedByNode += from -> stats
|
||||
case ReportTick ⇒
|
||||
log.info(s"[${title}] in progress\n${formatMetrics}\n\n${formatPhi}")
|
||||
log.info(s"[${title}] in progress\n${formatMetrics}\n\n${formatPhi}\n\n${formatStats}")
|
||||
case r: ClusterResult ⇒
|
||||
results :+= r
|
||||
if (results.size == expectedResults) {
|
||||
val aggregated = AggregatedClusterResult(title, maxDuration, totalClusterStats)
|
||||
log.info(s"[${title}] completed in [${aggregated.duration.toMillis}] ms\n${aggregated.clusterStats}\n${formatMetrics}\n\n${formatPhi}")
|
||||
log.info(s"[${title}] completed in [${aggregated.duration.toMillis}] ms\n${aggregated.clusterStats}\n${formatMetrics}\n\n${formatPhi}\n\n${formatStats}")
|
||||
reportTo foreach { _ ! aggregated }
|
||||
context stop self
|
||||
}
|
||||
|
|
@ -247,13 +252,7 @@ object StressMultiJvmSpec extends MultiNodeConfig {
|
|||
|
||||
def maxDuration = results.map(_.duration).max
|
||||
|
||||
def totalClusterStats = results.map(_.clusterStats).foldLeft(ClusterStats()) { (acc, s) ⇒
|
||||
ClusterStats(
|
||||
receivedGossipCount = acc.receivedGossipCount + s.receivedGossipCount,
|
||||
mergeConflictCount = acc.mergeConflictCount + s.mergeConflictCount,
|
||||
mergeCount = acc.mergeCount + s.mergeCount,
|
||||
mergeDetectedCount = acc.mergeDetectedCount + s.mergeDetectedCount)
|
||||
}
|
||||
def totalClusterStats = results.map(_.clusterStats).foldLeft(ClusterStats()){_ + _}
|
||||
|
||||
def formatMetrics: String = {
|
||||
import akka.cluster.Member.addressOrdering
|
||||
|
|
@ -300,6 +299,16 @@ object StressMultiJvmSpec extends MultiNodeConfig {
|
|||
def formatPhiLine(monitor: Address, subject: Address, phi: PhiValue): String =
|
||||
s"${monitor}\t${subject}\t${phi.count}\t${phi.countAboveOne}\t${phi.max.form}"
|
||||
|
||||
def formatStats: String = {
|
||||
if (clusterStatsObservedByNode.isEmpty) ""
|
||||
else {
|
||||
val lines =
|
||||
for {
|
||||
(monitor, stats) ← clusterStatsObservedByNode.toSeq
|
||||
} yield s"${monitor}\t${stats}"
|
||||
("ClusterStats" +: lines).mkString("\n")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -319,10 +328,10 @@ object StressMultiJvmSpec extends MultiNodeConfig {
|
|||
def formatHistory: String =
|
||||
(formatHistoryHeader +: (history map formatHistoryLine)).mkString("\n")
|
||||
|
||||
def formatHistoryHeader: String = "title\tduration (ms)\tgossip count\tmerge count"
|
||||
def formatHistoryHeader: String = "title\tduration (ms)\tcluster stats"
|
||||
|
||||
def formatHistoryLine(result: AggregatedClusterResult): String =
|
||||
s"${result.title}\t${result.duration.toMillis}\t${result.clusterStats.receivedGossipCount}\t${result.clusterStats.mergeCount}"
|
||||
s"${result.title}\t${result.duration.toMillis}\t${result.clusterStats}"
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -381,6 +390,31 @@ object StressMultiJvmSpec extends MultiNodeConfig {
|
|||
}
|
||||
}
|
||||
|
||||
class StatsObserver extends Actor {
|
||||
val cluster = Cluster(context.system)
|
||||
var reportTo: Option[ActorRef] = None
|
||||
var startStats = cluster.readView.latestStats
|
||||
|
||||
import context.dispatcher
|
||||
val checkStatsTask = context.system.scheduler.schedule(
|
||||
1.second, 1.second, self, StatsTick)
|
||||
|
||||
override def postStop(): Unit = {
|
||||
checkStatsTask.cancel()
|
||||
super.postStop()
|
||||
}
|
||||
|
||||
def receive = {
|
||||
case StatsTick ⇒
|
||||
val res = StatsResult(cluster.selfAddress, cluster.readView.latestStats - startStats)
|
||||
reportTo foreach { _ ! res }
|
||||
case ReportTo(ref) ⇒
|
||||
reportTo = ref
|
||||
case Reset ⇒
|
||||
startStats = cluster.readView.latestStats
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Master of routers
|
||||
*
|
||||
|
|
@ -568,6 +602,8 @@ object StressMultiJvmSpec extends MultiNodeConfig {
|
|||
case class PhiResult(from: Address, phiValues: Set[PhiValue])
|
||||
case class PhiValue(address: Address, countAboveOne: Int, count: Int, max: Double)
|
||||
case class ReportTo(ref: Option[ActorRef])
|
||||
case object StatsTick
|
||||
case class StatsResult(from: Address, stats: ClusterStats)
|
||||
|
||||
type JobId = Int
|
||||
trait Job { def id: JobId }
|
||||
|
|
@ -622,6 +658,7 @@ abstract class StressSpec
|
|||
sys.eventStream.publish(Mute(EventFilter[RuntimeException](pattern = ".*Simulated exception.*")))
|
||||
sys.eventStream.publish(Mute(EventFilter.warning(pattern = ".*PhiResult.*")))
|
||||
sys.eventStream.publish(Mute(EventFilter.warning(pattern = ".*SendBatch.*")))
|
||||
sys.eventStream.publish(Mute(EventFilter.warning(pattern = ".*ClusterStats.*")))
|
||||
}
|
||||
|
||||
val seedNodes = roles.take(numberOfSeedNodes)
|
||||
|
|
@ -645,6 +682,8 @@ abstract class StressSpec
|
|||
enterBarrier("result-aggregator-created-" + step)
|
||||
runOn(roles.take(nbrUsedRoles): _*) {
|
||||
phiObserver ! ReportTo(Some(clusterResultAggregator))
|
||||
statsObserver ! Reset
|
||||
statsObserver ! ReportTo(Some(clusterResultAggregator))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -654,6 +693,8 @@ abstract class StressSpec
|
|||
|
||||
lazy val phiObserver = system.actorOf(Props[PhiObserver], "phiObserver")
|
||||
|
||||
lazy val statsObserver = system.actorOf(Props[StatsObserver], "statsObserver")
|
||||
|
||||
def awaitClusterResult: Unit = {
|
||||
runOn(roles.head) {
|
||||
val r = clusterResultAggregator
|
||||
|
|
@ -789,14 +830,9 @@ abstract class StressSpec
|
|||
|
||||
val returnValue = thunk
|
||||
|
||||
val duration = (System.nanoTime - startTime).nanos
|
||||
val latestStats = clusterView.latestStats
|
||||
val clusterStats = ClusterStats(
|
||||
receivedGossipCount = latestStats.receivedGossipCount - startStats.receivedGossipCount,
|
||||
mergeConflictCount = latestStats.mergeConflictCount - startStats.mergeConflictCount,
|
||||
mergeCount = latestStats.mergeCount - startStats.mergeCount,
|
||||
mergeDetectedCount = latestStats.mergeDetectedCount - startStats.mergeDetectedCount)
|
||||
clusterResultAggregator ! ClusterResult(cluster.selfAddress, duration, clusterStats)
|
||||
clusterResultAggregator !
|
||||
ClusterResult(cluster.selfAddress, (System.nanoTime - startTime).nanos, cluster.readView.latestStats - startStats)
|
||||
|
||||
returnValue
|
||||
}
|
||||
|
||||
|
|
@ -813,6 +849,7 @@ abstract class StressSpec
|
|||
val t = title + " round " + counter
|
||||
runOn(usedRoles: _*) {
|
||||
phiObserver ! Reset
|
||||
statsObserver ! Reset
|
||||
}
|
||||
createResultAggregator(t, expectedResults = nbrUsedRoles, includeInHistory = true)
|
||||
val (nextAS, nextAddresses) = within(loopDuration) {
|
||||
|
|
@ -852,6 +889,7 @@ abstract class StressSpec
|
|||
runOn(usedRoles: _*) {
|
||||
awaitUpConvergence(nbrUsedRoles, timeout = remaining)
|
||||
phiObserver ! Reset
|
||||
statsObserver ! Reset
|
||||
}
|
||||
}
|
||||
enterBarrier("join-remove-shutdown-" + step)
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue