Introduce warning silencer plugin (#26588)
So now we can compile akka-distributed-data with -Xfatal-warnings - though I'm not yet sure about enabling the (other) undisciplineScalacOptions * Fix multi-node silencing * Fix scaladoc warnings * Introduce annotation to declare ccompat use * Add explicit toString * Fix deprecation on 2.13 * Move 'immutable' ccompat helpers to shared ccompat package * Add MiMa for internal scala 2.13 compatibility class * Internal API markers * Fix scaladoc generation Got bitten by https://github.com/scala/bug/issues/11021
This commit is contained in:
parent
140e5e0faa
commit
d390fcf183
43 changed files with 130 additions and 74 deletions
|
|
@ -13,3 +13,9 @@ ProblemFilters.exclude[DirectMissingMethodProblem]("akka.pattern.internal.Backof
|
|||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.pattern.HandleBackoff.replyWhileStopped")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.pattern.HandleBackoff.finalStopMessage")
|
||||
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.pattern.HandleBackoff.handleMessageToChild")
|
||||
|
||||
# Internal helpers for 2.13 compatibility
|
||||
ProblemFilters.exclude[MissingClassProblem]("akka.util.ccompat.imm.package$SortedSetOps")
|
||||
ProblemFilters.exclude[MissingClassProblem]("akka.util.ccompat.imm.package$SortedSetOps$")
|
||||
ProblemFilters.exclude[MissingClassProblem]("akka.util.ccompat.imm.package$")
|
||||
ProblemFilters.exclude[MissingClassProblem]("akka.util.ccompat.imm.package")
|
||||
|
|
|
|||
|
|
@ -0,0 +1,18 @@
|
|||
/*
|
||||
* Copyright (C) 2019 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.util.ccompat
|
||||
|
||||
import scala.annotation.Annotation
|
||||
|
||||
import akka.annotation.InternalApi
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*
|
||||
* Annotation to mark files that need ccompat to be imported for Scala 2.11 and/or 2.12,
|
||||
* but not 2.13. Gets rid of the 'unused import' warning on 2.13.
|
||||
*/
|
||||
@InternalApi
|
||||
private[akka] class ccompatUsedUntil213 extends Annotation
|
||||
|
|
@ -1,9 +0,0 @@
|
|||
/*
|
||||
* Copyright (C) 2018-2019 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.util.ccompat
|
||||
|
||||
import scala.collection.immutable
|
||||
|
||||
package object imm {}
|
||||
|
|
@ -0,0 +1,18 @@
|
|||
/*
|
||||
* Copyright (C) 2019 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.util.ccompat
|
||||
|
||||
import scala.annotation.Annotation
|
||||
|
||||
import akka.annotation.InternalApi
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*
|
||||
* Annotation to mark files that need ccompat to be imported for Scala 2.11 and/or 2.12,
|
||||
* but not 2.13. Gets rid of the 'unused import' warning on 2.13.
|
||||
*/
|
||||
@InternalApi
|
||||
private[akka] class ccompatUsedUntil213 extends Annotation
|
||||
|
|
@ -1,16 +0,0 @@
|
|||
/*
|
||||
* Copyright (C) 2018-2019 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.util.ccompat
|
||||
|
||||
import scala.collection.immutable
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
package object imm {
|
||||
implicit class SortedSetOps[A](val real: immutable.SortedSet[A]) extends AnyVal {
|
||||
def unsorted: immutable.Set[A] = real
|
||||
}
|
||||
}
|
||||
|
|
@ -6,12 +6,9 @@ package akka.util
|
|||
|
||||
import scala.language.implicitConversions
|
||||
import scala.language.higherKinds
|
||||
|
||||
import scala.collection.GenTraversable
|
||||
import scala.collection.{ GenTraversable, immutable => i, mutable => m }
|
||||
import scala.{ collection => c }
|
||||
import scala.collection.generic.{ CanBuildFrom, GenericCompanion, Sorted, SortedSetFactory }
|
||||
import scala.collection.{ immutable => i }
|
||||
import scala.collection.{ mutable => m }
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -81,4 +78,8 @@ package object ccompat {
|
|||
// in scala-library so we can't add to it
|
||||
type IterableOnce[+X] = c.TraversableOnce[X]
|
||||
val IterableOnce = c.TraversableOnce
|
||||
|
||||
implicit class ImmutableSortedSetOps[A](val real: i.SortedSet[A]) extends AnyVal {
|
||||
def unsorted: i.Set[A] = real
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -31,7 +31,7 @@ import akka.serialization.SerializerWithStringManifest
|
|||
import akka.util.Timeout
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.scalatest.WordSpecLike
|
||||
import akka.util.ccompat.imm._
|
||||
import akka.util.ccompat._
|
||||
|
||||
object ClusterShardingSpec {
|
||||
val config = ConfigFactory.parseString(s"""
|
||||
|
|
|
|||
|
|
@ -22,7 +22,7 @@ import akka.remote.testkit.MultiNodeSpec
|
|||
import akka.remote.testkit.STMultiNodeSpec
|
||||
import akka.remote.transport.ThrottlerTransportAdapter.Direction
|
||||
import akka.testkit._
|
||||
import akka.util.ccompat.imm._
|
||||
import akka.util.ccompat._
|
||||
|
||||
object ClusterShardingFailureSpec {
|
||||
case class Get(id: String)
|
||||
|
|
|
|||
|
|
@ -19,7 +19,7 @@ import org.apache.commons.io.FileUtils
|
|||
import scala.concurrent.duration._
|
||||
import akka.cluster.sharding.ShardRegion.GetClusterShardingStats
|
||||
import akka.cluster.sharding.ShardRegion.ClusterShardingStats
|
||||
import akka.util.ccompat.imm._
|
||||
import akka.util.ccompat._
|
||||
|
||||
object ClusterShardingMinMembersSpec {
|
||||
case object StopEntity
|
||||
|
|
|
|||
|
|
@ -15,7 +15,7 @@ import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec }
|
|||
import akka.testkit._
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.apache.commons.io.FileUtils
|
||||
import akka.util.ccompat.imm._
|
||||
import akka.util.ccompat._
|
||||
|
||||
import scala.concurrent.duration._
|
||||
|
||||
|
|
|
|||
|
|
@ -18,7 +18,7 @@ import akka.remote.testkit.MultiNodeSpec
|
|||
import akka.remote.testkit.STMultiNodeSpec
|
||||
import akka.testkit._
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.util.ccompat.imm._
|
||||
import akka.util.ccompat._
|
||||
|
||||
object MultiDcClusterShardingSpec {
|
||||
sealed trait EntityMsg {
|
||||
|
|
|
|||
|
|
@ -16,7 +16,7 @@ import akka.testkit.AkkaSpec
|
|||
import akka.testkit.TestActors.EchoActor
|
||||
import akka.testkit.TestProbe
|
||||
import akka.testkit.WithLogCapturing
|
||||
import akka.util.ccompat.imm._
|
||||
import akka.util.ccompat._
|
||||
|
||||
object CoordinatedShutdownShardingSpec {
|
||||
val config =
|
||||
|
|
|
|||
|
|
@ -17,7 +17,7 @@ import akka.remote.testkit.MultiNodeSpec
|
|||
import akka.remote.testkit.STMultiNodeSpec
|
||||
import akka.remote.transport.ThrottlerTransportAdapter
|
||||
import akka.testkit._
|
||||
import akka.util.ccompat.imm._
|
||||
import akka.util.ccompat._
|
||||
import com.typesafe.config.ConfigFactory
|
||||
|
||||
object ClusterSingletonManagerDownedSpec extends MultiNodeConfig {
|
||||
|
|
|
|||
|
|
@ -82,7 +82,7 @@ class ClusterSingletonLeavingSpeedSpec
|
|||
|
||||
Cluster(from).join(Cluster(to).selfAddress)
|
||||
within(15.seconds) {
|
||||
import akka.util.ccompat.imm._
|
||||
import akka.util.ccompat._
|
||||
awaitAssert {
|
||||
Cluster(from).state.members.map(_.uniqueAddress) should contain(Cluster(from).selfUniqueAddress)
|
||||
Cluster(from).state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
|
||||
|
|
|
|||
|
|
@ -62,7 +62,7 @@ class ClusterSingletonRestart2Spec extends AkkaSpec("""
|
|||
name = "echo")
|
||||
|
||||
within(45.seconds) {
|
||||
import akka.util.ccompat.imm._
|
||||
import akka.util.ccompat._
|
||||
awaitAssert {
|
||||
Cluster(from).join(Cluster(to).selfAddress)
|
||||
Cluster(from).state.members.map(_.uniqueAddress) should contain(Cluster(from).selfUniqueAddress)
|
||||
|
|
|
|||
|
|
@ -43,7 +43,7 @@ class ClusterSingletonRestartSpec extends AkkaSpec("""
|
|||
name = "echo")
|
||||
|
||||
within(10.seconds) {
|
||||
import akka.util.ccompat.imm._
|
||||
import akka.util.ccompat._
|
||||
awaitAssert {
|
||||
Cluster(from).join(Cluster(to).selfAddress)
|
||||
Cluster(from).state.members.map(_.uniqueAddress) should contain(Cluster(from).selfUniqueAddress)
|
||||
|
|
|
|||
|
|
@ -484,7 +484,7 @@ object ClusterEvent {
|
|||
if newMember.status != oldMember.status || newMember.upNumber != oldMember.upNumber =>
|
||||
newMember
|
||||
}
|
||||
import akka.util.ccompat.imm._
|
||||
import akka.util.ccompat._
|
||||
val memberEvents = (newMembers ++ changedMembers).unsorted.collect {
|
||||
case m if m.status == Joining => MemberJoined(m)
|
||||
case m if m.status == WeaklyUp => MemberWeaklyUp(m)
|
||||
|
|
|
|||
|
|
@ -22,7 +22,7 @@ import akka.cluster.InternalClusterAction._
|
|||
import akka.cluster.routing.{ ClusterRouterPool, ClusterRouterPoolSettings }
|
||||
import akka.routing.Pool
|
||||
import akka.util.ccompat._
|
||||
import akka.util.ccompat.imm._
|
||||
import akka.util.ccompat._
|
||||
import com.typesafe.config.{ Config, ConfigFactory, ConfigRenderOptions }
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -10,7 +10,7 @@ import akka.remote.testkit.MultiNodeConfig
|
|||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
import akka.cluster.MemberStatus._
|
||||
import akka.util.ccompat.imm._
|
||||
import akka.util.ccompat._
|
||||
|
||||
object MinMembersBeforeUpMultiJvmSpec extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
|
|
|
|||
|
|
@ -14,7 +14,7 @@ import com.typesafe.config.ConfigFactory
|
|||
import scala.collection.immutable
|
||||
import scala.collection.immutable.SortedSet
|
||||
import scala.concurrent.duration._
|
||||
import akka.util.ccompat.imm._
|
||||
import akka.util.ccompat._
|
||||
|
||||
object MultiDcHeartbeatTakingOverSpecMultiJvmSpec extends MultiNodeConfig {
|
||||
val first = role("first") // alpha
|
||||
|
|
|
|||
|
|
@ -25,7 +25,7 @@ import java.util.concurrent.ConcurrentHashMap
|
|||
|
||||
import akka.remote.DefaultFailureDetectorRegistry
|
||||
import akka.cluster.ClusterEvent.{ MemberEvent, MemberRemoved }
|
||||
import akka.util.ccompat.imm._
|
||||
import akka.util.ccompat._
|
||||
|
||||
import scala.concurrent.Await
|
||||
|
||||
|
|
|
|||
|
|
@ -7,7 +7,7 @@ package akka.cluster
|
|||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
import akka.util.ccompat.imm._
|
||||
import akka.util.ccompat._
|
||||
|
||||
object NodeMembershipMultiJvmSpec extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
|
|
|
|||
|
|
@ -19,7 +19,7 @@ import akka.actor.Actor
|
|||
import akka.actor.RootActorPath
|
||||
import akka.cluster.MemberStatus._
|
||||
import akka.actor.Deploy
|
||||
import akka.util.ccompat.imm._
|
||||
import akka.util.ccompat._
|
||||
|
||||
object RestartFirstSeedNodeMultiJvmSpec extends MultiNodeConfig {
|
||||
val seed1 = role("seed1")
|
||||
|
|
|
|||
|
|
@ -18,7 +18,7 @@ import akka.remote.testkit.MultiNodeConfig
|
|||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.util.ccompat.imm._
|
||||
import akka.util.ccompat._
|
||||
|
||||
object RestartNode2SpecMultiJvmSpec extends MultiNodeConfig {
|
||||
val seed1 = role("seed1")
|
||||
|
|
|
|||
|
|
@ -18,7 +18,7 @@ import akka.remote.testkit.MultiNodeSpec
|
|||
import akka.remote.transport.ThrottlerTransportAdapter.Direction
|
||||
import akka.testkit._
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.util.ccompat.imm._
|
||||
import akka.util.ccompat._
|
||||
|
||||
object RestartNode3MultiJvmSpec extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
|
|
|
|||
|
|
@ -23,7 +23,7 @@ import akka.remote.testkit.MultiNodeConfig
|
|||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.util.ccompat.imm._
|
||||
import akka.util.ccompat._
|
||||
|
||||
object RestartNodeMultiJvmSpec extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
|
|
|
|||
|
|
@ -19,7 +19,7 @@ import akka.testkit._
|
|||
import akka.actor.Props
|
||||
import akka.cluster.MultiNodeClusterSpec.EndActor
|
||||
import akka.remote.RARP
|
||||
import akka.util.ccompat.imm._
|
||||
import akka.util.ccompat._
|
||||
|
||||
object UnreachableNodeJoinsAgainMultiNodeConfig extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
|
|
|
|||
|
|
@ -119,7 +119,7 @@ object ServiceDiscovery {
|
|||
* For example `portName` could be used to distinguish between
|
||||
* Akka remoting ports and HTTP ports.
|
||||
*
|
||||
* @throws IllegalArgumentException if [[serviceName]] is 'null' or an empty String
|
||||
* @param serviceName must not be 'null' or an empty String
|
||||
*/
|
||||
final class Lookup(val serviceName: String, val portName: Option[String], val protocol: Option[String]) {
|
||||
|
||||
|
|
@ -236,8 +236,8 @@ case object Lookup {
|
|||
* The string is parsed and dismembered to build a Lookup as following:
|
||||
* Lookup(serviceName).withPortName(portName).withProtocol(protocol)
|
||||
*
|
||||
* @throws NullPointerException If the passed string is null
|
||||
* @throws IllegalArgumentException If the string doesn't not conform with the SRV format
|
||||
* @throws java.lang.NullPointerException If the passed string is null
|
||||
* @throws java.lang.IllegalArgumentException If the string doesn't not conform with the SRV format
|
||||
*/
|
||||
def parseSrv(str: String): Lookup =
|
||||
str match {
|
||||
|
|
|
|||
|
|
@ -17,7 +17,9 @@ import akka.util.ccompat._
|
|||
* INTERNAL API: Used by the Replicator actor.
|
||||
* Extracted to separate trait to make it easy to test.
|
||||
*/
|
||||
@InternalApi private[akka] trait DeltaPropagationSelector {
|
||||
@ccompatUsedUntil213
|
||||
@InternalApi
|
||||
private[akka] trait DeltaPropagationSelector {
|
||||
|
||||
private var _propagationCount = 0L
|
||||
def propagationCount: Long = _propagationCount
|
||||
|
|
|
|||
|
|
@ -216,7 +216,7 @@ final class ORMap[A, B <: ReplicatedData] private[akka] (
|
|||
|
||||
/**
|
||||
* Adds an entry to the map
|
||||
* @see [[#put]]
|
||||
* @see [[ORMap#put(node:akka\.cluster\.ddata\.SelfUniqueAddress*]]
|
||||
*/
|
||||
def :+(entry: (A, B))(implicit node: SelfUniqueAddress): ORMap[A, B] = {
|
||||
val (key, value) = entry
|
||||
|
|
@ -235,12 +235,13 @@ final class ORMap[A, B <: ReplicatedData] private[akka] (
|
|||
* on other nodes and the outcome depends on what `ReplicatedData`
|
||||
* type that is used.
|
||||
*
|
||||
* Consider using [[#updated]] instead of `put` if you want modify
|
||||
* existing entry.
|
||||
* Consider using [[ORMap#updated(node:akka\.cluster\.ddata\.SelfUniqueAddress*]]
|
||||
* instead of `put` if you want modify existing entry.
|
||||
*
|
||||
* `IllegalArgumentException` is thrown if you try to replace an existing `ORSet`
|
||||
* value, because important history can be lost when replacing the `ORSet` and
|
||||
* undesired effects of merging will occur. Use [[ORMultiMap]] or [[#updated]] instead.
|
||||
* undesired effects of merging will occur. Use [[ORMultiMap]] or
|
||||
* [[ORMap#updated(node:akka\.cluster\.ddata\.SelfUniqueAddress*]] instead.
|
||||
*/
|
||||
def put(node: SelfUniqueAddress, key: A, value: B): ORMap[A, B] = put(node.uniqueAddress, key, value)
|
||||
|
||||
|
|
|
|||
|
|
@ -136,7 +136,7 @@ final class ORMultiMap[A, B] private[akka] (
|
|||
|
||||
/**
|
||||
* Convenience for put. Requires an implicit SelfUniqueAddress.
|
||||
* @see [[#put]]
|
||||
* @see [[ORMultiMap#put(node:akka\.cluster\.ddata\.SelfUniqueAddress,key:A,value:Set*]]
|
||||
*/
|
||||
def :+(entry: (A, Set[B]))(implicit node: SelfUniqueAddress): ORMultiMap[A, B] = {
|
||||
val (key, value) = entry
|
||||
|
|
@ -196,7 +196,7 @@ final class ORMultiMap[A, B] private[akka] (
|
|||
|
||||
/**
|
||||
* Convenience for remove. Requires an implicit Cluster.
|
||||
* @see [[#remove]]
|
||||
* @see [[ORMultiMap#remove(node:akka\.cluster\.ddata\.SelfUniqueAddress*]]
|
||||
*/
|
||||
@deprecated("Use `remove` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||
def -(key: A)(implicit node: Cluster): ORMultiMap[A, B] = remove(node.selfUniqueAddress, key)
|
||||
|
|
|
|||
|
|
@ -388,8 +388,9 @@ final class ORSet[A] private[akka] (
|
|||
|
||||
/**
|
||||
* Removes all elements from the set, but keeps the history.
|
||||
* This has the same result as using [[#remove]] for each
|
||||
* element, but it is more efficient.
|
||||
* This has the same result as using
|
||||
* [[ORSet#remove(node:akka\.cluster\.ddata\.SelfUniqueAddress*]]
|
||||
* for each element, but it is more efficient.
|
||||
*/
|
||||
def clear(@unused node: SelfUniqueAddress): ORSet[A] = clear()
|
||||
|
||||
|
|
|
|||
|
|
@ -55,6 +55,7 @@ import scala.annotation.varargs
|
|||
import akka.util.JavaDurationConverters._
|
||||
import akka.util.ccompat._
|
||||
|
||||
@ccompatUsedUntil213
|
||||
object ReplicatorSettings {
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -386,5 +386,5 @@ final case class ManyVersionVector(versions: TreeMap[UniqueAddress, Long]) exten
|
|||
else this
|
||||
|
||||
override def toString: String =
|
||||
versions.map { case ((n, v)) => n + " -> " + v }.mkString("VersionVector(", ", ", ")")
|
||||
versions.map { case ((n, v)) => n.toString + " -> " + v }.mkString("VersionVector(", ", ", ")")
|
||||
}
|
||||
|
|
|
|||
|
|
@ -25,11 +25,14 @@ import akka.protobuf.{ ByteString, GeneratedMessage }
|
|||
import akka.util.ByteString.UTF_8
|
||||
import java.io.NotSerializableException
|
||||
|
||||
import com.github.ghik.silencer.silent
|
||||
|
||||
import akka.actor.ActorRef
|
||||
import akka.cluster.ddata.protobuf.msg.ReplicatorMessages.OtherMessage
|
||||
import akka.serialization.Serialization
|
||||
import akka.util.ccompat._
|
||||
|
||||
@ccompatUsedUntil213
|
||||
private object ReplicatedDataSerializer {
|
||||
/*
|
||||
* Generic superclass to allow to compare Entry types used in protobuf.
|
||||
|
|
@ -43,6 +46,8 @@ private object ReplicatedDataSerializer {
|
|||
*/
|
||||
def getKey(entry: A): Any
|
||||
final def compare(x: A, y: A): Int = compareKeys(getKey(x), getKey(y))
|
||||
|
||||
@silent
|
||||
private final def compareKeys(t1: Any, t2: Any): Int = (t1, t2) match {
|
||||
case (k1: String, k2: String) => k1.compareTo(k2)
|
||||
case (k1: String, k2) => -1
|
||||
|
|
|
|||
|
|
@ -37,7 +37,9 @@ import akka.util.ccompat._
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] object ReplicatorMessageSerializer {
|
||||
@ccompatUsedUntil213
|
||||
@InternalApi
|
||||
private[akka] object ReplicatorMessageSerializer {
|
||||
|
||||
/**
|
||||
* A cache that is designed for a small number (<= 32) of
|
||||
|
|
|
|||
|
|
@ -25,6 +25,7 @@ import akka.util.ccompat._
|
|||
/**
|
||||
* Some useful serialization helper methods.
|
||||
*/
|
||||
@ccompatUsedUntil213
|
||||
trait SerializationSupport {
|
||||
|
||||
private final val BufferSize = 1024 * 4
|
||||
|
|
|
|||
|
|
@ -16,8 +16,9 @@ import akka.actor.ActorSystem
|
|||
import akka.actor.ActorRef
|
||||
import scala.concurrent.Await
|
||||
import akka.cluster.MemberStatus
|
||||
import akka.util.ccompat.imm._
|
||||
import akka.util.ccompat._
|
||||
|
||||
@ccompatUsedUntil213
|
||||
object DurablePruningSpec extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
val second = role("second")
|
||||
|
|
@ -75,6 +76,7 @@ class DurablePruningSpec extends MultiNodeSpec(DurablePruningSpec) with STMultiN
|
|||
|
||||
val sys2 = ActorSystem(system.name, system.settings.config)
|
||||
val cluster2 = Cluster(sys2)
|
||||
val distributedData2 = DistributedData(sys2)
|
||||
val replicator2 = startReplicator(sys2)
|
||||
val probe2 = TestProbe()(sys2)
|
||||
Cluster(sys2).join(node(first).address)
|
||||
|
|
@ -98,7 +100,9 @@ class DurablePruningSpec extends MultiNodeSpec(DurablePruningSpec) with STMultiN
|
|||
replicator ! Update(KeyA, GCounter(), WriteLocal)(_ :+ 3)
|
||||
expectMsg(UpdateSuccess(KeyA, None))
|
||||
|
||||
replicator2.tell(Update(KeyA, GCounter(), WriteLocal)(_.increment(cluster2, 2)), probe2.ref)
|
||||
replicator2.tell(
|
||||
Update(KeyA, GCounter(), WriteLocal)(_.increment(distributedData2.selfUniqueAddress, 2)),
|
||||
probe2.ref)
|
||||
probe2.expectMsg(UpdateSuccess(KeyA, None))
|
||||
|
||||
enterBarrier("updates-done")
|
||||
|
|
|
|||
|
|
@ -331,7 +331,7 @@ class ReplicatorMapDeltaSpec extends MultiNodeSpec(ReplicatorMapDeltaSpec) with
|
|||
|
||||
enterBarrier("replicated-2")
|
||||
// no OversizedPayloadException logging
|
||||
errorLogProbe.expectNoMsg(100.millis)
|
||||
errorLogProbe.expectNoMessage(100.millis)
|
||||
|
||||
enterBarrierAfterTestStep()
|
||||
}
|
||||
|
|
|
|||
|
|
@ -239,7 +239,7 @@ class ReplicatorPruningSpec extends MultiNodeSpec(ReplicatorPruningSpec) with ST
|
|||
|
||||
// after full replication should still not be able to update with data from removed node
|
||||
// but it would not work after removal of the PruningPerformed markers
|
||||
expectNoMsg(maxPruningDissemination + 3.seconds)
|
||||
expectNoMessage(maxPruningDissemination + 3.seconds)
|
||||
|
||||
runOn(first) {
|
||||
updateAfterPruning(expectedValue = 12)
|
||||
|
|
|
|||
|
|
@ -24,7 +24,7 @@ object LotsOfDataBot {
|
|||
if (args.isEmpty)
|
||||
startup(Seq("2551", "2552", "0"))
|
||||
else
|
||||
startup(args)
|
||||
startup(args.toIndexedSeq)
|
||||
}
|
||||
|
||||
def startup(ports: Seq[String]): Unit = {
|
||||
|
|
|
|||
|
|
@ -21,7 +21,14 @@ object AkkaDisciplinePlugin extends AutoPlugin with ScalafixSupport {
|
|||
override def requires: Plugins = JvmPlugin && ScalafixPlugin
|
||||
override lazy val projectSettings = disciplineSettings
|
||||
|
||||
val strictProjects = Set("akka-discovery")
|
||||
val fatalWarningsFor = Set(
|
||||
"akka-discovery",
|
||||
"akka-distributed-data",
|
||||
)
|
||||
|
||||
val strictProjects = Set(
|
||||
"akka-discovery",
|
||||
)
|
||||
|
||||
lazy val scalaFixSettings = Seq(
|
||||
Compile / scalacOptions += "-Yrangepos")
|
||||
|
|
@ -35,14 +42,27 @@ object AkkaDisciplinePlugin extends AutoPlugin with ScalafixSupport {
|
|||
!VersionNumber(scalaVersion.value).matchesSemVer(SemanticSelector("<=2.11.1"))
|
||||
})
|
||||
|
||||
val silencerVersion = "1.3.1"
|
||||
lazy val silencerSettings = Seq(
|
||||
libraryDependencies ++= Seq(
|
||||
compilerPlugin("com.github.ghik" %% "silencer-plugin" % silencerVersion),
|
||||
"com.github.ghik" %% "silencer-lib" % silencerVersion % Provided,
|
||||
)
|
||||
)
|
||||
|
||||
lazy val disciplineSettings =
|
||||
scalaFixSettings ++
|
||||
silencerSettings ++
|
||||
scoverageSettings ++ Seq(
|
||||
Compile / scalacOptions ++= (if (strictProjects.contains(name.value)) {
|
||||
disciplineScalacOptions
|
||||
} else {
|
||||
disciplineScalacOptions -- undisciplineScalacOptions
|
||||
}).toSeq,
|
||||
Compile / scalacOptions ++= (
|
||||
if (fatalWarningsFor(name.value)) Seq("-Xfatal-warnings")
|
||||
else Seq.empty
|
||||
),
|
||||
Compile / console / scalacOptions --= Seq("-deprecation", "-Xfatal-warnings", "-Xlint", "-Ywarn-unused:imports"),
|
||||
// Discipline is not needed for the docs compilation run (which uses
|
||||
// different compiler phases from the regular run), and in particular
|
||||
|
|
@ -74,15 +94,14 @@ object AkkaDisciplinePlugin extends AutoPlugin with ScalafixSupport {
|
|||
"-Ywarn-value-discard",
|
||||
"-Ywarn-numeric-widen",
|
||||
"-Yno-adapted-args",
|
||||
"-Xfatal-warnings")
|
||||
)
|
||||
|
||||
/** These options are desired, but some are excluded for the time being*/
|
||||
val disciplineScalacOptions = Set(
|
||||
// start: must currently remove, version regardless
|
||||
"-Xfatal-warnings",
|
||||
"-Ywarn-value-discard",
|
||||
"-Yno-adapted-args",
|
||||
"-Ywarn-numeric-widen",
|
||||
"-Yno-adapted-args",
|
||||
// end
|
||||
"-deprecation",
|
||||
"-Xfuture",
|
||||
|
|
|
|||
|
|
@ -68,6 +68,8 @@ object MultiNode extends AutoPlugin {
|
|||
SbtMultiJvm.multiJvmSettings ++
|
||||
inConfig(MultiJvm)(scalafmtConfigSettings) ++
|
||||
Seq(
|
||||
// Hack because 'provided' dependencies by default are not picked up by the multi-jvm plugin:
|
||||
managedClasspath in MultiJvm ++= (managedClasspath in Compile).value.filter(_.data.name.contains("silencer-lib")),
|
||||
jvmOptions in MultiJvm := defaultMultiJvmOptions,
|
||||
scalacOptions in MultiJvm := (scalacOptions in Test).value,
|
||||
logLevel in multiJvmCreateLogger := Level.Debug, // to see ssh establishment
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue