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:
Arnout Engelen 2019-03-26 14:41:29 +01:00 committed by Patrik Nordwall
parent 140e5e0faa
commit d390fcf183
43 changed files with 130 additions and 74 deletions

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

@ -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._

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

@ -55,6 +55,7 @@ import scala.annotation.varargs
import akka.util.JavaDurationConverters._
import akka.util.ccompat._
@ccompatUsedUntil213
object ReplicatorSettings {
/**

View file

@ -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(", ", ", ")")
}

View file

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

View file

@ -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 (&lt;= 32) of

View file

@ -25,6 +25,7 @@ import akka.util.ccompat._
/**
* Some useful serialization helper methods.
*/
@ccompatUsedUntil213
trait SerializationSupport {
private final val BufferSize = 1024 * 4

View file

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

View file

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

View file

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

View file

@ -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 = {

View file

@ -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",

View file

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