MINOR: Enable fatal warnings with scala 2.13 (#8429)

* Upgrade to Scala 2.13.2 which introduces the ability to suppress warnings.
* Upgrade to scala-collection-compat 2.1.6 as it introduces the
@nowarn annotation for Scala 2.12.
* While at it, also update scala-java8-compat to 0.9.1.
* Fix compiler warnings and add @nowarn for the unfixed ones.

Scala 2.13.2 highlights (besides @nowarn):

* Rewrite Vector (using "radix-balanced finger tree vectors"),
for performance. Small vectors are now more compactly
represented. Some operations are now drastically faster on
large vectors. A few operations may be a little slower.
* Matching strings makes switches in bytecode.

https://github.com/scala/scala/releases/tag/v2.13.2

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
This commit is contained in:
Ismael Juma 2020-04-23 00:44:03 -07:00 committed by GitHub
parent 039774038c
commit c5ae154a3f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
30 changed files with 75 additions and 23 deletions

View File

@ -484,6 +484,12 @@ subprojects {
scalaCompileOptions.additionalParameters += ["-opt:l:inline"] scalaCompileOptions.additionalParameters += ["-opt:l:inline"]
scalaCompileOptions.additionalParameters += inlineFrom scalaCompileOptions.additionalParameters += inlineFrom
if (versions.baseScala != '2.12') {
scalaCompileOptions.additionalParameters += ["-opt-warnings"]
// Scala 2.13.2 introduces compiler warnings suppression, which is a pre-requisite for -Xfatal-warnings
scalaCompileOptions.additionalParameters += ["-Xfatal-warnings"]
}
// these options are valid for Scala versions < 2.13 only // these options are valid for Scala versions < 2.13 only
// Scala 2.13 removes them, see https://github.com/scala/scala/pull/6502 and https://github.com/scala/scala/pull/5969 // Scala 2.13 removes them, see https://github.com/scala/scala/pull/6502 and https://github.com/scala/scala/pull/5969
if (versions.baseScala == '2.12') { if (versions.baseScala == '2.12') {

View File

@ -39,6 +39,7 @@ import org.apache.kafka.common.security.scram.internals.{ScramCredentialUtils, S
import org.apache.kafka.common.utils.{Sanitizer, Time, Utils} import org.apache.kafka.common.utils.{Sanitizer, Time, Utils}
import org.apache.zookeeper.client.ZKClientConfig import org.apache.zookeeper.client.ZKClientConfig
import scala.annotation.nowarn
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
import scala.collection._ import scala.collection._
@ -297,6 +298,7 @@ object ConfigCommand extends Config {
} }
} }
@nowarn("cat=deprecation")
private[admin] def alterConfig(adminClient: Admin, opts: ConfigCommandOptions): Unit = { private[admin] def alterConfig(adminClient: Admin, opts: ConfigCommandOptions): Unit = {
val entityTypes = opts.entityTypes val entityTypes = opts.entityTypes
val entityNames = opts.entityNames val entityNames = opts.entityNames

View File

@ -498,7 +498,7 @@ class Log(@volatile private var _dir: File,
def newLeaderEpochFileCache(): LeaderEpochFileCache = { def newLeaderEpochFileCache(): LeaderEpochFileCache = {
val checkpointFile = new LeaderEpochCheckpointFile(leaderEpochFile, logDirFailureChannel) val checkpointFile = new LeaderEpochCheckpointFile(leaderEpochFile, logDirFailureChannel)
new LeaderEpochFileCache(topicPartition, logEndOffset _, checkpointFile) new LeaderEpochFileCache(topicPartition, () => logEndOffset, checkpointFile)
} }
if (recordVersion.precedes(RecordVersion.V2)) { if (recordVersion.precedes(RecordVersion.V2)) {

View File

@ -38,6 +38,7 @@ import org.apache.kafka.common.requests._
import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.security.auth.KafkaPrincipal
import org.apache.kafka.common.utils.{Sanitizer, Time} import org.apache.kafka.common.utils.{Sanitizer, Time}
import scala.annotation.nowarn
import scala.collection.mutable import scala.collection.mutable
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
import scala.reflect.ClassTag import scala.reflect.ClassTag
@ -106,7 +107,7 @@ object RequestChannel extends Logging {
def requestDesc(details: Boolean): String = s"$header -- ${loggableRequest.toString(details)}" def requestDesc(details: Boolean): String = s"$header -- ${loggableRequest.toString(details)}"
def body[T <: AbstractRequest](implicit classTag: ClassTag[T], nn: NotNothing[T]): T = { def body[T <: AbstractRequest](implicit classTag: ClassTag[T], @nowarn("cat=unused") nn: NotNothing[T]): T = {
bodyAndSize.request match { bodyAndSize.request match {
case r: T => r case r: T => r
case r => case r =>

View File

@ -700,7 +700,7 @@ private[kafka] class Acceptor(val endPoint: EndPoint,
* Wakeup the thread for selection. * Wakeup the thread for selection.
*/ */
@Override @Override
def wakeup = nioSelector.wakeup() def wakeup(): Unit = nioSelector.wakeup()
} }

View File

@ -39,7 +39,8 @@ import org.apache.kafka.server.authorizer.AclDeleteResult.AclBindingDeleteResult
import org.apache.kafka.server.authorizer._ import org.apache.kafka.server.authorizer._
import org.apache.zookeeper.client.ZKClientConfig import org.apache.zookeeper.client.ZKClientConfig
import scala.collection.{mutable, Seq} import scala.annotation.nowarn
import scala.collection.{Seq, mutable}
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
import scala.util.{Failure, Random, Success, Try} import scala.util.{Failure, Random, Success, Try}
@ -249,7 +250,7 @@ class AclAuthorizer extends Authorizer with Logging {
} }
} catch { } catch {
case e: Exception => case e: Exception =>
resourceBindingsBeingDeleted.foreach { case (binding, index) => resourceBindingsBeingDeleted.keys.foreach { binding =>
deleteExceptions.getOrElseUpdate(binding, apiException(e)) deleteExceptions.getOrElseUpdate(binding, apiException(e))
} }
} }
@ -263,6 +264,7 @@ class AclAuthorizer extends Authorizer with Logging {
}.map(CompletableFuture.completedFuture[AclDeleteResult]).asJava }.map(CompletableFuture.completedFuture[AclDeleteResult]).asJava
} }
@nowarn("cat=optimizer")
override def acls(filter: AclBindingFilter): lang.Iterable[AclBinding] = { override def acls(filter: AclBindingFilter): lang.Iterable[AclBinding] = {
val aclBindings = new util.ArrayList[AclBinding]() val aclBindings = new util.ArrayList[AclBinding]()
aclCache.foreach { case (resource, versionedAcls) => aclCache.foreach { case (resource, versionedAcls) =>
@ -342,6 +344,7 @@ class AclAuthorizer extends Authorizer with Logging {
} else false } else false
} }
@nowarn("cat=deprecation")
private def matchingAcls(resourceType: ResourceType, resourceName: String): AclSeqs = { private def matchingAcls(resourceType: ResourceType, resourceName: String): AclSeqs = {
// save aclCache reference to a local val to get a consistent view of the cache during acl updates. // save aclCache reference to a local val to get a consistent view of the cache during acl updates.
val aclCacheSnapshot = aclCache val aclCacheSnapshot = aclCache

View File

@ -28,9 +28,12 @@ import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.utils.Utils
import org.apache.kafka.server.authorizer.{AuthorizableRequestContext, Authorizer} import org.apache.kafka.server.authorizer.{AuthorizableRequestContext, Authorizer}
import scala.annotation.nowarn
object AuthorizerUtils { object AuthorizerUtils {
@nowarn("cat=deprecation")
def createAuthorizer(className: String): Authorizer = { def createAuthorizer(className: String): Authorizer = {
Utils.newInstance(className, classOf[Object]) match { Utils.newInstance(className, classOf[Object]) match {
case auth: Authorizer => auth case auth: Authorizer => auth

View File

@ -27,7 +27,6 @@ import java.util.concurrent.atomic.AtomicInteger
import kafka.admin.{AdminUtils, RackAwareMode} import kafka.admin.{AdminUtils, RackAwareMode}
import kafka.api.ElectLeadersRequestOps import kafka.api.ElectLeadersRequestOps
import kafka.api.LeaderAndIsr
import kafka.api.{ApiVersion, KAFKA_0_11_0_IV0, KAFKA_2_3_IV0} import kafka.api.{ApiVersion, KAFKA_0_11_0_IV0, KAFKA_2_3_IV0}
import kafka.cluster.Partition import kafka.cluster.Partition
import kafka.common.OffsetAndMetadata import kafka.common.OffsetAndMetadata

View File

@ -133,7 +133,7 @@ class ReplicationQuotaManager(val config: ReplicationQuotaManagerConfig,
* @param value * @param value
*/ */
def record(value: Long): Unit = { def record(value: Long): Unit = {
sensor().record(value, time.milliseconds(), false) sensor().record(value.toDouble, time.milliseconds(), false)
} }
/** /**

View File

@ -37,6 +37,7 @@ import org.apache.kafka.common.requests.ListOffsetRequest
import org.apache.kafka.common.serialization.{ByteArrayDeserializer, Deserializer} import org.apache.kafka.common.serialization.{ByteArrayDeserializer, Deserializer}
import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.utils.Utils
import scala.annotation.nowarn
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
/** /**
@ -575,6 +576,7 @@ class ChecksumMessageFormatter extends MessageFormatter {
topicStr = "" topicStr = ""
} }
@nowarn("cat=deprecation")
def writeTo(consumerRecord: ConsumerRecord[Array[Byte], Array[Byte]], output: PrintStream): Unit = { def writeTo(consumerRecord: ConsumerRecord[Array[Byte], Array[Byte]], output: PrintStream): Unit = {
output.println(topicStr + "checksum:" + consumerRecord.checksum) output.println(topicStr + "checksum:" + consumerRecord.checksum)
} }

View File

@ -36,6 +36,7 @@ import org.apache.kafka.common.serialization.{ByteArrayDeserializer, ByteArraySe
import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.utils.{Time, Utils}
import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.{KafkaException, TopicPartition}
import scala.annotation.nowarn
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
import scala.collection.mutable.HashMap import scala.collection.mutable.HashMap
import scala.util.control.ControlThrowable import scala.util.control.ControlThrowable
@ -190,6 +191,7 @@ object MirrorMaker extends Logging with KafkaMetricsGroup {
setName(threadName) setName(threadName)
@nowarn("cat=deprecation")
private def toBaseConsumerRecord(record: ConsumerRecord[Array[Byte], Array[Byte]]): BaseConsumerRecord = private def toBaseConsumerRecord(record: ConsumerRecord[Array[Byte], Array[Byte]]): BaseConsumerRecord =
BaseConsumerRecord(record.topic, BaseConsumerRecord(record.topic,
record.partition, record.partition,
@ -412,10 +414,12 @@ object MirrorMaker extends Logging with KafkaMetricsGroup {
* If message.handler.args is specified. A constructor that takes in a String as argument must exist. * If message.handler.args is specified. A constructor that takes in a String as argument must exist.
*/ */
trait MirrorMakerMessageHandler { trait MirrorMakerMessageHandler {
@nowarn("cat=deprecation")
def handle(record: BaseConsumerRecord): util.List[ProducerRecord[Array[Byte], Array[Byte]]] def handle(record: BaseConsumerRecord): util.List[ProducerRecord[Array[Byte], Array[Byte]]]
} }
private[tools] object defaultMirrorMakerMessageHandler extends MirrorMakerMessageHandler { private[tools] object defaultMirrorMakerMessageHandler extends MirrorMakerMessageHandler {
@nowarn("cat=deprecation")
override def handle(record: BaseConsumerRecord): util.List[ProducerRecord[Array[Byte], Array[Byte]]] = { override def handle(record: BaseConsumerRecord): util.List[ProducerRecord[Array[Byte], Array[Byte]]] = {
val timestamp: java.lang.Long = if (record.timestamp == RecordBatch.NO_TIMESTAMP) null else record.timestamp val timestamp: java.lang.Long = if (record.timestamp == RecordBatch.NO_TIMESTAMP) null else record.timestamp
Collections.singletonList(new ProducerRecord(record.topic, null, timestamp, record.key, record.value, record.headers)) Collections.singletonList(new ProducerRecord(record.topic, null, timestamp, record.key, record.value, record.headers))

View File

@ -83,7 +83,7 @@ class Pool[K,V](valueFactory: Option[K => V] = None) extends Iterable[(K, V)] {
def hasNext: Boolean = iter.hasNext def hasNext: Boolean = iter.hasNext
def next: (K, V) = { def next(): (K, V) = {
val n = iter.next val n = iter.next
(n.getKey, n.getValue) (n.getKey, n.getValue)
} }

View File

@ -198,7 +198,7 @@ class ZooKeeperClient(connectString: String,
case GetDataRequest(path, ctx) => case GetDataRequest(path, ctx) =>
zooKeeper.getData(path, shouldWatch(request), new DataCallback { zooKeeper.getData(path, shouldWatch(request), new DataCallback {
def processResult(rc: Int, path: String, ctx: Any, data: Array[Byte], stat: Stat): Unit = def processResult(rc: Int, path: String, ctx: Any, data: Array[Byte], stat: Stat): Unit =
callback(GetDataResponse(Code.get(rc), path, Option(ctx), data, stat, responseMetadata(sendTimeMs))), callback(GetDataResponse(Code.get(rc), path, Option(ctx), data, stat, responseMetadata(sendTimeMs)))
}, ctx.orNull) }, ctx.orNull)
case GetChildrenRequest(path, _, ctx) => case GetChildrenRequest(path, _, ctx) =>
zooKeeper.getChildren(path, shouldWatch(request), new Children2Callback { zooKeeper.getChildren(path, shouldWatch(request), new Children2Callback {

View File

@ -31,6 +31,7 @@ import org.junit.{After, Before, Rule, Test}
import org.junit.rules.Timeout import org.junit.rules.Timeout
import org.scalatest.Assertions.intercept import org.scalatest.Assertions.intercept
import scala.annotation.nowarn
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
/** /**
@ -92,6 +93,7 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with
PlaintextAdminIntegrationTest.checkInvalidAlterConfigs(zkClient, servers, client) PlaintextAdminIntegrationTest.checkInvalidAlterConfigs(zkClient, servers, client)
} }
@nowarn("cat=deprecation")
@Test @Test
def testInvalidAlterConfigsDueToPolicy(): Unit = { def testInvalidAlterConfigsDueToPolicy(): Unit = {
client = Admin.create(createConfig) client = Admin.create(createConfig)

View File

@ -59,6 +59,7 @@ import org.junit.Assert._
import org.junit.{After, Before, Test} import org.junit.{After, Before, Test}
import org.scalatest.Assertions.intercept import org.scalatest.Assertions.intercept
import scala.annotation.nowarn
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
import scala.collection.mutable import scala.collection.mutable
import scala.collection.mutable.Buffer import scala.collection.mutable.Buffer
@ -949,6 +950,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
consumeRecords(consumer) consumeRecords(consumer)
} }
@nowarn("cat=deprecation")
@Test @Test
def testPatternSubscriptionWithNoTopicAccess(): Unit = { def testPatternSubscriptionWithNoTopicAccess(): Unit = {
createTopic(topic) createTopic(topic)
@ -985,6 +987,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
assertEquals(Collections.singleton(topic), e.unauthorizedTopics()) assertEquals(Collections.singleton(topic), e.unauthorizedTopics())
} }
@nowarn("cat=deprecation")
@Test @Test
def testPatternSubscriptionWithTopicAndGroupRead(): Unit = { def testPatternSubscriptionWithTopicAndGroupRead(): Unit = {
createTopic(topic) createTopic(topic)
@ -1016,6 +1019,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
assertTrue(consumer.assignment().isEmpty) assertTrue(consumer.assignment().isEmpty)
} }
@nowarn("cat=deprecation")
@Test @Test
def testPatternSubscriptionMatchingInternalTopic(): Unit = { def testPatternSubscriptionMatchingInternalTopic(): Unit = {
createTopic(topic) createTopic(topic)

View File

@ -36,6 +36,7 @@ import org.junit.Assert._
import org.junit.{After, Before, Test} import org.junit.{After, Before, Test}
import org.scalatest.Assertions.fail import org.scalatest.Assertions.fail
import scala.annotation.nowarn
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
import scala.collection.mutable.Buffer import scala.collection.mutable.Buffer
import scala.concurrent.ExecutionException import scala.concurrent.ExecutionException
@ -102,6 +103,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness {
* 1. Send with null key/value/partition-id should be accepted; send with null topic should be rejected. * 1. Send with null key/value/partition-id should be accepted; send with null topic should be rejected.
* 2. Last message of the non-blocking send should return the correct offset metadata * 2. Last message of the non-blocking send should return the correct offset metadata
*/ */
@nowarn("cat=deprecation")
@Test @Test
def testSendOffset(): Unit = { def testSendOffset(): Unit = {
val producer = createProducer(brokerList) val producer = createProducer(brokerList)

View File

@ -29,6 +29,7 @@ import org.apache.kafka.common.requests.{FindCoordinatorRequest, FindCoordinator
import org.junit.Assert._ import org.junit.Assert._
import org.junit.{After, Ignore, Test} import org.junit.{After, Ignore, Test}
import scala.annotation.nowarn
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
import scala.collection.{Seq, mutable} import scala.collection.{Seq, mutable}
@ -83,6 +84,7 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging {
* 1. Produce a bunch of messages * 1. Produce a bunch of messages
* 2. Then consume the messages while killing and restarting brokers at random * 2. Then consume the messages while killing and restarting brokers at random
*/ */
@nowarn("cat=deprecation")
def consumeWithBrokerFailures(numIters: Int): Unit = { def consumeWithBrokerFailures(numIters: Int): Unit = {
val numRecords = 1000 val numRecords = 1000
val producer = createProducer() val producer = createProducer()
@ -379,6 +381,7 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging {
checkCloseDuringRebalance("group1", topic, executor, true) checkCloseDuringRebalance("group1", topic, executor, true)
} }
@nowarn("cat=deprecation")
private def checkCloseDuringRebalance(groupId: String, topic: String, executor: ExecutorService, brokersAvailableDuringClose: Boolean): Unit = { private def checkCloseDuringRebalance(groupId: String, topic: String, executor: ExecutorService, brokersAvailableDuringClose: Boolean): Unit = {
def subscribeAndPoll(consumer: KafkaConsumer[Array[Byte], Array[Byte]], revokeSemaphore: Option[Semaphore] = None): Future[Any] = { def subscribeAndPoll(consumer: KafkaConsumer[Array[Byte], Array[Byte]], revokeSemaphore: Option[Semaphore] = None): Future[Any] = {

View File

@ -45,6 +45,7 @@ import org.junit.Assert._
import org.junit.{After, Before, Ignore, Test} import org.junit.{After, Before, Ignore, Test}
import org.scalatest.Assertions.intercept import org.scalatest.Assertions.intercept
import scala.annotation.nowarn
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
import scala.collection.Seq import scala.collection.Seq
import scala.concurrent.duration.Duration import scala.concurrent.duration.Duration
@ -2176,6 +2177,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
/** /**
* The AlterConfigs API is deprecated and should not support altering log levels * The AlterConfigs API is deprecated and should not support altering log levels
*/ */
@nowarn("cat=deprecation")
@Test @Test
@Ignore // To be re-enabled once KAFKA-8779 is resolved @Ignore // To be re-enabled once KAFKA-8779 is resolved
def testAlterConfigsForLog4jLogLevelsDoesNotWork(): Unit = { def testAlterConfigsForLog4jLogLevelsDoesNotWork(): Unit = {
@ -2227,6 +2229,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
object PlaintextAdminIntegrationTest { object PlaintextAdminIntegrationTest {
@nowarn("cat=deprecation")
def checkValidAlterConfigs(client: Admin, topicResource1: ConfigResource, topicResource2: ConfigResource): Unit = { def checkValidAlterConfigs(client: Admin, topicResource1: ConfigResource, topicResource2: ConfigResource): Unit = {
// Alter topics // Alter topics
var topicConfigEntries1 = Seq( var topicConfigEntries1 = Seq(
@ -2289,6 +2292,7 @@ object PlaintextAdminIntegrationTest {
assertEquals("0.9", configs.get(topicResource2).get(LogConfig.MinCleanableDirtyRatioProp).value) assertEquals("0.9", configs.get(topicResource2).get(LogConfig.MinCleanableDirtyRatioProp).value)
} }
@nowarn("cat=deprecation")
def checkInvalidAlterConfigs(zkClient: KafkaZkClient, servers: Seq[KafkaServer], client: Admin): Unit = { def checkInvalidAlterConfigs(zkClient: KafkaZkClient, servers: Seq[KafkaServer], client: Admin): Unit = {
// Create topics // Create topics
val topic1 = "invalid-alter-configs-topic-1" val topic1 = "invalid-alter-configs-topic-1"
@ -2356,12 +2360,12 @@ object PlaintextAdminIntegrationTest {
assertEquals(Defaults.LogCleanerMinCleanRatio.toString, assertEquals(Defaults.LogCleanerMinCleanRatio.toString,
configs.get(topicResource1).get(LogConfig.MinCleanableDirtyRatioProp).value) configs.get(topicResource1).get(LogConfig.MinCleanableDirtyRatioProp).value)
assertEquals(Defaults.CompressionType.toString, assertEquals(Defaults.CompressionType,
configs.get(topicResource1).get(LogConfig.CompressionTypeProp).value) configs.get(topicResource1).get(LogConfig.CompressionTypeProp).value)
assertEquals("snappy", configs.get(topicResource2).get(LogConfig.CompressionTypeProp).value) assertEquals("snappy", configs.get(topicResource2).get(LogConfig.CompressionTypeProp).value)
assertEquals(Defaults.CompressionType.toString, configs.get(brokerResource).get(KafkaConfig.CompressionTypeProp).value) assertEquals(Defaults.CompressionType, configs.get(brokerResource).get(KafkaConfig.CompressionTypeProp).value)
} }
} }

View File

@ -30,6 +30,7 @@ import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
import org.junit.Assert.{assertEquals, assertTrue} import org.junit.Assert.{assertEquals, assertTrue}
import org.junit.{After, Assert, Before, Test} import org.junit.{After, Assert, Before, Test}
import scala.annotation.nowarn
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
import scala.collection.Seq import scala.collection.Seq
import scala.compat.java8.OptionConverters._ import scala.compat.java8.OptionConverters._
@ -46,6 +47,7 @@ abstract class AuthorizationAdmin {
// Note: this test currently uses the deprecated SimpleAclAuthorizer to ensure we have test coverage // Note: this test currently uses the deprecated SimpleAclAuthorizer to ensure we have test coverage
// It must be replaced with the new AclAuthorizer when SimpleAclAuthorizer is removed // It must be replaced with the new AclAuthorizer when SimpleAclAuthorizer is removed
class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetup { class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetup {
@nowarn("cat=deprecation")
val authorizationAdmin: AuthorizationAdmin = new LegacyAuthorizationAdmin val authorizationAdmin: AuthorizationAdmin = new LegacyAuthorizationAdmin
this.serverConfig.setProperty(KafkaConfig.ZkEnableSecureAclsProp, "true") this.serverConfig.setProperty(KafkaConfig.ZkEnableSecureAclsProp, "true")

View File

@ -63,6 +63,7 @@ import org.junit.Assert._
import org.junit.{After, Before, Ignore, Test} import org.junit.{After, Before, Ignore, Test}
import org.scalatest.Assertions.intercept import org.scalatest.Assertions.intercept
import scala.annotation.nowarn
import scala.collection._ import scala.collection._
import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.ArrayBuffer
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
@ -1324,6 +1325,7 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet
}.mkString(",") }.mkString(",")
} }
@nowarn("cat=deprecation")
private def alterAdvertisedListener(adminClient: Admin, externalAdminClient: Admin, oldHost: String, newHost: String): Unit = { private def alterAdvertisedListener(adminClient: Admin, externalAdminClient: Admin, oldHost: String, newHost: String): Unit = {
val configs = servers.map { server => val configs = servers.map { server =>
val resource = new ConfigResource(ConfigResource.Type.BROKER, server.config.brokerId.toString) val resource = new ConfigResource(ConfigResource.Type.BROKER, server.config.brokerId.toString)
@ -1350,6 +1352,7 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet
assertTrue(s"Advertised listener update not propagated by controller: $endpoints", altered) assertTrue(s"Advertised listener update not propagated by controller: $endpoints", altered)
} }
@nowarn("cat=deprecation")
private def alterConfigsOnServer(server: KafkaServer, props: Properties): Unit = { private def alterConfigsOnServer(server: KafkaServer, props: Properties): Unit = {
val configEntries = props.asScala.map { case (k, v) => new ConfigEntry(k, v) }.toList.asJava val configEntries = props.asScala.map { case (k, v) => new ConfigEntry(k, v) }.toList.asJava
val newConfig = new Config(configEntries) val newConfig = new Config(configEntries)
@ -1358,6 +1361,7 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet
props.asScala.foreach { case (k, v) => waitForConfigOnServer(server, k, v) } props.asScala.foreach { case (k, v) => waitForConfigOnServer(server, k, v) }
} }
@nowarn("cat=deprecation")
private def alterConfigs(servers: Seq[KafkaServer], adminClient: Admin, props: Properties, private def alterConfigs(servers: Seq[KafkaServer], adminClient: Admin, props: Properties,
perBrokerConfig: Boolean): AlterConfigsResult = { perBrokerConfig: Boolean): AlterConfigsResult = {
val configEntries = props.asScala.map { case (k, v) => new ConfigEntry(k, v) }.toList.asJava val configEntries = props.asScala.map { case (k, v) => new ConfigEntry(k, v) }.toList.asJava

View File

@ -53,7 +53,7 @@ class MetricsDuringTopicCreationDeletionTest extends KafkaServerTestHarness with
.map(KafkaConfig.fromProps(_, overridingProps)) .map(KafkaConfig.fromProps(_, overridingProps))
@Before @Before
override def setUp: Unit = { override def setUp(): Unit = {
// Do some Metrics Registry cleanup by removing the metrics that this test checks. // Do some Metrics Registry cleanup by removing the metrics that this test checks.
// This is a test workaround to the issue that prior harness runs may have left a populated registry. // This is a test workaround to the issue that prior harness runs may have left a populated registry.
// see https://issues.apache.org/jira/browse/KAFKA-4605 // see https://issues.apache.org/jira/browse/KAFKA-4605

View File

@ -40,6 +40,8 @@ import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, AlterConfigsRes
import org.junit.Assert._ import org.junit.Assert._
import org.scalatest.Assertions.intercept import org.scalatest.Assertions.intercept
import scala.annotation.nowarn
class UncleanLeaderElectionTest extends ZooKeeperTestHarness { class UncleanLeaderElectionTest extends ZooKeeperTestHarness {
val brokerId1 = 0 val brokerId1 = 0
val brokerId2 = 1 val brokerId2 = 1
@ -347,6 +349,7 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness {
assertEquals(List("first", "third"), consumeAllMessages(topic, 2)) assertEquals(List("first", "third"), consumeAllMessages(topic, 2))
} }
@nowarn("cat=deprecation")
private def alterTopicConfigs(adminClient: Admin, topic: String, topicConfigs: Properties): AlterConfigsResult = { private def alterTopicConfigs(adminClient: Admin, topic: String, topicConfigs: Properties): AlterConfigsResult = {
val configEntries = topicConfigs.asScala.map { case (k, v) => new ConfigEntry(k, v) }.toList.asJava val configEntries = topicConfigs.asScala.map { case (k, v) => new ConfigEntry(k, v) }.toList.asJava
val newConfig = new Config(configEntries) val newConfig = new Config(configEntries)

View File

@ -31,6 +31,8 @@ import scala.util.Random
import kafka.utils.TestUtils import kafka.utils.TestUtils
import org.apache.kafka.common.errors.InvalidOffsetException import org.apache.kafka.common.errors.InvalidOffsetException
import scala.annotation.nowarn
class OffsetIndexTest { class OffsetIndexTest {
var idx: OffsetIndex = null var idx: OffsetIndex = null
@ -47,7 +49,8 @@ class OffsetIndexTest {
if(this.idx != null) if(this.idx != null)
this.idx.file.delete() this.idx.file.delete()
} }
@nowarn("cat=deprecation")
@Test @Test
def randomLookupTest(): Unit = { def randomLookupTest(): Unit = {
assertEquals("Not present value should return physical offset 0.", OffsetPosition(idx.baseOffset, 0), idx.lookup(92L)) assertEquals("Not present value should return physical offset 0.", OffsetPosition(idx.baseOffset, 0), idx.lookup(92L))

View File

@ -29,6 +29,7 @@ import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.protocol.ApiKeys
import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, RequestHeader, ResponseHeader} import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, RequestHeader, ResponseHeader}
import scala.annotation.nowarn
import scala.collection.Seq import scala.collection.Seq
import scala.reflect.ClassTag import scala.reflect.ClassTag
@ -86,7 +87,7 @@ abstract class BaseRequestTest extends IntegrationTestHarness {
} }
def receive[T <: AbstractResponse](socket: Socket, apiKey: ApiKeys, version: Short) def receive[T <: AbstractResponse](socket: Socket, apiKey: ApiKeys, version: Short)
(implicit classTag: ClassTag[T], nn: NotNothing[T]): T = { (implicit classTag: ClassTag[T], @nowarn("cat=unused") nn: NotNothing[T]): T = {
val incoming = new DataInputStream(socket.getInputStream) val incoming = new DataInputStream(socket.getInputStream)
val len = incoming.readInt() val len = incoming.readInt()

View File

@ -35,7 +35,7 @@ import kafka.utils.timer.MockTimer
import kafka.utils.{MockScheduler, MockTime, TestUtils} import kafka.utils.{MockScheduler, MockTime, TestUtils}
import kafka.zk.KafkaZkClient import kafka.zk.KafkaZkClient
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState
import org.apache.kafka.common.message.StopReplicaRequestData.{StopReplicaPartitionState, StopReplicaTopicState} import org.apache.kafka.common.message.StopReplicaRequestData.StopReplicaPartitionState
import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.record._ import org.apache.kafka.common.record._

View File

@ -16,7 +16,7 @@ package kafka.server
import java.util import java.util
import java.util.concurrent.{Executors, Future, TimeUnit} import java.util.concurrent.{Executors, Future, TimeUnit}
import java.util.{Collections, LinkedHashMap, Optional, Properties} import java.util.{Collections, Optional, Properties}
import kafka.api.LeaderAndIsr import kafka.api.LeaderAndIsr
import kafka.log.LogConfig import kafka.log.LogConfig

View File

@ -40,7 +40,7 @@ class LeaderEpochFileCacheTest {
override def write(epochs: Seq[EpochEntry]): Unit = this.epochs = epochs override def write(epochs: Seq[EpochEntry]): Unit = this.epochs = epochs
override def read(): Seq[EpochEntry] = this.epochs override def read(): Seq[EpochEntry] = this.epochs
} }
private val cache = new LeaderEpochFileCache(tp, logEndOffset _, checkpoint) private val cache = new LeaderEpochFileCache(tp, () => logEndOffset, checkpoint)
@Test @Test
def shouldAddEpochAndMessageOffsetToCache() = { def shouldAddEpochAndMessageOffsetToCache() = {
@ -231,12 +231,12 @@ class LeaderEpochFileCacheTest {
val checkpoint = new LeaderEpochCheckpointFile(new File(checkpointPath)) val checkpoint = new LeaderEpochCheckpointFile(new File(checkpointPath))
//Given //Given
val cache = new LeaderEpochFileCache(tp, logEndOffset _, checkpoint) val cache = new LeaderEpochFileCache(tp, () => logEndOffset, checkpoint)
cache.assign(epoch = 2, startOffset = 6) cache.assign(epoch = 2, startOffset = 6)
//When //When
val checkpoint2 = new LeaderEpochCheckpointFile(new File(checkpointPath)) val checkpoint2 = new LeaderEpochCheckpointFile(new File(checkpointPath))
val cache2 = new LeaderEpochFileCache(tp, logEndOffset _, checkpoint2) val cache2 = new LeaderEpochFileCache(tp, () => logEndOffset, checkpoint2)
//Then //Then
assertEquals(1, cache2.epochEntries.size) assertEquals(1, cache2.epochEntries.size)

View File

@ -19,10 +19,14 @@ package kafka.tools
import kafka.consumer.BaseConsumerRecord import kafka.consumer.BaseConsumerRecord
import org.apache.kafka.common.record.{RecordBatch, TimestampType} import org.apache.kafka.common.record.{RecordBatch, TimestampType}
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
import org.junit.Assert._ import org.junit.Assert._
import org.junit.Test import org.junit.Test
import scala.annotation.nowarn
@nowarn("cat=deprecation")
class MirrorMakerTest { class MirrorMakerTest {
@Test @Test

View File

@ -38,7 +38,7 @@ class ShutdownableThreadTest {
} }
val latch = new CountDownLatch(1) val latch = new CountDownLatch(1)
val thread = new ShutdownableThread("shutdownable-thread-test") { val thread = new ShutdownableThread("shutdownable-thread-test") {
override def doWork: Unit = { override def doWork(): Unit = {
latch.countDown() latch.countDown()
throw new FatalExitError throw new FatalExitError
} }

View File

@ -28,7 +28,7 @@ ext {
// Add Scala version // Add Scala version
def defaultScala212Version = '2.12.11' def defaultScala212Version = '2.12.11'
def defaultScala213Version = '2.13.1' def defaultScala213Version = '2.13.2'
if (hasProperty('scalaVersion')) { if (hasProperty('scalaVersion')) {
if (scalaVersion == '2.12') { if (scalaVersion == '2.12') {
versions["scala"] = defaultScala212Version versions["scala"] = defaultScala212Version
@ -102,9 +102,9 @@ versions += [
powermock: "2.0.7", powermock: "2.0.7",
reflections: "0.9.12", reflections: "0.9.12",
rocksDB: "5.18.4", rocksDB: "5.18.4",
scalaCollectionCompat: "2.1.4", scalaCollectionCompat: "2.1.6",
scalafmt: "1.5.1", scalafmt: "1.5.1",
scalaJava8Compat : "0.9.0", scalaJava8Compat : "0.9.1",
scalatest: "3.0.8", scalatest: "3.0.8",
scoverage: "1.4.1", scoverage: "1.4.1",
scoveragePlugin: "4.0.1", scoveragePlugin: "4.0.1",