From 90bbeedf52f4b6a411e9630dd132583afa4cd428 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Wed, 1 Apr 2020 06:20:48 -0700 Subject: [PATCH] MINOR: Fix Scala 2.13 compiler warnings (#8390) Once Scala 2.13.2 is officially released, I will submit a follow up PR that enables `-Xfatal-warnings` with the necessary warning exclusions. Compiler warning exclusions were only introduced in 2.13.2 and hence why we have to wait for that. I used a snapshot build to test it in the meantime. Changes: * Remove Deprecated annotation from internal request classes * Class.newInstance is deprecated in favor of Class.getConstructor().newInstance * Replace deprecated JavaConversions with CollectionConverters * Remove unused kafka.cluster.Cluster * Don't use Map and Set methods deprecated in 2.13: - collection.Map +, ++, -, --, mapValues, filterKeys, retain - collection.Set +, ++, -, -- * Add scala-collection-compat dependency to streams-scala and update version to 2.1.4. * Replace usages of deprecated Either.get and Either.right * Replace usage of deprecated Integer(String) constructor * `import scala.language.implicitConversions` is not needed in Scala 2.13 * Replace usage of deprecated `toIterator`, `Traversable`, `seq`, `reverseMap`, `hasDefiniteSize` * Replace usage of deprecated alterConfigs with incrementalAlterConfigs where possible * Fix implicit widening conversions from Long/Int to Double/Float * Avoid implicit conversions to String * Eliminate usage of deprecated procedure syntax * Remove `println`in `LogValidatorTest` instead of fixing the compiler warning since tests should not `println`. * Eliminate implicit conversion from Array to Seq * Remove unnecessary usage of 3 argument assertEquals * Replace `toStream` with `iterator` * Do not use deprecated SaslConfigs.DEFAULT_SASL_ENABLED_MECHANISMS * Replace StringBuilder.newBuilder with new StringBuilder * Rename AclBuffers to AclSeqs and remove usage of `filterKeys` * More consistent usage of Set/Map in Controller classes: this also fixes deprecated warnings with Scala 2.13 * Add spotBugs exclusion for inliner artifact in KafkaApis with Scala 2.12. Reviewers: Manikumar Reddy , Chia-Ping Tsai --- build.gradle | 1 + .../common/requests/ListOffsetRequest.java | 3 +- .../common/requests/ListOffsetResponse.java | 2 - .../common/requests/OffsetCommitRequest.java | 4 +- .../kafka/common/utils/SecurityUtils.java | 5 +- .../kafka/common/message/MessageTest.java | 4 +- .../connect/runtime/ConnectorConfig.java | 2 +- .../connect/runtime/AbstractHerderTest.java | 4 +- .../ConnectorPluginsResourceTest.java | 2 +- core/src/main/scala/kafka/Kafka.scala | 2 +- .../main/scala/kafka/admin/AclCommand.scala | 12 ++-- .../admin/BrokerApiVersionsCommand.scala | 2 +- .../scala/kafka/admin/ConfigCommand.scala | 3 +- .../kafka/admin/ConsumerGroupCommand.scala | 41 ++++++------ .../kafka/admin/DelegationTokenCommand.scala | 2 +- .../kafka/admin/DeleteRecordsCommand.scala | 2 +- .../kafka/admin/LeaderElectionCommand.scala | 2 +- .../scala/kafka/admin/LogDirsCommand.scala | 4 +- ...referredReplicaLeaderElectionCommand.scala | 2 +- .../admin/ReassignPartitionsCommand.scala | 34 +++++----- .../main/scala/kafka/admin/TopicCommand.scala | 4 +- .../kafka/admin/ZkSecurityMigrator.scala | 2 +- core/src/main/scala/kafka/api/package.scala | 2 +- .../src/main/scala/kafka/cluster/Broker.scala | 2 +- .../main/scala/kafka/cluster/Cluster.scala | 45 -------------- .../main/scala/kafka/cluster/Partition.scala | 8 +-- .../kafka/common/InterBrokerSendThread.scala | 2 +- .../controller/ControllerChannelManager.scala | 45 +++++++------- .../kafka/controller/ControllerContext.scala | 59 ++++++++++-------- .../controller/ControllerEventManager.scala | 2 +- .../kafka/controller/KafkaController.scala | 33 +++++----- .../controller/PartitionStateMachine.scala | 2 +- .../controller/ReplicaStateMachine.scala | 31 ++++------ .../controller/TopicDeletionManager.scala | 4 +- .../coordinator/group/GroupCoordinator.scala | 12 ++-- .../coordinator/group/GroupMetadata.scala | 20 +++--- .../group/GroupMetadataManager.scala | 14 +++-- .../transaction/ProducerIdManager.scala | 2 +- .../transaction/TransactionCoordinator.scala | 12 ++-- .../TransactionMarkerChannelManager.scala | 2 +- ...actionMarkerRequestCompletionHandler.scala | 2 +- .../transaction/TransactionStateManager.scala | 10 +-- core/src/main/scala/kafka/log/Log.scala | 8 +-- .../src/main/scala/kafka/log/LogCleaner.scala | 24 +++---- .../scala/kafka/log/LogCleanerManager.scala | 2 +- core/src/main/scala/kafka/log/LogConfig.scala | 2 +- .../src/main/scala/kafka/log/LogManager.scala | 6 +- .../src/main/scala/kafka/log/LogSegment.scala | 2 +- .../main/scala/kafka/log/LogValidator.scala | 2 +- .../main/scala/kafka/log/OffsetIndex.scala | 2 +- .../kafka/log/ProducerStateManager.scala | 6 +- core/src/main/scala/kafka/log/TimeIndex.scala | 2 +- .../scala/kafka/network/RequestChannel.scala | 2 +- .../scala/kafka/network/SocketServer.scala | 4 +- .../scala/kafka/security/auth/Resource.scala | 2 +- .../security/auth/SimpleAclAuthorizer.scala | 2 +- .../security/authorizer/AclAuthorizer.scala | 29 +++++---- .../kafka/security/authorizer/AclEntry.scala | 2 +- .../authorizer/AuthorizerWrapper.scala | 4 +- .../kafka/server/AbstractFetcherThread.scala | 2 +- .../scala/kafka/server/AdminManager.scala | 9 +-- .../server/BrokerMetadataCheckpoint.scala | 2 +- .../kafka/server/ClientQuotaManager.scala | 11 ++-- .../server/ClientRequestQuotaManager.scala | 2 +- .../scala/kafka/server/ConfigHandler.scala | 12 ++-- .../kafka/server/DelayedElectLeader.scala | 4 +- .../kafka/server/DelegationTokenManager.scala | 2 +- .../kafka/server/DynamicBrokerConfig.scala | 62 +++++++++++-------- .../scala/kafka/server/DynamicConfig.scala | 2 +- .../kafka/server/DynamicConfigManager.scala | 2 +- .../scala/kafka/server/FetchSession.scala | 2 +- .../main/scala/kafka/server/KafkaApis.scala | 51 +++++++-------- .../main/scala/kafka/server/KafkaConfig.scala | 4 +- .../kafka/server/KafkaRequestHandler.scala | 2 +- .../main/scala/kafka/server/KafkaServer.scala | 12 ++-- .../scala/kafka/server/MetadataCache.scala | 4 +- .../server/ReplicaAlterLogDirsThread.scala | 2 +- .../server/ReplicaFetcherBlockingSend.scala | 2 +- .../kafka/server/ReplicaFetcherThread.scala | 2 +- .../scala/kafka/server/ReplicaManager.scala | 12 ++-- .../server/ReplicationQuotaManager.scala | 2 +- .../scala/kafka/tools/ConsoleConsumer.scala | 2 +- .../scala/kafka/tools/ConsoleProducer.scala | 2 +- .../kafka/tools/ConsumerPerformance.scala | 4 +- .../scala/kafka/tools/DumpLogSegments.scala | 2 +- .../scala/kafka/tools/EndToEndLatency.scala | 4 +- .../scala/kafka/tools/GetOffsetShell.scala | 4 +- core/src/main/scala/kafka/tools/JmxTool.scala | 6 +- .../main/scala/kafka/tools/MirrorMaker.scala | 4 +- .../kafka/tools/ReplicaVerificationTool.scala | 18 +++--- .../main/scala/kafka/utils/CoreUtils.scala | 2 +- .../main/scala/kafka/utils/Implicits.scala | 2 +- core/src/main/scala/kafka/utils/Json.scala | 2 +- .../scala/kafka/utils/Log4jController.scala | 2 +- core/src/main/scala/kafka/utils/Pool.scala | 6 +- .../kafka/utils/VerifiableProperties.scala | 2 +- .../scala/kafka/utils/json/DecodeJson.scala | 6 +- .../scala/kafka/utils/json/JsonArray.scala | 4 +- .../scala/kafka/utils/json/JsonObject.scala | 2 +- .../main/scala/kafka/zk/AdminZkClient.scala | 7 ++- .../main/scala/kafka/zk/KafkaZkClient.scala | 6 +- core/src/main/scala/kafka/zk/ZkData.scala | 6 +- .../kafka/zookeeper/ZooKeeperClient.scala | 2 +- .../ZooKeeperMainWithTlsSupportForKafka.scala | 4 +- .../ReassignPartitionsIntegrationTest.scala | 2 +- .../kafka/api/AbstractConsumerTest.scala | 6 +- ...minClientWithPoliciesIntegrationTest.scala | 2 +- .../kafka/api/AuthorizerIntegrationTest.scala | 2 +- .../kafka/api/BaseAdminIntegrationTest.scala | 2 +- .../kafka/api/BaseConsumerTest.scala | 2 +- .../kafka/api/BaseProducerSendTest.scala | 2 +- .../integration/kafka/api/BaseQuotaTest.scala | 10 +-- .../kafka/api/ConsumerBounceTest.scala | 2 +- .../kafka/api/CustomQuotaCallbackTest.scala | 8 +-- ...gationTokenEndToEndAuthorizationTest.scala | 2 +- .../DescribeAuthorizedOperationsTest.scala | 2 +- .../kafka/api/EndToEndAuthorizationTest.scala | 4 +- .../kafka/api/EndToEndClusterIdTest.scala | 2 +- .../api/GroupAuthorizerIntegrationTest.scala | 4 +- .../api/GroupCoordinatorIntegrationTest.scala | 2 +- .../integration/kafka/api/MetricsTest.scala | 2 +- .../api/PlaintextAdminIntegrationTest.scala | 10 ++- .../kafka/api/PlaintextConsumerTest.scala | 8 +-- .../kafka/api/ProducerCompressionTest.scala | 2 +- ...aslClientsWithInvalidCredentialsTest.scala | 2 +- .../api/SaslEndToEndAuthorizationTest.scala | 2 +- .../api/SaslMultiMechanismConsumerTest.scala | 2 +- ...aslScramSslEndToEndAuthorizationTest.scala | 2 +- .../api/SaslSslAdminIntegrationTest.scala | 2 +- .../kafka/api/SslAdminIntegrationTest.scala | 2 +- .../kafka/api/TransactionsBounceTest.scala | 2 +- .../api/TransactionsExpirationTest.scala | 2 +- .../kafka/api/TransactionsTest.scala | 2 +- .../network/DynamicConnectionQuotaTest.scala | 4 +- .../DynamicBrokerReconfigurationTest.scala | 23 +++++-- .../server/GssapiAuthenticationTest.scala | 2 +- .../kafka/server/ScramServerStartupTest.scala | 2 +- .../kafka/security/minikdc/MiniKdc.scala | 2 +- .../kafka/tools/LogCompactionTester.scala | 4 +- .../scala/kafka/zk/LiteralAclStoreTest.scala | 4 +- .../kafka/ReplicationQuotasTestRig.scala | 10 +-- .../other/kafka/TestLinearWriteSpeed.scala | 2 +- .../kafka/TestPurgatoryPerformance.scala | 2 +- .../scala/unit/kafka/KafkaConfigTest.scala | 2 +- .../unit/kafka/admin/AddPartitionsTest.scala | 10 +-- .../unit/kafka/admin/AdminRackAwareTest.scala | 2 +- .../unit/kafka/admin/ConfigCommandTest.scala | 2 +- .../admin/ConsumerGroupCommandTest.scala | 6 +- .../admin/DelegationTokenCommandTest.scala | 2 +- .../unit/kafka/admin/DeleteTopicTest.scala | 4 +- .../admin/LeaderElectionCommandTest.scala | 2 +- ...rredReplicaLeaderElectionCommandTest.scala | 4 +- .../admin/ReassignPartitionsUnitTest.scala | 45 +++++++------- .../admin/ResetConsumerGroupOffsetTest.scala | 27 ++++---- .../TopicCommandWithAdminClientTest.scala | 2 +- .../kafka/cluster/AssignmentStateTest.scala | 3 +- .../kafka/cluster/PartitionLockTest.scala | 3 +- .../unit/kafka/cluster/PartitionTest.scala | 4 +- .../ControllerChannelManagerTest.scala | 4 +- .../controller/ControllerContextTest.scala | 2 +- .../ControllerEventManagerTest.scala | 16 +++-- .../ControllerIntegrationTest.scala | 15 ++--- .../PartitionStateMachineTest.scala | 22 +++---- .../controller/ReplicaStateMachineTest.scala | 6 +- .../controller/TopicDeletionManagerTest.scala | 6 +- .../AbstractCoordinatorConcurrencyTest.scala | 2 +- .../group/GroupCoordinatorTest.scala | 2 +- .../group/GroupMetadataManagerTest.scala | 2 +- .../coordinator/group/GroupMetadataTest.scala | 2 +- ...ransactionCoordinatorConcurrencyTest.scala | 2 +- .../transaction/TransactionLogTest.scala | 2 +- .../TransactionMarkerChannelManagerTest.scala | 14 ++--- .../transaction/TransactionMetadataTest.scala | 39 +++++++----- .../TransactionStateManagerTest.scala | 2 +- ...tricsDuringTopicCreationDeletionTest.scala | 4 +- .../UncleanLeaderElectionTest.scala | 17 +++-- .../kafka/log/BrokerCompressionTest.scala | 2 +- .../kafka/log/LogCleanerIntegrationTest.scala | 7 +-- .../log/LogCleanerLagIntegrationTest.scala | 3 +- .../kafka/log/LogCleanerManagerTest.scala | 4 +- ...gCleanerParameterizedIntegrationTest.scala | 2 +- .../scala/unit/kafka/log/LogCleanerTest.scala | 4 +- .../scala/unit/kafka/log/LogSegmentTest.scala | 2 +- .../test/scala/unit/kafka/log/LogTest.scala | 12 ++-- .../unit/kafka/log/LogValidatorTest.scala | 3 +- .../scala/unit/kafka/log/OffsetMapTest.scala | 2 +- .../unit/kafka/metrics/MetricsTest.scala | 2 +- .../kafka/network/RequestChannelTest.scala | 2 +- .../unit/kafka/network/SocketServerTest.scala | 11 ++-- .../security/auth/ZkAuthorizationTest.scala | 4 +- .../authorizer/AclAuthorizerTest.scala | 2 +- .../security/authorizer/AclEntryTest.scala | 2 +- .../DelegationTokenManagerTest.scala | 2 +- .../AbstractApiVersionsRequestTest.scala | 2 +- .../AbstractCreateTopicsRequestTest.scala | 2 +- .../server/AbstractFetcherManagerTest.scala | 4 +- .../server/AbstractFetcherThreadTest.scala | 4 +- .../AddPartitionsToTxnRequestTest.scala | 2 +- .../AlterReplicaLogDirsRequestTest.scala | 2 +- .../server/BrokerEpochIntegrationTest.scala | 4 +- .../kafka/server/ClientQuotaManagerTest.scala | 7 ++- .../server/ClientQuotasRequestTest.scala | 6 +- .../server/CreateTopicsRequestTest.scala | 2 +- .../CreateTopicsRequestWithPolicyTest.scala | 2 +- .../kafka/server/DelayedOperationTest.scala | 2 +- ...legationTokenRequestsOnPlainTextTest.scala | 2 +- .../server/DelegationTokenRequestsTest.scala | 2 +- ...nRequestsWithDisableTokenFeatureTest.scala | 2 +- .../server/DeleteTopicsRequestTest.scala | 2 +- .../server/DynamicBrokerConfigTest.scala | 2 +- .../server/DynamicConfigChangeTest.scala | 2 +- .../kafka/server/EdgeCaseRequestTest.scala | 2 +- .../server/FetchRequestMaxBytesTest.scala | 2 +- .../unit/kafka/server/FetchRequestTest.scala | 2 +- .../unit/kafka/server/KafkaApisTest.scala | 2 +- .../kafka/server/LeaderElectionTest.scala | 4 +- .../kafka/server/ListOffsetsRequestTest.scala | 2 +- .../unit/kafka/server/LogDirFailureTest.scala | 2 +- .../unit/kafka/server/LogOffsetTest.scala | 2 +- .../unit/kafka/server/MetadataCacheTest.scala | 2 +- .../kafka/server/MetadataRequestTest.scala | 2 +- .../OffsetsForLeaderEpochRequestTest.scala | 2 +- .../kafka/server/ProduceRequestTest.scala | 2 +- .../ReplicaAlterLogDirsThreadTest.scala | 2 +- .../server/ReplicaFetcherThreadTest.scala | 2 +- .../server/ReplicaManagerQuotasTest.scala | 2 +- .../kafka/server/ReplicaManagerTest.scala | 2 +- .../server/ReplicationQuotaManagerTest.scala | 2 +- .../kafka/server/ReplicationQuotasTest.scala | 4 +- .../unit/kafka/server/RequestQuotaTest.scala | 2 +- .../kafka/server/ServerShutdownTest.scala | 4 +- .../unit/kafka/server/ServerStartupTest.scala | 2 +- .../kafka/server/StopReplicaRequestTest.scala | 2 +- ...venReplicationProtocolAcceptanceTest.scala | 2 +- .../epoch/LeaderEpochIntegrationTest.scala | 7 ++- .../kafka/tools/ConsoleConsumerTest.scala | 2 +- .../unit/kafka/tools/MirrorMakerTest.scala | 2 +- .../unit/kafka/utils/CoreUtilsTest.scala | 4 +- .../scala/unit/kafka/utils/JsonTest.scala | 9 +-- .../scala/unit/kafka/utils/TestUtils.scala | 29 +-------- .../unit/kafka/zk/AdminZkClientTest.scala | 4 +- .../unit/kafka/zk/KafkaZkClientTest.scala | 9 +-- .../zk/ReassignPartitionsZNodeTest.scala | 10 +-- .../unit/kafka/zk/ZooKeeperTestHarness.scala | 2 +- .../kafka/zookeeper/ZooKeeperClientTest.scala | 16 +---- gradle/dependencies.gradle | 2 +- gradle/spotbugs-exclude.xml | 7 +++ .../kstream/internals/KStreamImplTest.java | 2 +- .../kstream/internals/KTableImplTest.java | 6 +- .../internals/KTableMapValuesTest.java | 8 +-- ...OptionsColumnFamilyOptionsAdapterTest.java | 4 +- .../streams/scala/FunctionConversions.scala | 2 +- .../scala/FunctionsCompatConversions.scala | 2 +- .../streams/scala/ImplicitConversions.scala | 2 - .../kafka/streams/scala/StreamsBuilder.scala | 2 +- .../kafka/streams/scala/kstream/KStream.scala | 2 +- .../kafka/streams/scala/TopologyTest.scala | 2 +- .../kafka/streams/scala/WordCountTest.scala | 8 +-- ...mToTableJoinScalaIntegrationTestBase.scala | 2 +- 259 files changed, 779 insertions(+), 814 deletions(-) delete mode 100644 core/src/main/scala/kafka/cluster/Cluster.scala diff --git a/build.gradle b/build.gradle index 09002228d68..53b7a269834 100644 --- a/build.gradle +++ b/build.gradle @@ -1302,6 +1302,7 @@ project(':streams:streams-scala') { compile project(':streams') compile libs.scalaLibrary + compile libs.scalaCollectionCompat testCompile project(':core') testCompile project(':core').sourceSets.test.output diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java index 1b6ad127d6e..94213efb2b0 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java @@ -186,7 +186,6 @@ public class ListOffsetRequest extends AbstractRequest { public static final class PartitionData { public final long timestamp; - @Deprecated public final int maxNumOffsets; // only supported in v0 public final Optional currentLeaderEpoch; @@ -196,7 +195,7 @@ public class ListOffsetRequest extends AbstractRequest { this.currentLeaderEpoch = currentLeaderEpoch; } - @Deprecated + // For V0 public PartitionData(long timestamp, int maxNumOffsets) { this(timestamp, maxNumOffsets, Optional.empty()); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java index 74dc09d8e46..3588d815c89 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java @@ -137,7 +137,6 @@ public class ListOffsetResponse extends AbstractResponse { public static final class PartitionData { public final Errors error; // The offsets list is only used in ListOffsetResponse v0. - @Deprecated public final List offsets; public final Long timestamp; public final Long offset; @@ -146,7 +145,6 @@ public class ListOffsetResponse extends AbstractResponse { /** * Constructor for ListOffsetResponse v0 */ - @Deprecated public PartitionData(Errors error, List offsets) { this.error = error; this.offsets = offsets; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java index bb1b74bf411..ba6d182af69 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java @@ -39,9 +39,7 @@ public class OffsetCommitRequest extends AbstractRequest { public static final String DEFAULT_MEMBER_ID = ""; public static final long DEFAULT_RETENTION_TIME = -1L; - // default values for old versions, - // will be removed after these versions are deprecated - @Deprecated + // default values for old versions, will be removed after these versions are no longer supported public static final long DEFAULT_TIMESTAMP = -1L; // for V0, V1 private final OffsetCommitRequestData data; diff --git a/clients/src/main/java/org/apache/kafka/common/utils/SecurityUtils.java b/clients/src/main/java/org/apache/kafka/common/utils/SecurityUtils.java index 2e705f259c9..12defdc7173 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/SecurityUtils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/SecurityUtils.java @@ -82,7 +82,8 @@ public class SecurityUtils { try { String[] securityProviderClasses = securityProviderClassesStr.replaceAll("\\s+", "").split(","); for (int index = 0; index < securityProviderClasses.length; index++) { - SecurityProviderCreator securityProviderCreator = (SecurityProviderCreator) Class.forName(securityProviderClasses[index]).newInstance(); + SecurityProviderCreator securityProviderCreator = + (SecurityProviderCreator) Class.forName(securityProviderClasses[index]).getConstructor().newInstance(); securityProviderCreator.configure(configs); Security.insertProviderAt(securityProviderCreator.getProvider(), index + 1); } @@ -91,7 +92,7 @@ public class SecurityUtils { " are expected to be sub-classes of SecurityProviderCreator"); } catch (ClassNotFoundException cnfe) { LOGGER.error("Unrecognized security provider creator class", cnfe); - } catch (IllegalAccessException | InstantiationException e) { + } catch (ReflectiveOperationException e) { LOGGER.error("Unexpected implementation of security provider creator class", e); } } diff --git a/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java b/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java index e8826a588df..8100d0c28f6 100644 --- a/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java +++ b/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java @@ -648,7 +648,7 @@ public final class MessageTest { message.write(byteBufferAccessor, cache, version); assertEquals("The result of the size function does not match the number of bytes " + "written for version " + version, size, buf.position()); - Message message2 = message.getClass().newInstance(); + Message message2 = message.getClass().getConstructor().newInstance(); buf.flip(); message2.read(byteBufferAccessor, version); assertEquals("The result of the size function does not match the number of bytes " + @@ -661,7 +661,7 @@ public final class MessageTest { private void testStructRoundTrip(short version, Message message, Message expected) throws Exception { Struct struct = message.toStruct(version); - Message message2 = message.getClass().newInstance(); + Message message2 = message.getClass().getConstructor().newInstance(); message2.fromStruct(struct, version); assertEquals(expected, message2); assertEquals(expected.hashCode(), message2.hashCode()); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java index 632f5d3454e..cce1573a30d 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java @@ -331,7 +331,7 @@ public class ConnectorConfig extends AbstractConfig { } Transformation transformation; try { - transformation = transformationCls.asSubclass(Transformation.class).newInstance(); + transformation = transformationCls.asSubclass(Transformation.class).getConstructor().newInstance(); } catch (Exception e) { throw new ConfigException(key, String.valueOf(transformationCls), "Error getting config definition from Transformation: " + e.getMessage()); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java index 48a79b1ca36..149877aedf1 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java @@ -524,8 +524,8 @@ public class AbstractHerderTest { EasyMock.expect(worker.getPlugins()).andStubReturn(plugins); final Connector connector; try { - connector = connectorClass.newInstance(); - } catch (InstantiationException | IllegalAccessException e) { + connector = connectorClass.getConstructor().newInstance(); + } catch (ReflectiveOperationException e) { throw new RuntimeException("Couldn't create connector", e); } EasyMock.expect(plugins.newConnector(connectorClass.getName())).andReturn(connector); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java index 72ef29f6212..ff90d2a4687 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java @@ -479,7 +479,7 @@ public class ConnectorPluginsResourceTest { public MockConnectorPluginDesc(Class klass) throws Exception { super( klass, - klass.newInstance().version(), + klass.getConstructor().newInstance().version(), new MockPluginClassLoader(null, new URL[0]) ); } diff --git a/core/src/main/scala/kafka/Kafka.scala b/core/src/main/scala/kafka/Kafka.scala index 6842cd72bc6..7b4001c5a6a 100755 --- a/core/src/main/scala/kafka/Kafka.scala +++ b/core/src/main/scala/kafka/Kafka.scala @@ -25,7 +25,7 @@ import kafka.server.{KafkaServer, KafkaServerStartable} import kafka.utils.{CommandLineUtils, Exit, Logging} import org.apache.kafka.common.utils.{Java, LoggingSignalHandler, OperatingSystem, Utils} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ object Kafka extends Logging { diff --git a/core/src/main/scala/kafka/admin/AclCommand.scala b/core/src/main/scala/kafka/admin/AclCommand.scala index 0dca4585590..537b7e34050 100644 --- a/core/src/main/scala/kafka/admin/AclCommand.scala +++ b/core/src/main/scala/kafka/admin/AclCommand.scala @@ -34,7 +34,7 @@ import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.utils.{Utils, SecurityUtils => JSecurityUtils} import org.apache.kafka.server.authorizer.Authorizer -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.compat.java8.OptionConverters._ import scala.collection.mutable import scala.io.StdIn @@ -147,8 +147,9 @@ object AclCommand extends Logging { } else { listPrincipals.foreach(principal => { println(s"ACLs for principal `$principal`") - val filteredResourceToAcls = resourceToAcls.mapValues(acls => - acls.filter(acl => principal.toString.equals(acl.principal))).filter(entry => entry._2.nonEmpty) + val filteredResourceToAcls = resourceToAcls.map { case (resource, acls) => + resource -> acls.filter(acl => principal.toString.equals(acl.principal)) + }.filter { case (_, acls) => acls.nonEmpty } for ((resource, acls) <- filteredResourceToAcls) println(s"Current ACLs for resource `$resource`: $Newline ${acls.map("\t" + _).mkString(Newline)} $Newline") @@ -263,8 +264,9 @@ object AclCommand extends Logging { } else { listPrincipals.foreach(principal => { println(s"ACLs for principal `$principal`") - val filteredResourceToAcls = resourceToAcls.mapValues(acls => - acls.filter(acl => principal.toString.equals(acl.principal))).filter(entry => entry._2.nonEmpty) + val filteredResourceToAcls = resourceToAcls.map { case (resource, acls) => + resource -> acls.filter(acl => principal.toString.equals(acl.principal)) + }.filter { case (_, acls) => acls.nonEmpty } for ((resource, acls) <- filteredResourceToAcls) println(s"Current ACLs for resource `$resource`: $Newline ${acls.map("\t" + _).mkString(Newline)} $Newline") diff --git a/core/src/main/scala/kafka/admin/BrokerApiVersionsCommand.scala b/core/src/main/scala/kafka/admin/BrokerApiVersionsCommand.scala index 1639fc287a1..9ee13be70d8 100644 --- a/core/src/main/scala/kafka/admin/BrokerApiVersionsCommand.scala +++ b/core/src/main/scala/kafka/admin/BrokerApiVersionsCommand.scala @@ -42,7 +42,7 @@ import org.apache.kafka.common.Node import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersionsResponseKeyCollection import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, ApiVersionsRequest, ApiVersionsResponse, MetadataRequest, MetadataResponse} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.util.{Failure, Success, Try} /** diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala index de2b593883f..3be24aab34d 100644 --- a/core/src/main/scala/kafka/admin/ConfigCommand.scala +++ b/core/src/main/scala/kafka/admin/ConfigCommand.scala @@ -39,10 +39,9 @@ import org.apache.kafka.common.security.scram.internals.{ScramCredentialUtils, S import org.apache.kafka.common.utils.{Sanitizer, Time, Utils} import org.apache.zookeeper.client.ZKClientConfig -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection._ - /** * This script can be used to change configs for topics/clients/users/brokers dynamically * An entity described or altered by the command may be one of: diff --git a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala index b4adf2fc976..34d8bbb7746 100755 --- a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala +++ b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala @@ -31,9 +31,9 @@ import org.apache.kafka.clients.CommonClientConfigs import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.{KafkaException, Node, TopicPartition} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable.ListBuffer -import scala.collection.{Map, Seq, Set, immutable, mutable} +import scala.collection.{Map, Seq, immutable, mutable} import scala.util.{Failure, Success, Try} import joptsimple.OptionSpec import org.apache.kafka.common.protocol.Errors @@ -403,7 +403,7 @@ object ConsumerGroupCommand extends Logging { } def deleteOffsets(groupId: String, topics: List[String]): (Errors, Map[TopicPartition, Throwable]) = { - var partitionLevelResult: Map[TopicPartition, Throwable] = mutable.HashMap() + val partitionLevelResult = mutable.Map[TopicPartition, Throwable]() val (topicWithPartitions, topicWithoutPartitions) = topics.partition(_.contains(":")) @@ -527,16 +527,17 @@ object ConsumerGroupCommand extends Logging { partitionOffsets, Some(s"${consumerSummary.consumerId}"), Some(s"${consumerSummary.host}"), Some(s"${consumerSummary.clientId}")) } - val rowsWithoutConsumer = committedOffsets.filterKeys(!assignedTopicPartitions.contains(_)).flatMap { - case (topicPartition, offset) => - collectConsumerAssignment( - groupId, - Option(consumerGroup.coordinator), - Seq(topicPartition), - Map(topicPartition -> Some(offset.offset)), - Some(MISSING_COLUMN_VALUE), - Some(MISSING_COLUMN_VALUE), - Some(MISSING_COLUMN_VALUE)).toSeq + val rowsWithoutConsumer = committedOffsets.filter { case (tp, _) => + !assignedTopicPartitions.contains(tp) + }.flatMap { case (topicPartition, offset) => + collectConsumerAssignment( + groupId, + Option(consumerGroup.coordinator), + Seq(topicPartition), + Map(topicPartition -> Some(offset.offset)), + Some(MISSING_COLUMN_VALUE), + Some(MISSING_COLUMN_VALUE), + Some(MISSING_COLUMN_VALUE)).toSeq } groupId -> (Some(state.toString), Some(rowsWithConsumer ++ rowsWithoutConsumer)) }).toMap @@ -862,10 +863,10 @@ object ConsumerGroupCommand extends Logging { withTimeoutMs(new DeleteConsumerGroupsOptions) ).deletedGroups().asScala - val result = groupsToDelete.mapValues { f => + val result = groupsToDelete.map { case (g, f) => Try(f.get) match { - case _: Success[_] => null - case Failure(e) => e + case Success(_) => g -> null + case Failure(e) => g -> e } } @@ -1009,11 +1010,11 @@ object ConsumerGroupCommand extends Logging { options = parser.parse(args : _*) - val allGroupSelectionScopeOpts: Set[OptionSpec[_]] = Set(groupOpt, allGroupsOpt) - val allConsumerGroupLevelOpts: Set[OptionSpec[_]] = Set(listOpt, describeOpt, deleteOpt, resetOffsetsOpt) - val allResetOffsetScenarioOpts: Set[OptionSpec[_]] = Set(resetToOffsetOpt, resetShiftByOpt, + val allGroupSelectionScopeOpts = immutable.Set[OptionSpec[_]](groupOpt, allGroupsOpt) + val allConsumerGroupLevelOpts = immutable.Set[OptionSpec[_]](listOpt, describeOpt, deleteOpt, resetOffsetsOpt) + val allResetOffsetScenarioOpts = immutable.Set[OptionSpec[_]](resetToOffsetOpt, resetShiftByOpt, resetToDatetimeOpt, resetByDurationOpt, resetToEarliestOpt, resetToLatestOpt, resetToCurrentOpt, resetFromFileOpt) - val allDeleteOffsetsOpts: Set[OptionSpec[_]] = Set(groupOpt, topicOpt) + val allDeleteOffsetsOpts = immutable.Set[OptionSpec[_]](groupOpt, topicOpt) def checkArgs(): Unit = { diff --git a/core/src/main/scala/kafka/admin/DelegationTokenCommand.scala b/core/src/main/scala/kafka/admin/DelegationTokenCommand.scala index 6de0ebbfa0a..6465b143e3f 100644 --- a/core/src/main/scala/kafka/admin/DelegationTokenCommand.scala +++ b/core/src/main/scala/kafka/admin/DelegationTokenCommand.scala @@ -29,7 +29,7 @@ import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.security.token.delegation.DelegationToken import org.apache.kafka.common.utils.{SecurityUtils, Utils} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.Set /** diff --git a/core/src/main/scala/kafka/admin/DeleteRecordsCommand.scala b/core/src/main/scala/kafka/admin/DeleteRecordsCommand.scala index 424e4bca417..7469e89c4a0 100644 --- a/core/src/main/scala/kafka/admin/DeleteRecordsCommand.scala +++ b/core/src/main/scala/kafka/admin/DeleteRecordsCommand.scala @@ -28,7 +28,7 @@ import org.apache.kafka.clients.CommonClientConfigs import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.utils.Utils -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.Seq /** diff --git a/core/src/main/scala/kafka/admin/LeaderElectionCommand.scala b/core/src/main/scala/kafka/admin/LeaderElectionCommand.scala index e71702ab582..15de7f328c8 100644 --- a/core/src/main/scala/kafka/admin/LeaderElectionCommand.scala +++ b/core/src/main/scala/kafka/admin/LeaderElectionCommand.scala @@ -32,7 +32,7 @@ import org.apache.kafka.common.errors.ClusterAuthorizationException import org.apache.kafka.common.errors.ElectionNotNeededException import org.apache.kafka.common.errors.TimeoutException import org.apache.kafka.common.utils.Utils -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable import scala.concurrent.duration._ diff --git a/core/src/main/scala/kafka/admin/LogDirsCommand.scala b/core/src/main/scala/kafka/admin/LogDirsCommand.scala index 67387ce343b..6e4576274ed 100644 --- a/core/src/main/scala/kafka/admin/LogDirsCommand.scala +++ b/core/src/main/scala/kafka/admin/LogDirsCommand.scala @@ -25,7 +25,7 @@ import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, DescribeLogDirs import org.apache.kafka.common.requests.DescribeLogDirsResponse.LogDirInfo import org.apache.kafka.common.utils.Utils -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.Map /** @@ -48,7 +48,7 @@ object LogDirsCommand { out.println("Querying brokers for log directories information") val describeLogDirsResult: DescribeLogDirsResult = adminClient.describeLogDirs(brokerList.map(Integer.valueOf).toSeq.asJava) - val logDirInfosByBroker = describeLogDirsResult.all.get().asScala.mapValues(_.asScala).toMap + val logDirInfosByBroker = describeLogDirsResult.all.get().asScala.map { case (k, v) => k -> v.asScala } out.println(s"Received log directory information from brokers ${brokerList.mkString(",")}") out.println(formatAsJson(logDirInfosByBroker, topicList.toSet)) diff --git a/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala b/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala index 757ef1177f6..eabc58c5417 100755 --- a/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala +++ b/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala @@ -16,7 +16,7 @@ */ package kafka.admin -import collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import collection._ import java.util.Properties import java.util.concurrent.ExecutionException diff --git a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala index 28b2d011cb7..96718eb819d 100755 --- a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala +++ b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala @@ -34,7 +34,7 @@ import org.apache.kafka.common.security.JaasUtils import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{KafkaException, KafkaFuture, TopicPartition, TopicPartitionReplica} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.{Map, Seq, mutable} import scala.compat.java8.OptionConverters._ import scala.math.Ordered.orderingToOrdered @@ -1062,17 +1062,16 @@ object ReassignPartitionsCommand extends Logging { def curReassignmentsToString(adminClient: Admin): String = { val currentReassignments = adminClient. listPartitionReassignments().reassignments().get().asScala - val text = currentReassignments.keySet.toBuffer.sortWith(compareTopicPartitions).map { - case part => - val reassignment = currentReassignments(part) - val replicas = reassignment.replicas().asScala - val addingReplicas = reassignment.addingReplicas().asScala - val removingReplicas = reassignment.removingReplicas().asScala - "%s: replicas: %s.%s%s".format(part, replicas.mkString(","), - if (addingReplicas.isEmpty) "" else - " adding: %s.".format(addingReplicas.mkString(",")), - if (removingReplicas.isEmpty) "" else - " removing: %s.".format(removingReplicas.mkString(","))) + val text = currentReassignments.keySet.toBuffer.sortWith(compareTopicPartitions).map { part => + val reassignment = currentReassignments(part) + val replicas = reassignment.replicas().asScala + val addingReplicas = reassignment.addingReplicas().asScala + val removingReplicas = reassignment.removingReplicas().asScala + "%s: replicas: %s.%s%s".format(part, replicas.mkString(","), + if (addingReplicas.isEmpty) "" else + " adding: %s.".format(addingReplicas.mkString(",")), + if (removingReplicas.isEmpty) "" else + " removing: %s.".format(removingReplicas.mkString(","))) }.mkString(System.lineSeparator()) if (text.isEmpty) { "No partition reassignments found." @@ -1155,7 +1154,7 @@ object ReassignPartitionsCommand extends Logging { def currentPartitionReplicaAssignmentToString(proposedParts: Map[TopicPartition, Seq[Int]], currentParts: Map[TopicPartition, Seq[Int]]): String = { "Current partition replica assignment%n%n%s%n%nSave this to use as the %s". - format(formatAsReassignmentJson(currentParts.filterKeys(proposedParts.contains(_)).toMap, Map.empty), + format(formatAsReassignmentJson(currentParts.filter { case (k, _) => proposedParts.contains(k) }.toMap, Map.empty), "--reassignment-json-file option during rollback") } @@ -1192,13 +1191,10 @@ object ReassignPartitionsCommand extends Logging { * @return A map from partition objects to error strings. */ def alterPartitionReassignments(adminClient: Admin, - reassignments: Map[TopicPartition, Seq[Int]]) - : Map[TopicPartition, Throwable] = { + reassignments: Map[TopicPartition, Seq[Int]]): Map[TopicPartition, Throwable] = { val results: Map[TopicPartition, KafkaFuture[Void]] = - adminClient.alterPartitionReassignments(reassignments.map { - case (part, replicas) => { - (part, Optional.of(new NewPartitionReassignment(replicas.map(Integer.valueOf(_)).asJava))) - } + adminClient.alterPartitionReassignments(reassignments.map { case (part, replicas) => + (part, Optional.of(new NewPartitionReassignment(replicas.map(Integer.valueOf(_)).asJava))) }.asJava).values().asScala results.flatMap { case (part, future) => { diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index 304107b4826..9c2c22369e4 100755 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -38,7 +38,7 @@ import org.apache.kafka.common.security.JaasUtils import org.apache.kafka.common.utils.{Time, Utils} import org.apache.zookeeper.KeeperException.NodeExistsException -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection._ import scala.compat.java8.OptionConverters._ import scala.concurrent.ExecutionException @@ -653,7 +653,7 @@ object TopicCommand extends Logging { options = parser.parse(args : _*) - private val allTopicLevelOpts: Set[OptionSpec[_]] = Set(alterOpt, createOpt, describeOpt, listOpt, deleteOpt) + private val allTopicLevelOpts = immutable.Set[OptionSpec[_]](alterOpt, createOpt, describeOpt, listOpt, deleteOpt) private val allReplicationReportOpts: Set[OptionSpec[_]] = Set(reportUnderReplicatedPartitionsOpt, reportUnderMinIsrPartitionsOpt, reportAtMinIsrPartitionsOpt, reportUnavailablePartitionsOpt) diff --git a/core/src/main/scala/kafka/admin/ZkSecurityMigrator.scala b/core/src/main/scala/kafka/admin/ZkSecurityMigrator.scala index e466d06f9b3..6ae3ab16b21 100644 --- a/core/src/main/scala/kafka/admin/ZkSecurityMigrator.scala +++ b/core/src/main/scala/kafka/admin/ZkSecurityMigrator.scala @@ -30,7 +30,7 @@ import org.apache.zookeeper.client.ZKClientConfig import org.apache.zookeeper.data.Stat import scala.annotation.tailrec -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable.Queue import scala.concurrent._ import scala.concurrent.duration._ diff --git a/core/src/main/scala/kafka/api/package.scala b/core/src/main/scala/kafka/api/package.scala index 98d00454229..11a956d40b5 100644 --- a/core/src/main/scala/kafka/api/package.scala +++ b/core/src/main/scala/kafka/api/package.scala @@ -19,7 +19,7 @@ package kafka import org.apache.kafka.common.ElectionType import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.requests.ElectLeadersRequest -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ package object api { implicit final class ElectLeadersRequestOps(val self: ElectLeadersRequest) extends AnyVal { diff --git a/core/src/main/scala/kafka/cluster/Broker.scala b/core/src/main/scala/kafka/cluster/Broker.scala index e8e621f0355..cd1aba36be9 100755 --- a/core/src/main/scala/kafka/cluster/Broker.scala +++ b/core/src/main/scala/kafka/cluster/Broker.scala @@ -27,7 +27,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.server.authorizer.AuthorizerServerInfo import scala.collection.Seq -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ object Broker { private[cluster] case class ServerInfo(clusterResource: ClusterResource, diff --git a/core/src/main/scala/kafka/cluster/Cluster.scala b/core/src/main/scala/kafka/cluster/Cluster.scala deleted file mode 100644 index 75bbec054ae..00000000000 --- a/core/src/main/scala/kafka/cluster/Cluster.scala +++ /dev/null @@ -1,45 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.cluster - -import scala.collection._ - -/** - * The set of active brokers in the cluster - */ -private[kafka] class Cluster { - - private val brokers = new mutable.HashMap[Int, Broker] - - def this(brokerList: Iterable[Broker]) { - this() - for(broker <- brokerList) - brokers.put(broker.id, broker) - } - - def getBroker(id: Int): Option[Broker] = brokers.get(id) - - def add(broker: Broker) = brokers.put(broker.id, broker) - - def remove(id: Int) = brokers.remove(id) - - def size = brokers.size - - override def toString: String = - "Cluster(" + brokers.values.mkString(", ") + ")" -} diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 3305b319a9a..9bacf3c7b84 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -40,7 +40,7 @@ import org.apache.kafka.common.requests.EpochEndOffset._ import org.apache.kafka.common.requests._ import org.apache.kafka.common.utils.Time -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.{Map, Seq} trait PartitionStateStore { @@ -662,13 +662,11 @@ class Partition(val topicPartition: TopicPartition, isr: Set[Int], addingReplicas: Seq[Int], removingReplicas: Seq[Int]): Unit = { - val replicaSet = assignment.toSet - val removedReplicas = remoteReplicasMap.keys -- replicaSet - + remoteReplicasMap.clear() assignment .filter(_ != localBrokerId) .foreach(id => remoteReplicasMap.getAndMaybePut(id, new Replica(id, topicPartition))) - removedReplicas.foreach(remoteReplicasMap.remove) + if (addingReplicas.nonEmpty || removingReplicas.nonEmpty) assignmentState = OngoingReassignmentState(addingReplicas, removingReplicas, assignment) else diff --git a/core/src/main/scala/kafka/common/InterBrokerSendThread.scala b/core/src/main/scala/kafka/common/InterBrokerSendThread.scala index 1551704203a..3eff03db22a 100644 --- a/core/src/main/scala/kafka/common/InterBrokerSendThread.scala +++ b/core/src/main/scala/kafka/common/InterBrokerSendThread.scala @@ -27,7 +27,7 @@ import org.apache.kafka.common.internals.FatalExitError import org.apache.kafka.common.requests.AbstractRequest import org.apache.kafka.common.utils.Time -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** * Class for inter-broker send thread that utilize a non-blocking network client. diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index 4d96b74716e..c4a4d5fea03 100755 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -37,7 +37,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.{LogContext, Time} import org.apache.kafka.common.{KafkaException, Node, Reconfigurable, TopicPartition} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable.{HashMap, ListBuffer} import scala.collection.{Seq, Set, mutable} @@ -445,8 +445,8 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, else if (config.interBrokerProtocolVersion >= KAFKA_1_0_IV0) 1 else 0 - leaderAndIsrRequestMap.filterKeys(controllerContext.liveOrShuttingDownBrokerIds.contains).foreach { - case (broker, leaderAndIsrPartitionStates) => + leaderAndIsrRequestMap.foreach { case (broker, leaderAndIsrPartitionStates) => + if (controllerContext.liveOrShuttingDownBrokerIds.contains(broker)) { val numBecomeLeaders = leaderAndIsrPartitionStates.count { case (topicPartition, state) => val isBecomeLeader = broker == state.leader val typeOfRequest = @@ -470,8 +470,7 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, val leaderAndIsrResponse = r.asInstanceOf[LeaderAndIsrResponse] sendEvent(LeaderAndIsrResponseReceived(leaderAndIsrResponse, broker)) }) - - + } } leaderAndIsrRequestMap.clear() } @@ -553,25 +552,27 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, } val traceEnabled = stateChangeLog.isTraceEnabled - stopReplicaRequestMap.filterKeys(controllerContext.liveOrShuttingDownBrokerIds.contains).foreach { case (brokerId, replicaInfoList) => - val (stopReplicaWithDelete, stopReplicaWithoutDelete) = replicaInfoList.partition(r => r.deletePartition) - val brokerEpoch = controllerContext.liveBrokerIdAndEpochs(brokerId) + stopReplicaRequestMap.foreach { case (brokerId, replicaInfoList) => + if (controllerContext.liveOrShuttingDownBrokerIds.contains(brokerId)) { + val (stopReplicaWithDelete, stopReplicaWithoutDelete) = replicaInfoList.partition(r => r.deletePartition) + val brokerEpoch = controllerContext.liveBrokerIdAndEpochs(brokerId) - if (stopReplicaWithDelete.nonEmpty) { - stateChangeLog.info(s"Sending a stop replica request (delete = true) for ${stopReplicaWithDelete.size} replicas to broker $brokerId") - if (traceEnabled) - stateChangeLog.trace(s"The stop replica request (delete = true) sent to broker $brokerId contains ${stopReplicaWithDelete.map(_.replica).mkString(",")}") - val stopReplicaRequest = createStopReplicaRequest(brokerEpoch, stopReplicaWithDelete, deletePartitions = true) - val callback = stopReplicaPartitionDeleteResponseCallback(brokerId) _ - sendRequest(brokerId, stopReplicaRequest, callback) - } + if (stopReplicaWithDelete.nonEmpty) { + stateChangeLog.info(s"Sending a stop replica request (delete = true) for ${stopReplicaWithDelete.size} replicas to broker $brokerId") + if (traceEnabled) + stateChangeLog.trace(s"The stop replica request (delete = true) sent to broker $brokerId contains ${stopReplicaWithDelete.map(_.replica).mkString(",")}") + val stopReplicaRequest = createStopReplicaRequest(brokerEpoch, stopReplicaWithDelete, deletePartitions = true) + val callback = stopReplicaPartitionDeleteResponseCallback(brokerId) _ + sendRequest(brokerId, stopReplicaRequest, callback) + } - if (stopReplicaWithoutDelete.nonEmpty) { - stateChangeLog.info(s"Sending a stop replica request (delete = false) for ${stopReplicaWithoutDelete.size} replicas to broker $brokerId") - if (traceEnabled) - stateChangeLog.trace(s"The stop replica request (delete = false) sent to broker $brokerId contains ${stopReplicaWithoutDelete.map(_.replica).mkString(",")}") - val stopReplicaRequest = createStopReplicaRequest(brokerEpoch, stopReplicaWithoutDelete, deletePartitions = false) - sendRequest(brokerId, stopReplicaRequest) + if (stopReplicaWithoutDelete.nonEmpty) { + stateChangeLog.info(s"Sending a stop replica request (delete = false) for ${stopReplicaWithoutDelete.size} replicas to broker $brokerId") + if (traceEnabled) + stateChangeLog.trace(s"The stop replica request (delete = false) sent to broker $brokerId contains ${stopReplicaWithoutDelete.map(_.replica).mkString(",")}") + val stopReplicaRequest = createStopReplicaRequest(brokerEpoch, stopReplicaWithoutDelete, deletePartitions = false) + sendRequest(brokerId, stopReplicaRequest) + } } } stopReplicaRequestMap.clear() diff --git a/core/src/main/scala/kafka/controller/ControllerContext.scala b/core/src/main/scala/kafka/controller/ControllerContext.scala index f7a6cdd19a1..51986a4abc6 100644 --- a/core/src/main/scala/kafka/controller/ControllerContext.scala +++ b/core/src/main/scala/kafka/controller/ControllerContext.scala @@ -74,19 +74,19 @@ case class ReplicaAssignment private (replicas: Seq[Int], class ControllerContext { val stats = new ControllerStats var offlinePartitionCount = 0 - var shuttingDownBrokerIds: mutable.Set[Int] = mutable.Set.empty - private var liveBrokers: Set[Broker] = Set.empty - private var liveBrokerEpochs: Map[Int, Long] = Map.empty + val shuttingDownBrokerIds = mutable.Set.empty[Int] + private val liveBrokers = mutable.Set.empty[Broker] + private val liveBrokerEpochs = mutable.Map.empty[Int, Long] var epoch: Int = KafkaController.InitialControllerEpoch var epochZkVersion: Int = KafkaController.InitialControllerEpochZkVersion - var allTopics: Set[String] = Set.empty + val allTopics = mutable.Set.empty[String] val partitionAssignments = mutable.Map.empty[String, mutable.Map[Int, ReplicaAssignment]] val partitionLeadershipInfo = mutable.Map.empty[TopicPartition, LeaderIsrAndControllerEpoch] val partitionsBeingReassigned = mutable.Set.empty[TopicPartition] val partitionStates = mutable.Map.empty[TopicPartition, PartitionState] val replicaStates = mutable.Map.empty[PartitionAndReplica, ReplicaState] - val replicasOnOfflineDirs: mutable.Map[Int, Set[TopicPartition]] = mutable.Map.empty + val replicasOnOfflineDirs = mutable.Map.empty[Int, Set[TopicPartition]] val topicsToBeDeleted = mutable.Set.empty[String] @@ -113,7 +113,7 @@ class ControllerContext { val topicsIneligibleForDeletion = mutable.Set.empty[String] private def clearTopicsState(): Unit = { - allTopics = Set.empty + allTopics.clear() partitionAssignments.clear() partitionLeadershipInfo.clear() partitionsBeingReassigned.clear() @@ -124,11 +124,10 @@ class ControllerContext { } def partitionReplicaAssignment(topicPartition: TopicPartition): Seq[Int] = { - partitionAssignments.getOrElse(topicPartition.topic, mutable.Map.empty) - .get(topicPartition.partition) match { - case Some(partitionAssignment) => partitionAssignment.replicas - case None => Seq.empty - } + partitionAssignments.getOrElse(topicPartition.topic, mutable.Map.empty).get(topicPartition.partition) match { + case Some(partitionAssignment) => partitionAssignment.replicas + case None => Seq.empty + } } def partitionFullReplicaAssignment(topicPartition: TopicPartition): ReplicaAssignment = { @@ -161,21 +160,24 @@ class ControllerContext { }.toSet } - def setLiveBrokerAndEpochs(brokerAndEpochs: Map[Broker, Long]): Unit = { - liveBrokers = brokerAndEpochs.keySet - liveBrokerEpochs = - brokerAndEpochs map { case (broker, brokerEpoch) => (broker.id, brokerEpoch)} + def setLiveBrokers(brokerAndEpochs: Map[Broker, Long]): Unit = { + clearLiveBrokers() + addLiveBrokers(brokerAndEpochs) } - def addLiveBrokersAndEpochs(brokerAndEpochs: Map[Broker, Long]): Unit = { - liveBrokers = liveBrokers ++ brokerAndEpochs.keySet - liveBrokerEpochs = liveBrokerEpochs ++ - (brokerAndEpochs map { case (broker, brokerEpoch) => (broker.id, brokerEpoch)}) + private def clearLiveBrokers(): Unit = { + liveBrokers.clear() + liveBrokerEpochs.clear() + } + + def addLiveBrokers(brokerAndEpochs: Map[Broker, Long]): Unit = { + liveBrokers ++= brokerAndEpochs.keySet + liveBrokerEpochs ++= brokerAndEpochs.map { case (broker, brokerEpoch) => (broker.id, brokerEpoch) } } def removeLiveBrokers(brokerIds: Set[Int]): Unit = { - liveBrokers = liveBrokers.filter(broker => !brokerIds.contains(broker.id)) - liveBrokerEpochs = liveBrokerEpochs.filter { case (id, _) => !brokerIds.contains(id) } + liveBrokers --= liveBrokers.filter(broker => brokerIds.contains(broker.id)) + liveBrokerEpochs --= brokerIds } def updateBrokerMetadata(oldMetadata: Broker, newMetadata: Broker): Unit = { @@ -184,7 +186,7 @@ class ControllerContext { } // getter - def liveBrokerIds: Set[Int] = liveBrokerEpochs.keySet -- shuttingDownBrokerIds + def liveBrokerIds: Set[Int] = liveBrokerEpochs.keySet.diff(shuttingDownBrokerIds) def liveOrShuttingDownBrokerIds: Set[Int] = liveBrokerEpochs.keySet def liveOrShuttingDownBrokers: Set[Broker] = liveBrokers def liveBrokerIdAndEpochs: Map[Int, Long] = liveBrokerEpochs @@ -270,15 +272,20 @@ class ControllerContext { epoch = 0 epochZkVersion = 0 clearTopicsState() - setLiveBrokerAndEpochs(Map.empty) + clearLiveBrokers() + } + + def setAllTopics(topics: Set[String]): Unit = { + allTopics.clear() + allTopics ++= topics } def removeTopic(topic: String): Unit = { allTopics -= topic partitionAssignments.remove(topic) - partitionLeadershipInfo.foreach { - case (topicPartition, _) if topicPartition.topic == topic => partitionLeadershipInfo.remove(topicPartition) - case _ => + partitionLeadershipInfo.foreach { case (topicPartition, _) => + if (topicPartition.topic == topic) + partitionLeadershipInfo.remove(topicPartition) } } diff --git a/core/src/main/scala/kafka/controller/ControllerEventManager.scala b/core/src/main/scala/kafka/controller/ControllerEventManager.scala index b04b85f1fa1..7a0d70f754d 100644 --- a/core/src/main/scala/kafka/controller/ControllerEventManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerEventManager.scala @@ -27,7 +27,7 @@ import kafka.utils.ShutdownableThread import org.apache.kafka.common.utils.Time import scala.collection._ -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ object ControllerEventManager { val ControllerEventThreadName = "controller-event-thread" diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index f6375474eed..9ad7b6ff7b6 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -39,7 +39,7 @@ import org.apache.kafka.common.utils.Time import org.apache.zookeeper.KeeperException import org.apache.zookeeper.KeeperException.Code -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.{Map, Seq, Set, immutable, mutable} import scala.collection.mutable.ArrayBuffer import scala.util.{Failure, Try} @@ -348,7 +348,7 @@ class KafkaController(val config: KafkaConfig, info(s"New broker startup callback for ${newBrokers.mkString(",")}") newBrokers.foreach(controllerContext.replicasOnOfflineDirs.remove) val newBrokersSet = newBrokers.toSet - val existingBrokers = controllerContext.liveOrShuttingDownBrokerIds -- newBrokers + val existingBrokers = controllerContext.liveOrShuttingDownBrokerIds.diff(newBrokersSet) // Send update metadata request to all the existing brokers in the cluster so that they know about the new brokers // via this update. No need to include any partition states in the request since there are no partition state changes. sendUpdateMetadataRequest(existingBrokers.toSeq, Set.empty) @@ -725,9 +725,9 @@ class KafkaController(val config: KafkaConfig, private def initializeControllerContext(): Unit = { // update controller cache with delete topic information val curBrokerAndEpochs = zkClient.getAllBrokerAndEpochsInCluster - controllerContext.setLiveBrokerAndEpochs(curBrokerAndEpochs) + controllerContext.setLiveBrokers(curBrokerAndEpochs) info(s"Initialized broker epochs cache: ${controllerContext.liveBrokerIdAndEpochs}") - controllerContext.allTopics = zkClient.getAllTopicsInCluster(true) + controllerContext.setAllTopics(zkClient.getAllTopicsInCluster(true)) registerPartitionModificationsHandlers(controllerContext.allTopics.toSeq) zkClient.getFullReplicaAssignmentForTopics(controllerContext.allTopics.toSet).foreach { case (topicPartition, replicaAssignment) => @@ -736,7 +736,7 @@ class KafkaController(val config: KafkaConfig, controllerContext.partitionsBeingReassigned.add(topicPartition) } controllerContext.partitionLeadershipInfo.clear() - controllerContext.shuttingDownBrokerIds = mutable.Set.empty[Int] + controllerContext.shuttingDownBrokerIds.clear() // register broker modifications handlers registerBrokerModificationsHandler(controllerContext.liveOrShuttingDownBrokerIds) // update the leader and isr cache for all existing partitions from Zookeeper @@ -848,8 +848,9 @@ class KafkaController(val config: KafkaConfig, } private def updateReplicaAssignmentForPartition(topicPartition: TopicPartition, assignment: ReplicaAssignment): Unit = { - var topicAssignment = controllerContext.partitionFullReplicaAssignmentForTopic(topicPartition.topic) - topicAssignment += topicPartition -> assignment + val topicAssignment = mutable.Map() ++= + controllerContext.partitionFullReplicaAssignmentForTopic(topicPartition.topic) += + (topicPartition -> assignment) val setDataResponse = zkClient.setTopicAssignmentRaw(topicPartition.topic, topicAssignment, controllerContext.epochZkVersion) setDataResponse.resultCode match { @@ -1194,9 +1195,9 @@ class KafkaController(val config: KafkaConfig, } val previousOfflineReplicas = controllerContext.replicasOnOfflineDirs.getOrElse(brokerId, Set.empty[TopicPartition]) - val currentOfflineReplicas = previousOfflineReplicas -- onlineReplicas ++ offlineReplicas + val currentOfflineReplicas = mutable.Set() ++= previousOfflineReplicas --= onlineReplicas ++= offlineReplicas controllerContext.replicasOnOfflineDirs.put(brokerId, currentOfflineReplicas) - val newOfflineReplicas = currentOfflineReplicas -- previousOfflineReplicas + val newOfflineReplicas = currentOfflineReplicas.diff(previousOfflineReplicas) if (newOfflineReplicas.nonEmpty) { stateChangeLogger.info(s"Mark replicas ${newOfflineReplicas.mkString(",")} on broker $brokerId as offline") @@ -1221,7 +1222,7 @@ class KafkaController(val config: KafkaConfig, topicDeletionManager.failReplicaDeletion(replicasInError) if (replicasInError.size != partitionErrors.size) { // some replicas could have been successfully deleted - val deletedReplicas = partitionErrors.keySet -- partitionsInError + val deletedReplicas = partitionErrors.keySet.diff(partitionsInError) topicDeletionManager.completeReplicaDeletion(deletedReplicas.map(PartitionAndReplica(_, replicaId))) } } @@ -1364,8 +1365,8 @@ class KafkaController(val config: KafkaConfig, val curBrokerIdAndEpochs = curBrokerAndEpochs map { case (broker, epoch) => (broker.id, epoch) } val curBrokerIds = curBrokerIdAndEpochs.keySet val liveOrShuttingDownBrokerIds = controllerContext.liveOrShuttingDownBrokerIds - val newBrokerIds = curBrokerIds -- liveOrShuttingDownBrokerIds - val deadBrokerIds = liveOrShuttingDownBrokerIds -- curBrokerIds + val newBrokerIds = curBrokerIds.diff(liveOrShuttingDownBrokerIds) + val deadBrokerIds = liveOrShuttingDownBrokerIds.diff(curBrokerIds) val bouncedBrokerIds = (curBrokerIds & liveOrShuttingDownBrokerIds) .filter(brokerId => curBrokerIdAndEpochs(brokerId) > controllerContext.liveBrokerIdAndEpochs(brokerId)) val newBrokerAndEpochs = curBrokerAndEpochs.filter { case (broker, _) => newBrokerIds.contains(broker.id) } @@ -1384,13 +1385,13 @@ class KafkaController(val config: KafkaConfig, bouncedBrokerAndEpochs.keySet.foreach(controllerChannelManager.addBroker) deadBrokerIds.foreach(controllerChannelManager.removeBroker) if (newBrokerIds.nonEmpty) { - controllerContext.addLiveBrokersAndEpochs(newBrokerAndEpochs) + controllerContext.addLiveBrokers(newBrokerAndEpochs) onBrokerStartup(newBrokerIdsSorted) } if (bouncedBrokerIds.nonEmpty) { controllerContext.removeLiveBrokers(bouncedBrokerIds) onBrokerFailure(bouncedBrokerIdsSorted) - controllerContext.addLiveBrokersAndEpochs(bouncedBrokerAndEpochs) + controllerContext.addLiveBrokers(bouncedBrokerAndEpochs) onBrokerStartup(bouncedBrokerIdsSorted) } if (deadBrokerIds.nonEmpty) { @@ -1422,8 +1423,8 @@ class KafkaController(val config: KafkaConfig, if (!isActive) return val topics = zkClient.getAllTopicsInCluster(true) val newTopics = topics -- controllerContext.allTopics - val deletedTopics = controllerContext.allTopics -- topics - controllerContext.allTopics = topics + val deletedTopics = controllerContext.allTopics.diff(topics) + controllerContext.setAllTopics(topics) registerPartitionModificationsHandlers(newTopics.toSeq) val addedPartitionReplicaAssignment = zkClient.getFullReplicaAssignmentForTopics(newTopics) diff --git a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala index 940bf55c3bc..eba2478e658 100755 --- a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala +++ b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala @@ -434,7 +434,7 @@ class ZkPartitionStateMachine(config: KafkaConfig, val UpdateLeaderAndIsrResult(finishedUpdates, updatesToRetry) = zkClient.updateLeaderAndIsr( adjustedLeaderAndIsrs, controllerContext.epoch, controllerContext.epochZkVersion) finishedUpdates.foreach { case (partition, result) => - result.right.foreach { leaderAndIsr => + result.foreach { leaderAndIsr => val replicaAssignment = controllerContext.partitionFullReplicaAssignment(partition) val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerContext.epoch) controllerContext.partitionLeadershipInfo.put(partition, leaderIsrAndControllerEpoch) diff --git a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala index bb56e6c3a0b..a0df78ba371 100644 --- a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala +++ b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala @@ -332,14 +332,14 @@ class ZkReplicaStateMachine(config: KafkaConfig, ): (Map[TopicPartition, Either[Exception, LeaderIsrAndControllerEpoch]], Seq[TopicPartition]) = { val (leaderAndIsrs, partitionsWithNoLeaderAndIsrInZk) = getTopicPartitionStatesFromZk(partitions) val (leaderAndIsrsWithReplica, leaderAndIsrsWithoutReplica) = leaderAndIsrs.partition { case (_, result) => - result.right.map { leaderAndIsr => + result.map { leaderAndIsr => leaderAndIsr.isr.contains(replicaId) - }.right.getOrElse(false) + }.getOrElse(false) } val adjustedLeaderAndIsrs: Map[TopicPartition, LeaderAndIsr] = leaderAndIsrsWithReplica.flatMap { case (partition, result) => - result.right.toOption.map { leaderAndIsr => + result.toOption.map { leaderAndIsr => val newLeader = if (replicaId == leaderAndIsr.leader) LeaderAndIsr.NoLeader else leaderAndIsr.leader val adjustedIsr = if (leaderAndIsr.isr.size == 1) leaderAndIsr.isr else leaderAndIsr.isr.filter(_ != replicaId) partition -> leaderAndIsr.newLeaderAndIsr(newLeader, adjustedIsr) @@ -347,10 +347,7 @@ class ZkReplicaStateMachine(config: KafkaConfig, } val UpdateLeaderAndIsrResult(finishedPartitions, updatesToRetry) = zkClient.updateLeaderAndIsr( - adjustedLeaderAndIsrs, - controllerContext.epoch, - controllerContext.epochZkVersion - ) + adjustedLeaderAndIsrs, controllerContext.epoch, controllerContext.epochZkVersion) val exceptionsForPartitionsWithNoLeaderAndIsrInZk: Map[TopicPartition, Either[Exception, LeaderIsrAndControllerEpoch]] = partitionsWithNoLeaderAndIsrInZk.iterator.flatMap { partition => @@ -364,21 +361,15 @@ class ZkReplicaStateMachine(config: KafkaConfig, }.toMap val leaderIsrAndControllerEpochs: Map[TopicPartition, Either[Exception, LeaderIsrAndControllerEpoch]] = - (leaderAndIsrsWithoutReplica ++ finishedPartitions).map { case (partition, result: Either[Exception, LeaderAndIsr]) => - ( - partition, - result.right.map { leaderAndIsr => - val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerContext.epoch) - controllerContext.partitionLeadershipInfo.put(partition, leaderIsrAndControllerEpoch) - leaderIsrAndControllerEpoch - } - ) + (leaderAndIsrsWithoutReplica ++ finishedPartitions).map { case (partition, result) => + (partition, result.map { leaderAndIsr => + val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerContext.epoch) + controllerContext.partitionLeadershipInfo.put(partition, leaderIsrAndControllerEpoch) + leaderIsrAndControllerEpoch + }) } - ( - leaderIsrAndControllerEpochs ++ exceptionsForPartitionsWithNoLeaderAndIsrInZk, - updatesToRetry - ) + (leaderIsrAndControllerEpochs ++ exceptionsForPartitionsWithNoLeaderAndIsrInZk, updatesToRetry) } /** diff --git a/core/src/main/scala/kafka/controller/TopicDeletionManager.scala b/core/src/main/scala/kafka/controller/TopicDeletionManager.scala index 64f9ff052d1..a9639b46a34 100755 --- a/core/src/main/scala/kafka/controller/TopicDeletionManager.scala +++ b/core/src/main/scala/kafka/controller/TopicDeletionManager.scala @@ -257,7 +257,7 @@ class TopicDeletionManager(config: KafkaConfig, * removed from their caches. */ private def onTopicDeletion(topics: Set[String]): Unit = { - val unseenTopicsForDeletion = topics -- controllerContext.topicsWithDeletionStarted + val unseenTopicsForDeletion = topics.diff(controllerContext.topicsWithDeletionStarted) if (unseenTopicsForDeletion.nonEmpty) { val unseenPartitionsForDeletion = unseenTopicsForDeletion.flatMap(controllerContext.partitionsForTopic) partitionStateMachine.handleStateChanges(unseenPartitionsForDeletion.toSeq, OfflinePartition) @@ -295,7 +295,7 @@ class TopicDeletionManager(config: KafkaConfig, } val successfullyDeletedReplicas = controllerContext.replicasInState(topic, ReplicaDeletionSuccessful) - val replicasForDeletionRetry = aliveReplicas -- successfullyDeletedReplicas + val replicasForDeletionRetry = aliveReplicas.diff(successfullyDeletedReplicas) allDeadReplicas ++= deadReplicas allReplicasForDeletionRetry ++= replicasForDeletionRetry diff --git a/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala index bf8f99118d1..e4525de3a9d 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala @@ -35,7 +35,7 @@ import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests._ import org.apache.kafka.common.utils.Time -import scala.collection.{Map, Seq, immutable} +import scala.collection.{Map, Seq, immutable, mutable} import scala.math.max /** @@ -423,7 +423,7 @@ class GroupCoordinator(val brokerId: Int, info(s"Assignment received from leader for group ${group.groupId} for generation ${group.generationId}") // fill any missing members with an empty assignment - val missing = group.allMembers -- groupAssignment.keySet + val missing = group.allMembers.diff(groupAssignment.keySet) val assignment = groupAssignment ++ missing.map(_ -> Array.empty[Byte]).toMap if (missing.nonEmpty) { @@ -519,8 +519,8 @@ class GroupCoordinator(val brokerId: Int, } def handleDeleteGroups(groupIds: Set[String]): Map[String, Errors] = { - var groupErrors: Map[String, Errors] = Map() - var groupsEligibleForDeletion: Seq[GroupMetadata] = Seq() + val groupErrors = mutable.Map.empty[String, Errors] + val groupsEligibleForDeletion = mutable.ArrayBuffer[GroupMetadata]() groupIds.foreach { groupId => validateGroupStatus(groupId, ApiKeys.DELETE_GROUPS) match { @@ -540,9 +540,9 @@ class GroupCoordinator(val brokerId: Int, (if (groupManager.groupNotExists(groupId)) Errors.GROUP_ID_NOT_FOUND else Errors.NOT_COORDINATOR) case Empty => group.transitionTo(Dead) - groupsEligibleForDeletion :+= group + groupsEligibleForDeletion += group case Stable | PreparingRebalance | CompletingRebalance => - groupErrors += groupId -> Errors.NON_EMPTY_GROUP + groupErrors(groupId) = Errors.NON_EMPTY_GROUP } } } diff --git a/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala b/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala index dde76c36071..44b33d3029f 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala @@ -30,7 +30,7 @@ import org.apache.kafka.common.protocol.types.SchemaException import org.apache.kafka.common.utils.Time import scala.collection.{Seq, immutable, mutable} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ private[group] sealed trait GroupState { val validPreviousStates: Set[GroupState] @@ -411,22 +411,21 @@ private[group] class GroupMetadata(val groupId: String, initialState: GroupState val candidates = candidateProtocols // let each member vote for one of the protocols and choose the one with the most votes - val votes: List[(String, Int)] = allMemberMetadata + val (protocol, _) = allMemberMetadata .map(_.vote(candidates)) .groupBy(identity) - .mapValues(_.size) - .toList + .maxBy { case (_, votes) => votes.size } - votes.maxBy(_._2)._1 + protocol } - private def candidateProtocols = { + private def candidateProtocols: Set[String] = { // get the set of protocols that are commonly supported by all members val numMembers = members.size supportedProtocols.filter(_._2 == numMembers).map(_._1).toSet } - def supportsProtocols(memberProtocolType: String, memberProtocols: Set[String]) = { + def supportsProtocols(memberProtocolType: String, memberProtocols: Set[String]): Boolean = { if (is(Empty)) !memberProtocolType.isEmpty && memberProtocols.nonEmpty else @@ -467,11 +466,10 @@ private[group] class GroupMetadata(val groupId: String, initialState: GroupState }.reduceLeft(_ ++ _) ) } catch { - case e: SchemaException => { + case e: SchemaException => warn(s"Failed to parse Consumer Protocol ${ConsumerProtocol.PROTOCOL_TYPE}:${protocolName.get} " + s"of group $groupId. Consumer group coordinator is not aware of the subscribed topics.", e) None - } } case Some(ConsumerProtocol.PROTOCOL_TYPE) if members.isEmpty => @@ -483,7 +481,7 @@ private[group] class GroupMetadata(val groupId: String, initialState: GroupState def updateMember(member: MemberMetadata, protocols: List[(String, Array[Byte])], - callback: JoinCallback) = { + callback: JoinCallback): Unit = { member.supportedProtocols.foreach{ case (protocol, _) => supportedProtocols(protocol) -= 1 } protocols.foreach{ case (protocol, _) => supportedProtocols(protocol) += 1 } member.supportedProtocols = protocols @@ -519,7 +517,7 @@ private[group] class GroupMetadata(val groupId: String, initialState: GroupState } } - def initNextGeneration() = { + def initNextGeneration(): Unit = { if (members.nonEmpty) { generationId += 1 protocolName = Some(selectProtocol) diff --git a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala index dd083ab9f5a..9f14e13435b 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala @@ -49,7 +49,7 @@ import org.apache.kafka.common.requests.{OffsetCommitRequest, OffsetFetchRespons import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{KafkaException, TopicPartition} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection._ import scala.collection.mutable.ArrayBuffer @@ -534,7 +534,7 @@ class GroupMetadataManager(brokerId: Int, doLoadGroupsAndOffsets(topicPartition, onGroupLoaded) val endTimeMs = time.milliseconds() val totalLoadingTimeMs = endTimeMs - startTimeMs - partitionLoadSensor.record(totalLoadingTimeMs, endTimeMs, false) + partitionLoadSensor.record(totalLoadingTimeMs.toDouble, endTimeMs, false) info(s"Finished loading offsets and group metadata from $topicPartition " + s"in $totalLoadingTimeMs milliseconds, of which $schedulerTimeMs milliseconds" + s" was spent in the scheduler.") @@ -666,19 +666,21 @@ class GroupMetadataManager(brokerId: Int, val (groupOffsets, emptyGroupOffsets) = loadedOffsets .groupBy(_._1.group) - .mapValues(_.map { case (groupTopicPartition, offset) => (groupTopicPartition.topicPartition, offset) }) - .partition { case (group, _) => loadedGroups.contains(group) } + .map { case (k, v) => + k -> v.map { case (groupTopicPartition, offset) => (groupTopicPartition.topicPartition, offset) } + }.partition { case (group, _) => loadedGroups.contains(group) } val pendingOffsetsByGroup = mutable.Map[String, mutable.Map[Long, mutable.Map[TopicPartition, CommitRecordMetadataAndOffset]]]() pendingOffsets.foreach { case (producerId, producerOffsets) => producerOffsets.keySet.map(_.group).foreach(addProducerGroup(producerId, _)) producerOffsets .groupBy(_._1.group) - .mapValues(_.map { case (groupTopicPartition, offset) => (groupTopicPartition.topicPartition, offset)}) .foreach { case (group, offsets) => val groupPendingOffsets = pendingOffsetsByGroup.getOrElseUpdate(group, mutable.Map.empty[Long, mutable.Map[TopicPartition, CommitRecordMetadataAndOffset]]) val groupProducerOffsets = groupPendingOffsets.getOrElseUpdate(producerId, mutable.Map.empty[TopicPartition, CommitRecordMetadataAndOffset]) - groupProducerOffsets ++= offsets + groupProducerOffsets ++= offsets.map { case (groupTopicPartition, offset) => + (groupTopicPartition.topicPartition, offset) + } } } diff --git a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala index b12dbf8124f..4ea86bbae33 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala @@ -22,7 +22,7 @@ import kafka.utils.{Json, Logging} import kafka.zk.{KafkaZkClient, ProducerIdBlockZNode} import org.apache.kafka.common.KafkaException -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** * ProducerIdManager is the part of the transaction coordinator that provides ProducerIds in a unique way diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala index 48bf2234124..dece3aaa1e3 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala @@ -118,7 +118,7 @@ class TransactionCoordinator(brokerId: Int, // check transactionTimeoutMs is not larger than the broker configured maximum allowed value responseCallback(initTransactionError(Errors.INVALID_TRANSACTION_TIMEOUT)) } else { - val coordinatorEpochAndMetadata = txnManager.getTransactionState(transactionalId).right.flatMap { + val coordinatorEpochAndMetadata = txnManager.getTransactionState(transactionalId).flatMap { case None => val producerId = producerIdManager.generateProducerId() val createdMetadata = new TransactionMetadata(transactionalId = transactionalId, @@ -135,7 +135,7 @@ class TransactionCoordinator(brokerId: Int, case Some(epochAndTxnMetadata) => Right(epochAndTxnMetadata) } - val result: ApiResult[(Int, TxnTransitMetadata)] = coordinatorEpochAndMetadata.right.flatMap { + val result: ApiResult[(Int, TxnTransitMetadata)] = coordinatorEpochAndMetadata.flatMap { existingEpochAndMetadata => val coordinatorEpoch = existingEpochAndMetadata.coordinatorEpoch val txnMetadata = existingEpochAndMetadata.transactionMetadata @@ -266,7 +266,7 @@ class TransactionCoordinator(brokerId: Int, } else { // try to update the transaction metadata and append the updated metadata to txn log; // if there is no such metadata treat it as invalid producerId mapping error. - val result: ApiResult[(Int, TxnTransitMetadata)] = txnManager.getTransactionState(transactionalId).right.flatMap { + val result: ApiResult[(Int, TxnTransitMetadata)] = txnManager.getTransactionState(transactionalId).flatMap { case None => Left(Errors.INVALID_PRODUCER_ID_MAPPING) case Some(epochAndMetadata) => @@ -368,7 +368,7 @@ class TransactionCoordinator(brokerId: Int, if (transactionalId == null || transactionalId.isEmpty) responseCallback(Errors.INVALID_REQUEST) else { - val preAppendResult: ApiResult[(Int, TxnTransitMetadata)] = txnManager.getTransactionState(transactionalId).right.flatMap { + val preAppendResult: ApiResult[(Int, TxnTransitMetadata)] = txnManager.getTransactionState(transactionalId).flatMap { case None => Left(Errors.INVALID_PRODUCER_ID_MAPPING) @@ -440,7 +440,7 @@ class TransactionCoordinator(brokerId: Int, case Right((coordinatorEpoch, newMetadata)) => def sendTxnMarkersCallback(error: Errors): Unit = { if (error == Errors.NONE) { - val preSendResult: ApiResult[(TransactionMetadata, TxnTransitMetadata)] = txnManager.getTransactionState(transactionalId).right.flatMap { + val preSendResult: ApiResult[(TransactionMetadata, TxnTransitMetadata)] = txnManager.getTransactionState(transactionalId).flatMap { case None => val errorMsg = s"The coordinator still owns the transaction partition for $transactionalId, but there is " + s"no metadata in the cache; this is not expected" @@ -535,7 +535,7 @@ class TransactionCoordinator(brokerId: Int, private[transaction] def abortTimedOutTransactions(onComplete: TransactionalIdAndProducerIdEpoch => EndTxnCallback): Unit = { txnManager.timedOutTransactions().foreach { txnIdAndPidEpoch => - txnManager.getTransactionState(txnIdAndPidEpoch.transactionalId).right.foreach { + txnManager.getTransactionState(txnIdAndPidEpoch.transactionalId).foreach { case None => error(s"Could not find transaction metadata when trying to timeout transaction for $txnIdAndPidEpoch") diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala index 6345b528792..eaa3232218e 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala @@ -34,7 +34,7 @@ import org.apache.kafka.common.security.JaasContext import org.apache.kafka.common.utils.{LogContext, Time} import org.apache.kafka.common.{Node, Reconfigurable, TopicPartition} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.{concurrent, immutable} object TransactionMarkerChannelManager { diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerRequestCompletionHandler.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerRequestCompletionHandler.scala index f655770ce6b..263e155cda0 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerRequestCompletionHandler.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerRequestCompletionHandler.scala @@ -24,7 +24,7 @@ import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.WriteTxnMarkersResponse import scala.collection.mutable -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class TransactionMarkerRequestCompletionHandler(brokerId: Int, txnStateManager: TransactionStateManager, diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala index baecf9fe9e5..3ef9400acf2 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala @@ -38,7 +38,7 @@ import org.apache.kafka.common.requests.TransactionResult import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{KafkaException, TopicPartition} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable @@ -216,8 +216,8 @@ class TransactionStateManager(brokerId: Int, } def putTransactionStateIfNotExists(txnMetadata: TransactionMetadata): Either[Errors, CoordinatorEpochAndTxnMetadata] = { - getAndMaybeAddTransactionState(txnMetadata.transactionalId, Some(txnMetadata)) - .right.map(_.getOrElse(throw new IllegalStateException(s"Unexpected empty transaction metadata returned while putting $txnMetadata"))) + getAndMaybeAddTransactionState(txnMetadata.transactionalId, Some(txnMetadata)).map(_.getOrElse( + throw new IllegalStateException(s"Unexpected empty transaction metadata returned while putting $txnMetadata"))) } /** @@ -391,7 +391,7 @@ class TransactionStateManager(brokerId: Int, val loadedTransactions = loadTransactionMetadata(topicPartition, coordinatorEpoch) val endTimeMs = time.milliseconds() val totalLoadingTimeMs = endTimeMs - startTimeMs - partitionLoadSensor.record(totalLoadingTimeMs, endTimeMs, false) + partitionLoadSensor.record(totalLoadingTimeMs.toDouble, endTimeMs, false) info(s"Finished loading ${loadedTransactions.size} transaction metadata from $topicPartition in " + s"$totalLoadingTimeMs milliseconds, of which $schedulerTimeMs milliseconds was spent in the scheduler.") @@ -439,7 +439,7 @@ class TransactionStateManager(brokerId: Int, def removeTransactionsForTxnTopicPartition(partitionId: Int): Unit = { val topicPartition = new TopicPartition(Topic.TRANSACTION_STATE_TOPIC_NAME, partitionId) inWriteLock(stateLock) { - loadingPartitions.retain(_.txnPartitionId != partitionId) + loadingPartitions --= loadingPartitions.filter(_.txnPartitionId == partitionId) transactionMetadataCache.remove(partitionId).foreach { txnMetadataCacheEntry => info(s"Unloaded transaction metadata $txnMetadataCacheEntry for $topicPartition following " + s"local partition deletion") diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 6c6f37624ba..e9252d640c6 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -44,7 +44,7 @@ import org.apache.kafka.common.requests.{EpochEndOffset, ListOffsetRequest} import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{InvalidRecordException, KafkaException, TopicPartition} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable.{ArrayBuffer, ListBuffer} import scala.collection.{Seq, Set, mutable} @@ -531,8 +531,8 @@ class Log(@volatile private var _dir: File, Files.deleteIfExists(Log.transactionIndexFile(dir, offset, suffix).toPath) } - var swapFiles = Set[File]() - var cleanFiles = Set[File]() + val swapFiles = mutable.Set[File]() + val cleanFiles = mutable.Set[File]() var minCleanedFileOffset = Long.MaxValue for (file <- dir.listFiles if file.isFile) { @@ -767,7 +767,7 @@ class Log(@volatile private var _dir: File, // if we have the clean shutdown marker, skip recovery if (!hasCleanShutdownFile) { // okay we need to actually recover this log - val unflushed = logSegments(this.recoveryPoint, Long.MaxValue).toIterator + val unflushed = logSegments(this.recoveryPoint, Long.MaxValue).iterator var truncated = false while (unflushed.hasNext && !truncated) { diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 22207e0b4e1..825215135d0 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -34,7 +34,7 @@ import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetention import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.Time -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable.ListBuffer import scala.collection.{Iterable, Seq, Set, mutable} import scala.util.control.ControlThrowable @@ -381,19 +381,19 @@ class LogCleaner(initialConfig: CleanerConfig, def mb(bytes: Double) = bytes / (1024*1024) val message = "%n\tLog cleaner thread %d cleaned log %s (dirty section = [%d, %d])%n".format(id, name, from, to) + - "\t%,.1f MB of log processed in %,.1f seconds (%,.1f MB/sec).%n".format(mb(stats.bytesRead), + "\t%,.1f MB of log processed in %,.1f seconds (%,.1f MB/sec).%n".format(mb(stats.bytesRead.toDouble), stats.elapsedSecs, - mb(stats.bytesRead/stats.elapsedSecs)) + - "\tIndexed %,.1f MB in %.1f seconds (%,.1f Mb/sec, %.1f%% of total time)%n".format(mb(stats.mapBytesRead), + mb(stats.bytesRead.toDouble / stats.elapsedSecs)) + + "\tIndexed %,.1f MB in %.1f seconds (%,.1f Mb/sec, %.1f%% of total time)%n".format(mb(stats.mapBytesRead.toDouble), stats.elapsedIndexSecs, - mb(stats.mapBytesRead)/stats.elapsedIndexSecs, - 100 * stats.elapsedIndexSecs/stats.elapsedSecs) + + mb(stats.mapBytesRead.toDouble) / stats.elapsedIndexSecs, + 100 * stats.elapsedIndexSecs / stats.elapsedSecs) + "\tBuffer utilization: %.1f%%%n".format(100 * stats.bufferUtilization) + - "\tCleaned %,.1f MB in %.1f seconds (%,.1f Mb/sec, %.1f%% of total time)%n".format(mb(stats.bytesRead), + "\tCleaned %,.1f MB in %.1f seconds (%,.1f Mb/sec, %.1f%% of total time)%n".format(mb(stats.bytesRead.toDouble), stats.elapsedSecs - stats.elapsedIndexSecs, - mb(stats.bytesRead)/(stats.elapsedSecs - stats.elapsedIndexSecs), 100 * (stats.elapsedSecs - stats.elapsedIndexSecs).toDouble/stats.elapsedSecs) + - "\tStart size: %,.1f MB (%,d messages)%n".format(mb(stats.bytesRead), stats.messagesRead) + - "\tEnd size: %,.1f MB (%,d messages)%n".format(mb(stats.bytesWritten), stats.messagesWritten) + + mb(stats.bytesRead.toDouble) / (stats.elapsedSecs - stats.elapsedIndexSecs), 100 * (stats.elapsedSecs - stats.elapsedIndexSecs) / stats.elapsedSecs) + + "\tStart size: %,.1f MB (%,d messages)%n".format(mb(stats.bytesRead.toDouble), stats.messagesRead) + + "\tEnd size: %,.1f MB (%,d messages)%n".format(mb(stats.bytesWritten.toDouble), stats.messagesWritten) + "\t%.1f%% size reduction (%.1f%% fewer messages)%n".format(100.0 * (1.0 - stats.bytesWritten.toDouble/stats.bytesRead), 100.0 * (1.0 - stats.messagesWritten.toDouble/stats.messagesRead)) info(message) @@ -1034,9 +1034,9 @@ private class CleanerStats(time: Time = Time.SYSTEM) { endTime = time.milliseconds } - def elapsedSecs = (endTime - startTime)/1000.0 + def elapsedSecs: Double = (endTime - startTime) / 1000.0 - def elapsedIndexSecs = (mapCompleteTime - startTime)/1000.0 + def elapsedIndexSecs: Double = (mapCompleteTime - startTime) / 1000.0 } diff --git a/core/src/main/scala/kafka/log/LogCleanerManager.scala b/core/src/main/scala/kafka/log/LogCleanerManager.scala index ba007c70ac2..f9ea46b764a 100755 --- a/core/src/main/scala/kafka/log/LogCleanerManager.scala +++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala @@ -410,7 +410,7 @@ private[log] class LogCleanerManager(val logDirs: Seq[File], if (checkpoint != null) { val existing = checkpoint.read() if (existing.getOrElse(topicPartition, 0L) > offset) - checkpoint.write(existing + (topicPartition -> offset)) + checkpoint.write(mutable.Map() ++= existing += topicPartition -> offset) } } } diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index 59fe81cc158..9348fee0b2d 100755 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -19,7 +19,7 @@ package kafka.log import java.util.{Collections, Locale, Properties} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import kafka.api.{ApiVersion, ApiVersionValidator} import kafka.message.BrokerCompressionCodec import kafka.server.{KafkaConfig, ThrottledReplicaListValidator} diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index bf4351c70f4..bed3429da46 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -30,7 +30,7 @@ import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.utils.Time import org.apache.kafka.common.errors.{KafkaStorageException, LogDirNotFoundException} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection._ import scala.collection.mutable.ArrayBuffer import scala.util.{Failure, Success, Try} @@ -741,7 +741,7 @@ class LogManager(logDirs: Seq[File], } val logDir = logDirs - .toStream // to prevent actually mapping the whole list, lazy map + .iterator // to prevent actually mapping the whole list, lazy map .map(createLogDirectory(_, logDirName)) .find(_.isSuccess) .getOrElse(Failure(new KafkaStorageException("No log directories available. Tried " + logDirs.map(_.getAbsolutePath).mkString(", ")))) @@ -934,7 +934,7 @@ class LogManager(logDirs: Seq[File], List(_liveLogDirs.peek()) } else { // count the number of logs in each parent directory (including 0 for empty directories - val logCounts = allLogs.groupBy(_.parentDir).mapValues(_.size) + val logCounts = allLogs.groupBy(_.parentDir).map { case (parent, logs) => parent -> logs.size } val zeros = _liveLogDirs.asScala.map(dir => (dir.getPath, 0)).toMap val dirCounts = (zeros ++ logCounts).toBuffer diff --git a/core/src/main/scala/kafka/log/LogSegment.scala b/core/src/main/scala/kafka/log/LogSegment.scala index 6d3beb71076..61a6e59240a 100755 --- a/core/src/main/scala/kafka/log/LogSegment.scala +++ b/core/src/main/scala/kafka/log/LogSegment.scala @@ -32,7 +32,7 @@ import org.apache.kafka.common.record.FileRecords.{LogOffsetPosition, TimestampA import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.Time -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.math._ /** diff --git a/core/src/main/scala/kafka/log/LogValidator.scala b/core/src/main/scala/kafka/log/LogValidator.scala index 1291624132f..08c22701430 100644 --- a/core/src/main/scala/kafka/log/LogValidator.scala +++ b/core/src/main/scala/kafka/log/LogValidator.scala @@ -32,7 +32,7 @@ import org.apache.kafka.common.requests.ProduceResponse.RecordError import org.apache.kafka.common.utils.Time import scala.collection.{Seq, mutable} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable.ArrayBuffer /** diff --git a/core/src/main/scala/kafka/log/OffsetIndex.scala b/core/src/main/scala/kafka/log/OffsetIndex.scala index cd90799519f..5719ee3814f 100755 --- a/core/src/main/scala/kafka/log/OffsetIndex.scala +++ b/core/src/main/scala/kafka/log/OffsetIndex.scala @@ -147,7 +147,7 @@ class OffsetIndex(_file: File, baseOffset: Long, maxIndexSize: Int = -1, writabl mmap.putInt(position) _entries += 1 _lastOffset = offset - require(_entries * entrySize == mmap.position(), entries + " entries but file position in index is " + mmap.position() + ".") + require(_entries * entrySize == mmap.position(), s"$entries entries but file position in index is ${mmap.position()}.") } else { throw new InvalidOffsetException(s"Attempt to append an offset ($offset) to position $entries no larger than" + s" the last offset appended (${_lastOffset}) to ${file.getAbsolutePath}.") diff --git a/core/src/main/scala/kafka/log/ProducerStateManager.scala b/core/src/main/scala/kafka/log/ProducerStateManager.scala index 1480c1777af..a7eaf1aeb75 100644 --- a/core/src/main/scala/kafka/log/ProducerStateManager.scala +++ b/core/src/main/scala/kafka/log/ProducerStateManager.scala @@ -30,7 +30,7 @@ import org.apache.kafka.common.protocol.types._ import org.apache.kafka.common.record.{ControlRecordType, DefaultRecordBatch, EndTransactionMarker, RecordBatch} import org.apache.kafka.common.utils.{ByteUtils, Crc32C} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable.ListBuffer import scala.collection.{immutable, mutable} @@ -580,9 +580,7 @@ class ProducerStateManager(val topicPartition: TopicPartition, * Expire any producer ids which have been idle longer than the configured maximum expiration timeout. */ def removeExpiredProducers(currentTimeMs: Long): Unit = { - producers.retain { case (_, lastEntry) => - !isProducerExpired(currentTimeMs, lastEntry) - } + producers --= producers.filter { case (_, lastEntry) => isProducerExpired(currentTimeMs, lastEntry) }.keySet } /** diff --git a/core/src/main/scala/kafka/log/TimeIndex.scala b/core/src/main/scala/kafka/log/TimeIndex.scala index 40e1dcf320e..6b2739cabca 100644 --- a/core/src/main/scala/kafka/log/TimeIndex.scala +++ b/core/src/main/scala/kafka/log/TimeIndex.scala @@ -135,7 +135,7 @@ class TimeIndex(_file: File, baseOffset: Long, maxIndexSize: Int = -1, writable: mmap.putInt(relativeOffset(offset)) _entries += 1 _lastEntry = TimestampOffset(timestamp, offset) - require(_entries * entrySize == mmap.position(), _entries + " entries but file position in index is " + mmap.position() + ".") + require(_entries * entrySize == mmap.position(), s"${_entries} entries but file position in index is ${mmap.position()}.") } } } diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index c9a7d030c07..faa20f0ba7f 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -39,7 +39,7 @@ import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.utils.{Sanitizer, Time} import scala.collection.mutable -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.reflect.ClassTag object RequestChannel extends Logging { diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index 35d9d7cc6e7..aebb43685f4 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -51,7 +51,7 @@ import org.apache.kafka.common.utils.{KafkaThread, LogContext, Time} import org.slf4j.event.Level import scala.collection._ -import JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable.{ArrayBuffer, Buffer} import scala.util.control.ControlThrowable @@ -180,7 +180,7 @@ class SocketServer(val config: KafkaConfig, .find(_.listenerName == config.interBrokerListenerName) .getOrElse(throw new IllegalStateException(s"Inter-broker listener ${config.interBrokerListenerName} not found, endpoints=${dataPlaneAcceptors.keySet}")) val orderedAcceptors = List(dataPlaneAcceptors.get(interBrokerListener)) ++ - dataPlaneAcceptors.asScala.filterKeys(_ != interBrokerListener).values + dataPlaneAcceptors.asScala.filter { case (k, _) => k != interBrokerListener }.values orderedAcceptors.foreach { acceptor => val endpoint = acceptor.endPoint debug(s"Wait for authorizer to complete start up on listener ${endpoint.listenerName}") diff --git a/core/src/main/scala/kafka/security/auth/Resource.scala b/core/src/main/scala/kafka/security/auth/Resource.scala index e0afb725cc0..8045c681416 100644 --- a/core/src/main/scala/kafka/security/auth/Resource.scala +++ b/core/src/main/scala/kafka/security/auth/Resource.scala @@ -70,7 +70,7 @@ case class Resource(resourceType: ResourceType, name: String, patternType: Patte * @deprecated Since 2.0, use [[kafka.security.auth.Resource(ResourceType, String, PatternType)]] */ @deprecated("Use Resource(ResourceType, String, PatternType", "Since 2.0") - def this(resourceType: ResourceType, name: String) { + def this(resourceType: ResourceType, name: String) = { this(resourceType, name, PatternType.LITERAL) } diff --git a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala index 12ae5b44634..c4ab1a19701 100644 --- a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala +++ b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala @@ -30,7 +30,7 @@ import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult} import scala.collection.mutable -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.compat.java8.OptionConverters._ @deprecated("Use kafka.security.authorizer.AclAuthorizer", "Since 2.4") diff --git a/core/src/main/scala/kafka/security/authorizer/AclAuthorizer.scala b/core/src/main/scala/kafka/security/authorizer/AclAuthorizer.scala index 53e715b5bbb..bbb2fc8e472 100644 --- a/core/src/main/scala/kafka/security/authorizer/AclAuthorizer.scala +++ b/core/src/main/scala/kafka/security/authorizer/AclAuthorizer.scala @@ -21,7 +21,7 @@ import java.util.concurrent.{CompletableFuture, CompletionStage} import com.typesafe.scalalogging.Logger import kafka.api.KAFKA_2_0_IV1 -import kafka.security.authorizer.AclAuthorizer.{AclBuffers, ResourceOrdering, VersionedAcls} +import kafka.security.authorizer.AclAuthorizer.{AclSeqs, ResourceOrdering, VersionedAcls} import kafka.security.authorizer.AclEntry.ResourceSeparator import kafka.server.{KafkaConfig, KafkaServer} import kafka.utils._ @@ -39,8 +39,8 @@ import org.apache.kafka.server.authorizer.AclDeleteResult.AclBindingDeleteResult import org.apache.kafka.server.authorizer._ import org.apache.zookeeper.client.ZKClientConfig -import scala.collection.mutable -import scala.collection.JavaConverters._ +import scala.collection.{mutable, Seq} +import scala.jdk.CollectionConverters._ import scala.util.{Failure, Random, Success, Try} object AclAuthorizer { @@ -62,7 +62,7 @@ object AclAuthorizer { def exists: Boolean = zkVersion != ZkVersion.UnknownVersion } - class AclBuffers(classes: mutable.Buffer[AclEntry]*) { + class AclSeqs(classes: Seq[AclEntry]*) { def find(p: AclEntry => Boolean): Option[AclEntry] = classes.flatMap(_.find(p)).headOption def isEmpty: Boolean = !classes.exists(_.nonEmpty) } @@ -255,8 +255,9 @@ class AclAuthorizer extends Authorizer with Logging { } } } - val deletedResult = deletedBindings.groupBy(_._2) - .mapValues(_.map { case (binding, _) => new AclBindingDeleteResult(binding, deleteExceptions.getOrElse(binding, null)) }) + val deletedResult = deletedBindings.groupBy(_._2).map { case (k, bindings) => + k -> bindings.keys.map { binding => new AclBindingDeleteResult(binding, deleteExceptions.get(binding).orNull) } + } (0 until aclBindingFilters.size).map { i => new AclDeleteResult(deletedResult.getOrElse(i, Set.empty[AclBindingDeleteResult]).toSet.asJava) }.map(CompletableFuture.completedFuture[AclDeleteResult]).asJava @@ -295,7 +296,7 @@ class AclAuthorizer extends Authorizer with Logging { val host = requestContext.clientAddress.getHostAddress val operation = action.operation - def isEmptyAclAndAuthorized(acls: AclBuffers): Boolean = { + def isEmptyAclAndAuthorized(acls: AclSeqs): Boolean = { if (acls.isEmpty) { // No ACLs found for this resource, permission is determined by value of config allow.everyone.if.no.acl.found authorizerLogger.debug(s"No acl found for resource $resource, authorized = $shouldAllowEveryoneIfNoAclIsFound") @@ -303,12 +304,12 @@ class AclAuthorizer extends Authorizer with Logging { } else false } - def denyAclExists(acls: AclBuffers): Boolean = { + def denyAclExists(acls: AclSeqs): Boolean = { // Check if there are any Deny ACLs which would forbid this operation. matchingAclExists(operation, resource, principal, host, DENY, acls) } - def allowAclExists(acls: AclBuffers): Boolean = { + def allowAclExists(acls: AclSeqs): Boolean = { // Check if there are any Allow ACLs which would allow this operation. // Allowing read, write, delete, or alter implies allowing describe. // See #{org.apache.kafka.common.acl.AclOperation} for more details about ACL inheritance. @@ -341,7 +342,7 @@ class AclAuthorizer extends Authorizer with Logging { } else false } - private def matchingAcls(resourceType: ResourceType, resourceName: String): AclBuffers = { + 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. val aclCacheSnapshot = aclCache val wildcard = aclCacheSnapshot.get(new ResourcePattern(resourceType, ResourcePattern.WILDCARD_RESOURCE, PatternType.LITERAL)) @@ -355,12 +356,10 @@ class AclAuthorizer extends Authorizer with Logging { val prefixed = aclCacheSnapshot .from(new ResourcePattern(resourceType, resourceName, PatternType.PREFIXED)) .to(new ResourcePattern(resourceType, resourceName.take(1), PatternType.PREFIXED)) - .filterKeys(resource => resourceName.startsWith(resource.name)) - .values - .flatMap { _.acls.toBuffer } + .flatMap { case (resource, acls) => if (resourceName.startsWith(resource.name)) acls.acls else Seq.empty } .toBuffer - new AclBuffers(prefixed, wildcard, literal) + new AclSeqs(prefixed, wildcard, literal) } private def matchingAclExists(operation: AclOperation, @@ -368,7 +367,7 @@ class AclAuthorizer extends Authorizer with Logging { principal: KafkaPrincipal, host: String, permissionType: AclPermissionType, - acls: AclBuffers): Boolean = { + acls: AclSeqs): Boolean = { acls.find { acl => acl.permissionType == permissionType && (acl.kafkaPrincipal == principal || acl.kafkaPrincipal == AclEntry.WildcardPrincipal) && diff --git a/core/src/main/scala/kafka/security/authorizer/AclEntry.scala b/core/src/main/scala/kafka/security/authorizer/AclEntry.scala index b48438dabe9..2014916e7e4 100644 --- a/core/src/main/scala/kafka/security/authorizer/AclEntry.scala +++ b/core/src/main/scala/kafka/security/authorizer/AclEntry.scala @@ -25,7 +25,7 @@ import org.apache.kafka.common.resource.{ResourcePattern, ResourceType} import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.utils.SecurityUtils -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ object AclEntry { val WildcardPrincipal: KafkaPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "*") diff --git a/core/src/main/scala/kafka/security/authorizer/AuthorizerWrapper.scala b/core/src/main/scala/kafka/security/authorizer/AuthorizerWrapper.scala index 7cbfdaf755b..9a8bf9da49f 100644 --- a/core/src/main/scala/kafka/security/authorizer/AuthorizerWrapper.scala +++ b/core/src/main/scala/kafka/security/authorizer/AuthorizerWrapper.scala @@ -33,7 +33,7 @@ import org.apache.kafka.common.utils.SecurityUtils.parseKafkaPrincipal import org.apache.kafka.server.authorizer.AclDeleteResult.AclBindingDeleteResult import org.apache.kafka.server.authorizer.{AuthorizableRequestContext, AuthorizerServerInfo, _} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable.ArrayBuffer import scala.collection.{Seq, immutable, mutable} import scala.util.{Failure, Success, Try} @@ -175,4 +175,4 @@ class AuthorizerWrapper(private[kafka] val baseAuthorizer: kafka.security.auth.A override def close(): Unit = { baseAuthorizer.close() } -} \ No newline at end of file +} diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala index be680744d54..4e4ab1e7230 100755 --- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala @@ -32,7 +32,7 @@ import kafka.utils.CoreUtils.inLock import org.apache.kafka.common.protocol.Errors import scala.collection.{mutable, Map, Set} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicLong diff --git a/core/src/main/scala/kafka/server/AdminManager.scala b/core/src/main/scala/kafka/server/AdminManager.scala index ea92bfda734..2c6fadd364a 100644 --- a/core/src/main/scala/kafka/server/AdminManager.scala +++ b/core/src/main/scala/kafka/server/AdminManager.scala @@ -46,7 +46,7 @@ import org.apache.kafka.common.requests.{AlterConfigsRequest, ApiError, Describe import org.apache.kafka.common.utils.Sanitizer import scala.collection.{Map, mutable, _} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class AdminManager(val config: KafkaConfig, val metrics: Metrics, @@ -518,8 +518,9 @@ class AdminManager(val config: KafkaConfig, configs.map { case (resource, alterConfigOps) => try { // throw InvalidRequestException if any duplicate keys - val duplicateKeys = alterConfigOps.groupBy(config => config.configEntry().name()) - .mapValues(_.size).filter(_._2 > 1).keys.toSet + val duplicateKeys = alterConfigOps.groupBy(config => config.configEntry.name).filter { case (_, v) => + v.size > 1 + }.keySet if (duplicateKeys.nonEmpty) throw new InvalidRequestException(s"Error due to duplicate config keys : ${duplicateKeys.mkString(",")}") val nullUpdates = alterConfigOps @@ -773,7 +774,7 @@ class AdminManager(val config: KafkaConfig, } def handleDescribeClientQuotas(userComponent: Option[ClientQuotaFilterComponent], - clientIdComponent: Option[ClientQuotaFilterComponent], strict: Boolean) = { + clientIdComponent: Option[ClientQuotaFilterComponent], strict: Boolean): Map[ClientQuotaEntity, Map[String, Double]] = { def toOption(opt: java.util.Optional[String]): Option[String] = if (opt == null) diff --git a/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala b/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala index ffbcb5df059..167ce2717bd 100755 --- a/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala +++ b/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala @@ -66,7 +66,7 @@ class BrokerMetadataCheckpoint(val file: File) extends Logging { } def read(): Option[BrokerMetadata] = { - Files.deleteIfExists(new File(file + ".tmp").toPath()) // try to delete any existing temp files for cleanliness + Files.deleteIfExists(new File(file.getPath + ".tmp").toPath()) // try to delete any existing temp files for cleanliness lock synchronized { try { diff --git a/core/src/main/scala/kafka/server/ClientQuotaManager.scala b/core/src/main/scala/kafka/server/ClientQuotaManager.scala index 8316d0c637b..8da19d94f75 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaManager.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaManager.scala @@ -32,7 +32,7 @@ import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.utils.{Sanitizer, Time} import org.apache.kafka.server.quota.{ClientQuotaCallback, ClientQuotaEntity, ClientQuotaType} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** * Represents the sensors aggregated per client @@ -162,7 +162,7 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, private val time: Time, threadNamePrefix: String, clientQuotaCallback: Option[ClientQuotaCallback] = None) extends Logging { - private val staticConfigClientIdQuota = Quota.upperBound(config.quotaBytesPerSecondDefault) + private val staticConfigClientIdQuota = Quota.upperBound(config.quotaBytesPerSecondDefault.toDouble) private val clientQuotaType = quotaTypeToClientQuotaType(quotaType) @volatile private var quotaTypesEnabled = clientQuotaCallback match { case Some(_) => QuotaTypes.CustomQuotas @@ -176,7 +176,7 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, private[server] val throttledChannelReaper = new ThrottledChannelReaper(delayQueue, threadNamePrefix) private val quotaCallback = clientQuotaCallback.getOrElse(new DefaultQuotaCallback) - private val delayQueueSensor = metrics.sensor(quotaType + "-delayQueue") + private val delayQueueSensor = metrics.sensor(quotaType.toString + "-delayQueue") delayQueueSensor.add(metrics.metricName("queue-size", quotaType.toString, "Tracks the size of the delay queue"), new CumulativeSum()) @@ -506,8 +506,8 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, } } else { val quotaMetricName = clientRateMetricName(Map.empty) - allMetrics.asScala.filterKeys(n => n.name == quotaMetricName.name && n.group == quotaMetricName.group).foreach { - case (metricName, metric) => + allMetrics.asScala.foreach { case (metricName, metric) => + if (metricName.name == quotaMetricName.name && metricName.group == quotaMetricName.group) { val metricTags = metricName.tags Option(quotaLimit(metricTags)).foreach { newQuota => if (newQuota != metric.config.quota.bound) { @@ -515,6 +515,7 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, metric.config(getQuotaMetricConfig(newQuota)) } } + } } } } diff --git a/core/src/main/scala/kafka/server/ClientRequestQuotaManager.scala b/core/src/main/scala/kafka/server/ClientRequestQuotaManager.scala index eb921dca7d3..817b7ce78ce 100644 --- a/core/src/main/scala/kafka/server/ClientRequestQuotaManager.scala +++ b/core/src/main/scala/kafka/server/ClientRequestQuotaManager.scala @@ -24,7 +24,7 @@ import org.apache.kafka.common.metrics._ import org.apache.kafka.common.utils.Time import org.apache.kafka.server.quota.ClientQuotaCallback -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class ClientRequestQuotaManager(private val config: ClientQuotaManagerConfig, diff --git a/core/src/main/scala/kafka/server/ConfigHandler.scala b/core/src/main/scala/kafka/server/ConfigHandler.scala index 2395dcde79d..3e30584010c 100644 --- a/core/src/main/scala/kafka/server/ConfigHandler.scala +++ b/core/src/main/scala/kafka/server/ConfigHandler.scala @@ -33,7 +33,7 @@ import org.apache.kafka.common.metrics.Quota import org.apache.kafka.common.metrics.Quota._ import org.apache.kafka.common.utils.Sanitizer -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.Seq import scala.util.Try @@ -128,13 +128,13 @@ class QuotaConfigHandler(private val quotaManagers: QuotaManagers) { val clientId = sanitizedClientId.map(Sanitizer.desanitize) val producerQuota = if (config.containsKey(DynamicConfig.Client.ProducerByteRateOverrideProp)) - Some(new Quota(config.getProperty(DynamicConfig.Client.ProducerByteRateOverrideProp).toLong, true)) + Some(new Quota(config.getProperty(DynamicConfig.Client.ProducerByteRateOverrideProp).toLong.toDouble, true)) else None quotaManagers.produce.updateQuota(sanitizedUser, clientId, sanitizedClientId, producerQuota) val consumerQuota = if (config.containsKey(DynamicConfig.Client.ConsumerByteRateOverrideProp)) - Some(new Quota(config.getProperty(DynamicConfig.Client.ConsumerByteRateOverrideProp).toLong, true)) + Some(new Quota(config.getProperty(DynamicConfig.Client.ConsumerByteRateOverrideProp).toLong.toDouble, true)) else None quotaManagers.fetch.updateQuota(sanitizedUser, clientId, sanitizedClientId, consumerQuota) @@ -197,9 +197,9 @@ class BrokerConfigHandler(private val brokerConfig: KafkaConfig, brokerConfig.dynamicConfig.updateDefaultConfig(properties) else if (brokerConfig.brokerId == brokerId.trim.toInt) { brokerConfig.dynamicConfig.updateBrokerConfig(brokerConfig.brokerId, properties) - quotaManagers.leader.updateQuota(upperBound(getOrDefault(LeaderReplicationThrottledRateProp))) - quotaManagers.follower.updateQuota(upperBound(getOrDefault(FollowerReplicationThrottledRateProp))) - quotaManagers.alterLogDirs.updateQuota(upperBound(getOrDefault(ReplicaAlterLogDirsIoMaxBytesPerSecondProp))) + quotaManagers.leader.updateQuota(upperBound(getOrDefault(LeaderReplicationThrottledRateProp).toDouble)) + quotaManagers.follower.updateQuota(upperBound(getOrDefault(FollowerReplicationThrottledRateProp).toDouble)) + quotaManagers.alterLogDirs.updateQuota(upperBound(getOrDefault(ReplicaAlterLogDirsIoMaxBytesPerSecondProp).toDouble)) } } } diff --git a/core/src/main/scala/kafka/server/DelayedElectLeader.scala b/core/src/main/scala/kafka/server/DelayedElectLeader.scala index d599f98411b..cd0a8040589 100644 --- a/core/src/main/scala/kafka/server/DelayedElectLeader.scala +++ b/core/src/main/scala/kafka/server/DelayedElectLeader.scala @@ -34,7 +34,7 @@ class DelayedElectLeader( responseCallback: Map[TopicPartition, ApiError] => Unit ) extends DelayedOperation(delayMs) { - private var waitingPartitions = expectedLeaders + private val waitingPartitions = mutable.Map() ++= expectedLeaders private val fullResults = mutable.Map() ++= results @@ -52,7 +52,7 @@ class DelayedElectLeader( updateWaiting() val timedOut = waitingPartitions.map { case (tp, _) => tp -> new ApiError(Errors.REQUEST_TIMED_OUT, null) - }.toMap + } responseCallback(timedOut ++ fullResults) } diff --git a/core/src/main/scala/kafka/server/DelegationTokenManager.scala b/core/src/main/scala/kafka/server/DelegationTokenManager.scala index 93053e23570..3b8fc6704f4 100644 --- a/core/src/main/scala/kafka/server/DelegationTokenManager.scala +++ b/core/src/main/scala/kafka/server/DelegationTokenManager.scala @@ -36,7 +36,7 @@ import org.apache.kafka.common.security.token.delegation.internals.DelegationTok import org.apache.kafka.common.security.token.delegation.{DelegationToken, TokenInformation} import org.apache.kafka.common.utils.{Sanitizer, SecurityUtils, Time} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable object DelegationTokenManager { diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index 7066e940817..4a88a884d34 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -36,7 +36,7 @@ import org.apache.kafka.common.security.authenticator.LoginManager import org.apache.kafka.common.utils.Utils import scala.collection._ -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** * Dynamic broker configurations are stored in ZooKeeper and may be defined at two levels: @@ -85,8 +85,8 @@ object DynamicBrokerConfig { SocketServer.ReconfigurableConfigs private val ClusterLevelListenerConfigs = Set(KafkaConfig.MaxConnectionsProp) - private val PerBrokerConfigs = DynamicSecurityConfigs ++ - DynamicListenerConfig.ReconfigurableConfigs -- ClusterLevelListenerConfigs + private val PerBrokerConfigs = (DynamicSecurityConfigs ++ DynamicListenerConfig.ReconfigurableConfigs).diff( + ClusterLevelListenerConfigs) private val ListenerMechanismConfigs = Set(KafkaConfig.SaslJaasConfigProp, KafkaConfig.SaslLoginCallbackHandlerClassProp, KafkaConfig.SaslLoginClassProp, @@ -168,10 +168,12 @@ object DynamicBrokerConfig { } private[server] def addDynamicConfigs(configDef: ConfigDef): Unit = { - KafkaConfig.configKeys.filterKeys(AllDynamicConfigs.contains).values.foreach { config => - configDef.define(config.name, config.`type`, config.defaultValue, config.validator, - config.importance, config.documentation, config.group, config.orderInGroup, config.width, - config.displayName, config.dependents, config.recommender) + KafkaConfig.configKeys.foreach { case (configName, config) => + if (AllDynamicConfigs.contains(configName)) { + configDef.define(config.name, config.`type`, config.defaultValue, config.validator, + config.importance, config.documentation, config.group, config.orderInGroup, config.width, + config.displayName, config.dependents, config.recommender) + } } } @@ -352,7 +354,10 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging props.setProperty(configName, passwordEncoder.encode(new Password(value))) } } - configProps.asScala.filterKeys(isPasswordConfig).foreach { case (name, value) => encodePassword(name, value) } + configProps.asScala.foreach { case (name, value) => + if (isPasswordConfig(name)) + encodePassword(name, value) + } props } @@ -386,7 +391,10 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging } } - props.asScala.filterKeys(isPasswordConfig).foreach { case (name, value) => decodePassword(name, value) } + props.asScala.foreach { case (name, value) => + if (isPasswordConfig(name)) + decodePassword(name, value) + } props } @@ -398,8 +406,8 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging val props = persistentProps.clone().asInstanceOf[Properties] if (props.asScala.keySet.exists(isPasswordConfig)) { maybeCreatePasswordEncoder(kafkaConfig.passwordEncoderOldSecret).foreach { passwordDecoder => - persistentProps.asScala.filterKeys(isPasswordConfig).foreach { case (configName, value) => - if (value != null) { + persistentProps.asScala.foreach { case (configName, value) => + if (isPasswordConfig(configName) && value != null) { val decoded = try { Some(passwordDecoder.decode(value).value) } catch { @@ -633,7 +641,7 @@ class DynamicLogConfig(logManager: LogManager, server: KafkaServer) extends Brok logManager.allLogs.foreach { log => val props = mutable.Map.empty[Any, Any] props ++= newBrokerDefaults - props ++= log.config.originals.asScala.filterKeys(log.config.overriddenConfigs.contains) + props ++= log.config.originals.asScala.filter { case (k, _) => log.config.overriddenConfigs.contains(k) } val logConfig = LogConfig(props.asJava, log.config.overriddenConfigs) log.updateConfig(logConfig) @@ -644,8 +652,8 @@ class DynamicLogConfig(logManager: LogManager, server: KafkaServer) extends Brok val currentLogConfig = logManager.currentDefaultConfig val origUncleanLeaderElectionEnable = logManager.currentDefaultConfig.uncleanLeaderElectionEnable val newBrokerDefaults = new util.HashMap[String, Object](currentLogConfig.originals) - newConfig.valuesFromThisConfig.asScala.filterKeys(DynamicLogConfig.ReconfigurableConfigs.contains).foreach { case (k, v) => - if (v != null) { + newConfig.valuesFromThisConfig.asScala.foreach { case (k, v) => + if (DynamicLogConfig.ReconfigurableConfigs.contains(k) && v != null) { DynamicLogConfig.KafkaConfigToLogConfigName.get(k).foreach { configName => newBrokerDefaults.put(configName, v.asInstanceOf[AnyRef]) } @@ -678,17 +686,19 @@ class DynamicThreadPool(server: KafkaServer) extends BrokerReconfigurable { } override def validateReconfiguration(newConfig: KafkaConfig): Unit = { - newConfig.values.asScala.filterKeys(DynamicThreadPool.ReconfigurableConfigs.contains).foreach { case (k, v) => - val newValue = v.asInstanceOf[Int] - val oldValue = currentValue(k) - if (newValue != oldValue) { - val errorMsg = s"Dynamic thread count update validation failed for $k=$v" - if (newValue <= 0) - throw new ConfigException(s"$errorMsg, value should be at least 1") - if (newValue < oldValue / 2) - throw new ConfigException(s"$errorMsg, value should be at least half the current value $oldValue") - if (newValue > oldValue * 2) - throw new ConfigException(s"$errorMsg, value should not be greater than double the current value $oldValue") + newConfig.values.asScala.foreach { case (k, v) => + if (DynamicThreadPool.ReconfigurableConfigs.contains(k)) { + val newValue = v.asInstanceOf[Int] + val oldValue = currentValue(k) + if (newValue != oldValue) { + val errorMsg = s"Dynamic thread count update validation failed for $k=$v" + if (newValue <= 0) + throw new ConfigException(s"$errorMsg, value should be at least 1") + if (newValue < oldValue / 2) + throw new ConfigException(s"$errorMsg, value should be at least half the current value $oldValue") + if (newValue > oldValue * 2) + throw new ConfigException(s"$errorMsg, value should not be greater than double the current value $oldValue") + } } } } @@ -762,7 +772,7 @@ class DynamicMetricsReporters(brokerId: Int, server: KafkaServer) extends Reconf override def reconfigure(configs: util.Map[String, _]): Unit = { val updatedMetricsReporters = metricsReporterClasses(configs) - val deleted = currentReporters.keySet -- updatedMetricsReporters + val deleted = currentReporters.keySet.toSet -- updatedMetricsReporters deleted.foreach(removeReporter) currentReporters.values.foreach { case reporter: Reconfigurable => dynamicConfig.maybeReconfigure(reporter, dynamicConfig.currentKafkaConfig, configs) diff --git a/core/src/main/scala/kafka/server/DynamicConfig.scala b/core/src/main/scala/kafka/server/DynamicConfig.scala index 4f7b2d4faa6..13c64bfe3d6 100644 --- a/core/src/main/scala/kafka/server/DynamicConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicConfig.scala @@ -26,7 +26,7 @@ import org.apache.kafka.common.config.ConfigDef.Importance._ import org.apache.kafka.common.config.ConfigDef.Range._ import org.apache.kafka.common.config.ConfigDef.Type._ -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** * Class used to hold dynamic configs. These are configs which have no physical manifestation in the server.properties diff --git a/core/src/main/scala/kafka/server/DynamicConfigManager.scala b/core/src/main/scala/kafka/server/DynamicConfigManager.scala index 4a80a6f15ea..a8e8194fcd1 100644 --- a/core/src/main/scala/kafka/server/DynamicConfigManager.scala +++ b/core/src/main/scala/kafka/server/DynamicConfigManager.scala @@ -27,7 +27,7 @@ import org.apache.kafka.common.config.types.Password import org.apache.kafka.common.security.scram.internals.ScramMechanism import org.apache.kafka.common.utils.Time -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection._ /** diff --git a/core/src/main/scala/kafka/server/FetchSession.scala b/core/src/main/scala/kafka/server/FetchSession.scala index 3b77c079c58..cc5d1d5e052 100644 --- a/core/src/main/scala/kafka/server/FetchSession.scala +++ b/core/src/main/scala/kafka/server/FetchSession.scala @@ -32,7 +32,7 @@ import org.apache.kafka.common.utils.{ImplicitLinkedHashCollection, Time, Utils} import scala.math.Ordered.orderingToOrdered import scala.collection.{mutable, _} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ object FetchSession { type REQ_MAP = util.Map[TopicPartition, FetchRequest.PartitionData] diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index f09746912fa..0a7607acb75 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -81,7 +81,7 @@ import org.apache.kafka.common.message.MetadataResponseData.{MetadataResponsePar import org.apache.kafka.server.authorizer._ import scala.compat.java8.OptionConverters._ -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable.ArrayBuffer import scala.collection.{Map, Seq, Set, immutable, mutable} import scala.util.{Failure, Success, Try} @@ -892,9 +892,9 @@ class KafkaApis(val requestChannel: RequestChannel, case (topicPartition, _) => authorizedTopics.contains(topicPartition.topic) } - val unauthorizedResponseStatus = unauthorizedRequestInfo.mapValues(_ => - new ListOffsetResponse.PartitionData(Errors.TOPIC_AUTHORIZATION_FAILED, List[JLong]().asJava) - ) + val unauthorizedResponseStatus = unauthorizedRequestInfo.map { case (k, _) => + k -> new ListOffsetResponse.PartitionData(Errors.TOPIC_AUTHORIZATION_FAILED, Seq.empty[JLong].asJava) + } val responseMap = authorizedRequestInfo.map {case (topicPartition, partitionData) => try { @@ -932,12 +932,12 @@ class KafkaApis(val requestChannel: RequestChannel, case (topicPartition, _) => authorizedTopics.contains(topicPartition.topic) } - val unauthorizedResponseStatus = unauthorizedRequestInfo.mapValues(_ => { - new ListOffsetResponse.PartitionData(Errors.TOPIC_AUTHORIZATION_FAILED, + val unauthorizedResponseStatus = unauthorizedRequestInfo.map { case (k, _) => + k -> new ListOffsetResponse.PartitionData(Errors.TOPIC_AUTHORIZATION_FAILED, ListOffsetResponse.UNKNOWN_TIMESTAMP, ListOffsetResponse.UNKNOWN_OFFSET, Optional.empty()) - }) + } val responseMap = authorizedRequestInfo.map { case (topicPartition, partitionData) => if (offsetRequest.duplicatePartitions.contains(topicPartition)) { @@ -1081,7 +1081,7 @@ class KafkaApis(val requestChannel: RequestChannel, if (topics.isEmpty || topicResponses.size == topics.size) { topicResponses } else { - val nonExistentTopics = topics -- topicResponses.map(_.name).toSet + val nonExistentTopics = topics.diff(topicResponses.map(_.name).toSet) val responsesForNonExistentTopics = nonExistentTopics.map { topic => if (isInternal(topic)) { val topicMetadata = createInternalTopic(topic) @@ -1120,8 +1120,8 @@ class KafkaApis(val requestChannel: RequestChannel, if (metadataRequest.allowAutoTopicCreation && config.autoCreateTopicsEnable && nonExistingTopics.nonEmpty) { if (!authorize(request, CREATE, CLUSTER, CLUSTER_NAME, logIfDenied = false)) { val authorizedForCreateTopics = filterAuthorized(request, CREATE, TOPIC, nonExistingTopics.toSeq) - unauthorizedForCreateTopics = nonExistingTopics -- authorizedForCreateTopics - authorizedTopics --= unauthorizedForCreateTopics + unauthorizedForCreateTopics = nonExistingTopics.diff(authorizedForCreateTopics) + authorizedTopics = authorizedTopics.diff(unauthorizedForCreateTopics) } } } @@ -2201,7 +2201,7 @@ class KafkaApis(val requestChannel: RequestChannel, // the callback for sending an offset commit response def sendResponseCallback(authorizedTopicErrors: Map[TopicPartition, Errors]): Unit = { - var combinedCommitStatus = authorizedTopicErrors ++ unauthorizedTopicErrors ++ nonExistingTopicErrors + val combinedCommitStatus = mutable.Map() ++= authorizedTopicErrors ++= unauthorizedTopicErrors ++= nonExistingTopicErrors if (isDebugEnabled) combinedCommitStatus.foreach { case (topicPartition, error) => if (error != Errors.NONE) { @@ -2216,10 +2216,8 @@ class KafkaApis(val requestChannel: RequestChannel, // txn commit protocol >= 2 (version 2.3 and onwards) are guaranteed to have // the fix to check for the loading error. if (txnOffsetCommitRequest.version < 2) { - combinedCommitStatus.foreach { case (topicPartition, error) => - if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS) { - combinedCommitStatus += topicPartition -> Errors.COORDINATOR_NOT_AVAILABLE - } + combinedCommitStatus ++= combinedCommitStatus.collect { + case (tp, error) if error == Errors.COORDINATOR_LOAD_IN_PROGRESS => tp -> Errors.COORDINATOR_NOT_AVAILABLE } } @@ -2370,9 +2368,10 @@ class KafkaApis(val requestChannel: RequestChannel, } val endOffsetsForAuthorizedPartitions = replicaManager.lastOffsetForLeaderEpoch(authorizedPartitions) - val endOffsetsForUnauthorizedPartitions = unauthorizedPartitions.mapValues(_ => - new EpochEndOffset(Errors.TOPIC_AUTHORIZATION_FAILED, EpochEndOffset.UNDEFINED_EPOCH, - EpochEndOffset.UNDEFINED_EPOCH_OFFSET)) + val endOffsetsForUnauthorizedPartitions = unauthorizedPartitions.map { case (k, _) => + k -> new EpochEndOffset(Errors.TOPIC_AUTHORIZATION_FAILED, EpochEndOffset.UNDEFINED_EPOCH, + EpochEndOffset.UNDEFINED_EPOCH_OFFSET) + } val endOffsetsForAllPartitions = endOffsetsForAuthorizedPartitions ++ endOffsetsForUnauthorizedPartitions sendResponseMaybeThrottle(request, requestThrottleMs => @@ -2844,8 +2843,9 @@ class KafkaApis(val requestChannel: RequestChannel, val describeClientQuotasRequest = request.body[DescribeClientQuotasRequest] if (authorize(request, DESCRIBE_CONFIGS, CLUSTER, CLUSTER_NAME)) { - val result = adminManager.describeClientQuotas( - describeClientQuotasRequest.filter).mapValues(_.mapValues(Double.box).toMap.asJava).toMap.asJava + val result = adminManager.describeClientQuotas(describeClientQuotasRequest.filter).map { case (quotaEntity, quotaConfigs) => + quotaEntity -> quotaConfigs.map { case (key, value) => key -> Double.box(value) }.asJava + }.asJava sendResponseMaybeThrottle(request, requestThrottleMs => new DescribeClientQuotasResponse(result, requestThrottleMs)) } else { @@ -2890,15 +2890,16 @@ class KafkaApis(val requestChannel: RequestChannel, logIfDenied: Boolean = true): Set[String] = { authorizer match { case Some(authZ) => - val resources = resourceNames.groupBy(identity).mapValues(_.size).toList - val actions = resources.map { case (resourceName, count) => + val groupedResourceNames = resourceNames.groupBy(identity) + val actions = resourceNames.map { resourceName => + val count = groupedResourceNames(resourceName).size val resource = new ResourcePattern(resourceType, resourceName, PatternType.LITERAL) new Action(operation, resource, count, logIfAllowed, logIfDenied) } authZ.authorize(request.context, actions.asJava).asScala - .zip(resources.map(_._1)) // zip with resource name - .filter(_._1 == AuthorizationResult.ALLOWED) // filter authorized resources - .map(_._2).toSet + .zip(resourceNames) + .filter { case (authzResult, _) => authzResult == AuthorizationResult.ALLOWED } + .map { case (_, resourceName) => resourceName }.toSet case None => resourceNames.toSet } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index fe0f8dfcb9b..636a1c49881 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -43,7 +43,7 @@ import org.apache.kafka.common.utils.Utils import org.apache.kafka.server.authorizer.Authorizer import org.apache.zookeeper.client.ZKClientConfig -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.{Map, Seq} object Defaults { @@ -239,7 +239,7 @@ object Defaults { /** ********* Sasl configuration ***********/ val SaslMechanismInterBrokerProtocol = SaslConfigs.DEFAULT_SASL_MECHANISM - val SaslEnabledMechanisms = SaslConfigs.DEFAULT_SASL_ENABLED_MECHANISMS + val SaslEnabledMechanisms = BrokerSecurityConfigs.DEFAULT_SASL_ENABLED_MECHANISMS val SaslKerberosKinitCmd = SaslConfigs.DEFAULT_KERBEROS_KINIT_CMD val SaslKerberosTicketRenewWindowFactor = SaslConfigs.DEFAULT_KERBEROS_TICKET_RENEW_WINDOW_FACTOR val SaslKerberosTicketRenewJitter = SaslConfigs.DEFAULT_KERBEROS_TICKET_RENEW_JITTER diff --git a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala index e4583fda926..dbf34848e51 100755 --- a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala +++ b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala @@ -28,7 +28,7 @@ import org.apache.kafka.common.internals.FatalExitError import org.apache.kafka.common.utils.{KafkaThread, Time} import scala.collection.mutable -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** * A thread that answers kafka requests. diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 4713f33615f..8a829b4cce2 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -50,7 +50,7 @@ import org.apache.kafka.common.{ClusterResource, Endpoint, Node} import org.apache.kafka.server.authorizer.Authorizer import org.apache.zookeeper.client.ZKClientConfig -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.{Map, Seq, mutable} object KafkaServer { @@ -307,9 +307,13 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP authorizer.foreach(_.configure(config.originals)) val authorizerFutures: Map[Endpoint, CompletableFuture[Void]] = authorizer match { case Some(authZ) => - authZ.start(brokerInfo.broker.toServerInfo(clusterId, config)).asScala.mapValues(_.toCompletableFuture).toMap + authZ.start(brokerInfo.broker.toServerInfo(clusterId, config)).asScala.map { case (ep, cs) => + ep -> cs.toCompletableFuture + } case None => - brokerInfo.broker.endPoints.map { ep => ep.toJava -> CompletableFuture.completedFuture[Void](null) }.toMap + brokerInfo.broker.endPoints.map { ep => + ep.toJava -> CompletableFuture.completedFuture[Void](null) + }.toMap } val fetchManager = new FetchManager(Time.SYSTEM, @@ -726,7 +730,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP } if (brokerMetadataSet.size > 1) { - val builder = StringBuilder.newBuilder + val builder = new StringBuilder for ((logDir, brokerMetadata) <- brokerMetadataMap) builder ++= s"- $logDir -> $brokerMetadata\n" diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala index 03d5c2a4a99..3772b75ae2a 100755 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -22,7 +22,7 @@ import java.util.{Collections} import java.util.concurrent.locks.ReentrantReadWriteLock import scala.collection.{mutable, Seq, Set} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import kafka.cluster.{Broker, EndPoint} import kafka.api._ import kafka.controller.StateChangeLogger @@ -199,7 +199,7 @@ class MetadataCache(brokerId: Int) extends Logging { } def getNonExistingTopics(topics: Set[String]): Set[String] = { - topics -- metadataSnapshot.partitionStates.keySet + topics.diff(metadataSnapshot.partitionStates.keySet) } def getAliveBroker(brokerId: Int): Option[Broker] = { diff --git a/core/src/main/scala/kafka/server/ReplicaAlterLogDirsThread.scala b/core/src/main/scala/kafka/server/ReplicaAlterLogDirsThread.scala index 6b3ea853a9a..43bae8f7f57 100644 --- a/core/src/main/scala/kafka/server/ReplicaAlterLogDirsThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaAlterLogDirsThread.scala @@ -34,7 +34,7 @@ import org.apache.kafka.common.requests.EpochEndOffset._ import org.apache.kafka.common.requests.FetchResponse.PartitionData import org.apache.kafka.common.requests.{EpochEndOffset, FetchRequest, FetchResponse} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.{mutable, Map, Seq, Set} class ReplicaAlterLogDirsThread(name: String, diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherBlockingSend.scala b/core/src/main/scala/kafka/server/ReplicaFetcherBlockingSend.scala index 20a0067a1f9..2d847e0b007 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherBlockingSend.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherBlockingSend.scala @@ -29,7 +29,7 @@ import org.apache.kafka.clients.{ApiVersions, ClientResponse, ManualMetadataUpda import org.apache.kafka.common.{Node, Reconfigurable} import org.apache.kafka.common.requests.AbstractRequest.Builder -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ trait BlockingSend { diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index f9e5fa7e557..a42f6891a48 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -34,7 +34,7 @@ import org.apache.kafka.common.requests.EpochEndOffset._ import org.apache.kafka.common.requests._ import org.apache.kafka.common.utils.{LogContext, Time} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.{mutable, Map} class ReplicaFetcherThread(name: String, diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 0d2b31e3672..3ae15714157 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -57,7 +57,7 @@ import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.requests._ import org.apache.kafka.common.utils.Time -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.{Map, Seq, Set, mutable} import scala.compat.java8.OptionConverters._ @@ -181,7 +181,7 @@ class ReplicaManager(val config: KafkaConfig, brokerTopicStats: BrokerTopicStats, metadataCache: MetadataCache, logDirFailureChannel: LogDirFailureChannel, - threadNamePrefix: Option[String] = None) { + threadNamePrefix: Option[String] = None) = { this(config, metrics, time, zkClient, scheduler, logManager, isShuttingDown, quotaManagers, brokerTopicStats, metadataCache, logDirFailureChannel, DelayedOperationPurgatory[DelayedProduce]( @@ -1256,14 +1256,14 @@ class ReplicaManager(val config: KafkaConfig, } } - val partitionsTobeLeader = partitionStates.filter { case (_, partitionState) => + val partitionsToBeLeader = partitionStates.filter { case (_, partitionState) => partitionState.leader == localBrokerId } - val partitionsToBeFollower = partitionStates -- partitionsTobeLeader.keys + val partitionsToBeFollower = partitionStates.filter { case (k, _) => !partitionsToBeLeader.contains(k) } val highWatermarkCheckpoints = new LazyOffsetCheckpoints(this.highWatermarkCheckpoints) - val partitionsBecomeLeader = if (partitionsTobeLeader.nonEmpty) - makeLeaders(controllerId, controllerEpoch, partitionsTobeLeader, correlationId, responseMap, + val partitionsBecomeLeader = if (partitionsToBeLeader.nonEmpty) + makeLeaders(controllerId, controllerEpoch, partitionsToBeLeader, correlationId, responseMap, highWatermarkCheckpoints) else Set.empty[Partition] diff --git a/core/src/main/scala/kafka/server/ReplicationQuotaManager.scala b/core/src/main/scala/kafka/server/ReplicationQuotaManager.scala index c4024645564..d388e34a8e4 100644 --- a/core/src/main/scala/kafka/server/ReplicationQuotaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicationQuotaManager.scala @@ -134,7 +134,7 @@ class ReplicationQuotaManager(val config: ReplicationQuotaManagerConfig, */ def record(value: Long): Unit = { try { - sensor().record(value) + sensor().record(value.toDouble) } catch { case qve: QuotaViolationException => trace(s"Record: Quota violated, but ignored, for sensor (${sensor.name}), metric: (${qve.metricName}), value : (${qve.value}), bound: (${qve.bound}), recordedValue ($value)") diff --git a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala index 92c6c9cb84e..bc69fc286a3 100755 --- a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala @@ -37,7 +37,7 @@ import org.apache.kafka.common.requests.ListOffsetRequest import org.apache.kafka.common.serialization.{ByteArrayDeserializer, Deserializer} import org.apache.kafka.common.utils.Utils -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** * Consumer that dumps messages to standard out. diff --git a/core/src/main/scala/kafka/tools/ConsoleProducer.scala b/core/src/main/scala/kafka/tools/ConsoleProducer.scala index 6a19981fbd3..06bef80e327 100644 --- a/core/src/main/scala/kafka/tools/ConsoleProducer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleProducer.scala @@ -31,7 +31,7 @@ import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, Produce import org.apache.kafka.common.KafkaException import org.apache.kafka.common.utils.Utils -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ object ConsoleProducer { diff --git a/core/src/main/scala/kafka/tools/ConsumerPerformance.scala b/core/src/main/scala/kafka/tools/ConsumerPerformance.scala index 8d1911d7e08..7a5858ccb42 100644 --- a/core/src/main/scala/kafka/tools/ConsumerPerformance.scala +++ b/core/src/main/scala/kafka/tools/ConsumerPerformance.scala @@ -30,7 +30,7 @@ import org.apache.kafka.common.serialization.ByteArrayDeserializer import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.{Metric, MetricName, TopicPartition} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable /** @@ -175,7 +175,7 @@ object ConsumerPerformance extends LazyLogging { startMs: Long, endMs: Long, dateFormat: SimpleDateFormat): Unit = { - val elapsedMs: Double = endMs - startMs + val elapsedMs: Double = (endMs - startMs).toDouble val totalMbRead = (bytesRead * 1.0) / (1024 * 1024) val intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024) val intervalMbPerSec = 1000.0 * intervalMbRead / elapsedMs diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala b/core/src/main/scala/kafka/tools/DumpLogSegments.scala index 5af4a473346..9e9546a9274 100755 --- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala +++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala @@ -27,7 +27,7 @@ import kafka.utils._ import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.Utils -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/kafka/tools/EndToEndLatency.scala b/core/src/main/scala/kafka/tools/EndToEndLatency.scala index b21f93be735..35417ad1d2d 100755 --- a/core/src/main/scala/kafka/tools/EndToEndLatency.scala +++ b/core/src/main/scala/kafka/tools/EndToEndLatency.scala @@ -29,7 +29,7 @@ import org.apache.kafka.clients.producer._ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.utils.Utils -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.util.Random @@ -147,7 +147,7 @@ object EndToEndLatency { //Report progress if (i % 1000 == 0) - println(i + "\t" + elapsed / 1000.0 / 1000.0) + println(i.toString + "\t" + elapsed / 1000.0 / 1000.0) totalTime += elapsed latencies(i) = elapsed / 1000 / 1000 } diff --git a/core/src/main/scala/kafka/tools/GetOffsetShell.scala b/core/src/main/scala/kafka/tools/GetOffsetShell.scala index b8b2ad60e37..840053912b6 100644 --- a/core/src/main/scala/kafka/tools/GetOffsetShell.scala +++ b/core/src/main/scala/kafka/tools/GetOffsetShell.scala @@ -27,7 +27,7 @@ import org.apache.kafka.common.{PartitionInfo, TopicPartition} import org.apache.kafka.common.requests.ListOffsetRequest import org.apache.kafka.common.serialization.ByteArrayDeserializer -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.Seq object GetOffsetShell { @@ -143,7 +143,7 @@ object GetOffsetShell { * Return the partition infos for `topic`. If the topic does not exist, `None` is returned. */ private def listPartitionInfos(consumer: KafkaConsumer[_, _], topic: String, partitionIds: Set[Int]): Option[Seq[PartitionInfo]] = { - val partitionInfos = consumer.listTopics.asScala.filterKeys(_ == topic).values.flatMap(_.asScala).toBuffer + val partitionInfos = consumer.listTopics.asScala.filter { case (k, _) => k == topic }.values.flatMap(_.asScala).toBuffer if (partitionInfos.isEmpty) None else if (partitionIds.isEmpty) diff --git a/core/src/main/scala/kafka/tools/JmxTool.scala b/core/src/main/scala/kafka/tools/JmxTool.scala index 7b677019a04..b696a122daa 100644 --- a/core/src/main/scala/kafka/tools/JmxTool.scala +++ b/core/src/main/scala/kafka/tools/JmxTool.scala @@ -26,7 +26,7 @@ import javax.rmi.ssl.SslRMIClientSocketFactory import joptsimple.OptionParser -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable import scala.math._ import kafka.utils.{CommandLineUtils, Exit, Logging} @@ -258,8 +258,8 @@ object JmxTool extends Logging { attributesWhitelist match { case Some(allowedAttributes) => if (allowedAttributes.contains(attr.getName)) - attributes(name + ":" + attr.getName) = attr.getValue - case None => attributes(name + ":" + attr.getName) = attr.getValue + attributes(name.toString + ":" + attr.getName) = attr.getValue + case None => attributes(name.toString + ":" + attr.getName) = attr.getValue } } } diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index 9540357918f..70e3e020e00 100755 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -36,7 +36,7 @@ import org.apache.kafka.common.serialization.{ByteArrayDeserializer, ByteArraySe import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{KafkaException, TopicPartition} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable.HashMap import scala.util.control.ControlThrowable import scala.util.{Failure, Success, Try} @@ -134,7 +134,7 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { case _: TimeoutException => Try(consumerWrapper.consumer.listTopics) match { case Success(visibleTopics) => - consumerWrapper.offsets.retain((tp, _) => visibleTopics.containsKey(tp.topic)) + consumerWrapper.offsets --= consumerWrapper.offsets.keySet.filter(tp => !visibleTopics.containsKey(tp.topic)) case Failure(e) => warn("Failed to list all authorized topics after committing offsets timed out: ", e) } diff --git a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala index 33d5256e5f6..81069f3190e 100644 --- a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala +++ b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala @@ -42,7 +42,7 @@ import org.apache.kafka.common.serialization.StringDeserializer import org.apache.kafka.common.utils.{LogContext, Time} import org.apache.kafka.common.{Node, TopicPartition} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.Seq /** @@ -225,16 +225,16 @@ object ReplicaVerificationTool extends Logging { } private def initialOffsets(topicPartitions: Seq[TopicPartition], consumerConfig: Properties, - initialOffsetTime: Long): Map[TopicPartition, Long] = { + initialOffsetTime: Long): collection.Map[TopicPartition, Long] = { val consumer = createConsumer(consumerConfig) try { if (ListOffsetRequest.LATEST_TIMESTAMP == initialOffsetTime) - consumer.endOffsets(topicPartitions.asJava).asScala.mapValues(_.longValue).toMap + consumer.endOffsets(topicPartitions.asJava).asScala.map { case (k, v) => k -> v.longValue } else if (ListOffsetRequest.EARLIEST_TIMESTAMP == initialOffsetTime) - consumer.beginningOffsets(topicPartitions.asJava).asScala.mapValues(_.longValue).toMap + consumer.beginningOffsets(topicPartitions.asJava).asScala.map { case (k, v) => k -> v.longValue } else { val timestampsToSearch = topicPartitions.map(tp => tp -> (initialOffsetTime: java.lang.Long)).toMap - consumer.offsetsForTimes(timestampsToSearch.asJava).asScala.mapValues(v => v.offset).toMap + consumer.offsetsForTimes(timestampsToSearch.asJava).asScala.map { case (k, v) => k -> v.offset } } } finally consumer.close() } @@ -256,8 +256,8 @@ private case class TopicPartitionReplica(topic: String, partitionId: Int, replic private case class MessageInfo(replicaId: Int, offset: Long, nextOffset: Long, checksum: Long) -private class ReplicaBuffer(expectedReplicasPerTopicPartition: Map[TopicPartition, Int], - initialOffsets: Map[TopicPartition, Long], +private class ReplicaBuffer(expectedReplicasPerTopicPartition: collection.Map[TopicPartition, Int], + initialOffsets: collection.Map[TopicPartition, Long], expectedNumFetchers: Int, reportInterval: Long) extends Logging { private val fetchOffsetMap = new Pool[TopicPartition, Long] @@ -358,8 +358,8 @@ private class ReplicaBuffer(expectedReplicasPerTopicPartition: Map[TopicPartitio if (isMessageInAllReplicas) { val nextOffset = messageInfoFromFirstReplicaOpt.get.nextOffset fetchOffsetMap.put(topicPartition, nextOffset) - debug(expectedReplicasPerTopicPartition(topicPartition) + " replicas match at offset " + - nextOffset + " for " + topicPartition) + debug(s"${expectedReplicasPerTopicPartition(topicPartition)} replicas match at offset " + + s"$nextOffset for $topicPartition") } } if (maxHw - fetchOffsetMap.get(topicPartition) > maxLag) { diff --git a/core/src/main/scala/kafka/utils/CoreUtils.scala b/core/src/main/scala/kafka/utils/CoreUtils.scala index 616b43b2a7a..0bfdf03a521 100755 --- a/core/src/main/scala/kafka/utils/CoreUtils.scala +++ b/core/src/main/scala/kafka/utils/CoreUtils.scala @@ -240,7 +240,7 @@ object CoreUtils { /** * Returns a list of duplicated items */ - def duplicates[T](s: Traversable[T]): Iterable[T] = { + def duplicates[T](s: Iterable[T]): Iterable[T] = { s.groupBy(identity) .map { case (k, l) => (k, l.size)} .filter { case (_, l) => l > 1 } diff --git a/core/src/main/scala/kafka/utils/Implicits.scala b/core/src/main/scala/kafka/utils/Implicits.scala index 5196d45ea91..47ab9c022bb 100644 --- a/core/src/main/scala/kafka/utils/Implicits.scala +++ b/core/src/main/scala/kafka/utils/Implicits.scala @@ -20,7 +20,7 @@ package kafka.utils import java.util import java.util.Properties -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** * In order to have these implicits in scope, add the following import: diff --git a/core/src/main/scala/kafka/utils/Json.scala b/core/src/main/scala/kafka/utils/Json.scala index c61e1490448..3fafeedaec9 100644 --- a/core/src/main/scala/kafka/utils/Json.scala +++ b/core/src/main/scala/kafka/utils/Json.scala @@ -63,7 +63,7 @@ object Json { catch { case _: JsonProcessingException => None } def tryParseBytes(input: Array[Byte]): Either[JsonProcessingException, JsonValue] = - try Right(mapper.readTree(input)).right.map(JsonValue(_)) + try Right(mapper.readTree(input)).map(JsonValue(_)) catch { case e: JsonProcessingException => Left(e) } /** diff --git a/core/src/main/scala/kafka/utils/Log4jController.scala b/core/src/main/scala/kafka/utils/Log4jController.scala index ba0649c0ca2..4bd1fb25f40 100755 --- a/core/src/main/scala/kafka/utils/Log4jController.scala +++ b/core/src/main/scala/kafka/utils/Log4jController.scala @@ -23,7 +23,7 @@ import java.util.Locale import org.apache.log4j.{Level, LogManager, Logger} import scala.collection.mutable -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ object Log4jController { diff --git a/core/src/main/scala/kafka/utils/Pool.scala b/core/src/main/scala/kafka/utils/Pool.scala index 2f24aff0c4c..964de7eae26 100644 --- a/core/src/main/scala/kafka/utils/Pool.scala +++ b/core/src/main/scala/kafka/utils/Pool.scala @@ -21,8 +21,8 @@ import java.util.concurrent._ import org.apache.kafka.common.KafkaException -import collection.mutable -import collection.JavaConverters._ +import collection.Set +import scala.jdk.CollectionConverters._ class Pool[K,V](valueFactory: Option[K => V] = None) extends Iterable[(K, V)] { @@ -69,7 +69,7 @@ class Pool[K,V](valueFactory: Option[K => V] = None) extends Iterable[(K, V)] { def remove(key: K, value: V): Boolean = pool.remove(key, value) - def keys: mutable.Set[K] = pool.keySet.asScala + def keys: Set[K] = pool.keySet.asScala def values: Iterable[V] = pool.values.asScala diff --git a/core/src/main/scala/kafka/utils/VerifiableProperties.scala b/core/src/main/scala/kafka/utils/VerifiableProperties.scala index 8a0f33837c1..bbacd95dc3b 100755 --- a/core/src/main/scala/kafka/utils/VerifiableProperties.scala +++ b/core/src/main/scala/kafka/utils/VerifiableProperties.scala @@ -21,7 +21,7 @@ import java.util.Properties import java.util.Collections import scala.collection._ import kafka.message.{CompressionCodec, NoCompressionCodec} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class VerifiableProperties(val props: Properties) extends Logging { diff --git a/core/src/main/scala/kafka/utils/json/DecodeJson.scala b/core/src/main/scala/kafka/utils/json/DecodeJson.scala index d1b4355a611..7f00c366af4 100644 --- a/core/src/main/scala/kafka/utils/json/DecodeJson.scala +++ b/core/src/main/scala/kafka/utils/json/DecodeJson.scala @@ -19,7 +19,7 @@ package kafka.utils.json import scala.collection.{Map, Seq} import scala.collection.compat._ -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import com.fasterxml.jackson.databind.{JsonMappingException, JsonNode} @@ -82,7 +82,7 @@ object DecodeJson { implicit def decodeOption[E](implicit decodeJson: DecodeJson[E]): DecodeJson[Option[E]] = (node: JsonNode) => { if (node.isNull) Right(None) - else decodeJson.decodeEither(node).right.map(Some(_)) + else decodeJson.decodeEither(node).map(Some(_)) } implicit def decodeSeq[E, S[+T] <: Seq[E]](implicit decodeJson: DecodeJson[E], factory: Factory[E, S[E]]): DecodeJson[S[E]] = (node: JsonNode) => { @@ -93,7 +93,7 @@ object DecodeJson { implicit def decodeMap[V, M[K, +V] <: Map[K, V]](implicit decodeJson: DecodeJson[V], factory: Factory[(String, V), M[String, V]]): DecodeJson[M[String, V]] = (node: JsonNode) => { if (node.isObject) - decodeIterator(node.fields.asScala)(e => decodeJson.decodeEither(e.getValue).right.map(v => (e.getKey, v))) + decodeIterator(node.fields.asScala)(e => decodeJson.decodeEither(e.getValue).map(v => (e.getKey, v))) else Left(s"Expected JSON object, received $node") } diff --git a/core/src/main/scala/kafka/utils/json/JsonArray.scala b/core/src/main/scala/kafka/utils/json/JsonArray.scala index 0b81a4ae138..c22eda8651d 100644 --- a/core/src/main/scala/kafka/utils/json/JsonArray.scala +++ b/core/src/main/scala/kafka/utils/json/JsonArray.scala @@ -17,8 +17,8 @@ package kafka.utils.json -import scala.collection.{Iterator, JavaConverters} -import JavaConverters._ +import scala.collection.Iterator +import scala.jdk.CollectionConverters._ import com.fasterxml.jackson.databind.node.ArrayNode diff --git a/core/src/main/scala/kafka/utils/json/JsonObject.scala b/core/src/main/scala/kafka/utils/json/JsonObject.scala index 9b48cb8a77a..9bf91ae1a6b 100644 --- a/core/src/main/scala/kafka/utils/json/JsonObject.scala +++ b/core/src/main/scala/kafka/utils/json/JsonObject.scala @@ -19,7 +19,7 @@ package kafka.utils.json import com.fasterxml.jackson.databind.JsonMappingException -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import com.fasterxml.jackson.databind.node.ObjectNode diff --git a/core/src/main/scala/kafka/zk/AdminZkClient.scala b/core/src/main/scala/kafka/zk/AdminZkClient.scala index 10b10c6ec27..9e62405b456 100644 --- a/core/src/main/scala/kafka/zk/AdminZkClient.scala +++ b/core/src/main/scala/kafka/zk/AdminZkClient.scala @@ -93,7 +93,8 @@ class AdminZkClient(zkClient: KafkaZkClient) extends Logging { zkClient.setOrCreateEntityConfigs(ConfigType.Topic, topic, config) // create the partition assignment - writeTopicPartitionAssignment(topic, partitionReplicaAssignment.mapValues(ReplicaAssignment(_)).toMap, isUpdate = false) + writeTopicPartitionAssignment(topic, partitionReplicaAssignment.map { case (k, v) => k -> ReplicaAssignment(v) }, + isUpdate = false) } /** @@ -140,7 +141,7 @@ class AdminZkClient(zkClient: KafkaZkClient) extends Logging { val assignment = replicaAssignment.map { case (partitionId, replicas) => (new TopicPartition(topic,partitionId), replicas) }.toMap if (!isUpdate) { - zkClient.createTopicAssignment(topic, assignment.mapValues(_.replicas).toMap) + zkClient.createTopicAssignment(topic, assignment.map { case (k, v) => k -> v.replicas }) } else { zkClient.setTopicAssignment(topic, assignment) } @@ -218,7 +219,7 @@ class AdminZkClient(zkClient: KafkaZkClient) extends Logging { writeTopicPartitionAssignment(topic, proposedAssignment, isUpdate = true) } - proposedAssignment.mapValues(_.replicas).toMap + proposedAssignment.map { case (k, v) => k -> v.replicas } } private def validateReplicaAssignment(replicaAssignment: Map[Int, Seq[Int]], diff --git a/core/src/main/scala/kafka/zk/KafkaZkClient.scala b/core/src/main/scala/kafka/zk/KafkaZkClient.scala index 36293504c6e..b10ddbe2757 100644 --- a/core/src/main/scala/kafka/zk/KafkaZkClient.scala +++ b/core/src/main/scala/kafka/zk/KafkaZkClient.scala @@ -514,8 +514,8 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo * @param assignment the partition to replica mapping to set for the given topic * @throws KeeperException if there is an error while creating assignment */ - def createTopicAssignment(topic: String, assignment: Map[TopicPartition, Seq[Int]]) = { - val persistedAssignments = assignment.mapValues(ReplicaAssignment(_)).toMap + def createTopicAssignment(topic: String, assignment: Map[TopicPartition, Seq[Int]]): Unit = { + val persistedAssignments = assignment.map { case (k, v) => k -> ReplicaAssignment(v) } createRecursive(TopicZNode.path(topic), TopicZNode.encode(persistedAssignments)) } @@ -583,7 +583,7 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo * @return the replica assignment for each partition from the given topics. */ def getReplicaAssignmentForTopics(topics: Set[String]): Map[TopicPartition, Seq[Int]] = { - getFullReplicaAssignmentForTopics(topics).mapValues(_.replicas).toMap + getFullReplicaAssignmentForTopics(topics).map { case (k, v) => k -> v.replicas } } /** diff --git a/core/src/main/scala/kafka/zk/ZkData.scala b/core/src/main/scala/kafka/zk/ZkData.scala index 439f7902ec2..5ced14f89e4 100644 --- a/core/src/main/scala/kafka/zk/ZkData.scala +++ b/core/src/main/scala/kafka/zk/ZkData.scala @@ -42,7 +42,7 @@ import org.apache.zookeeper.ZooDefs import org.apache.zookeeper.data.{ACL, Stat} import scala.beans.BeanProperty -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable.ArrayBuffer import scala.collection.{Map, Seq, mutable} import scala.util.{Failure, Success, Try} @@ -441,7 +441,7 @@ object ReassignPartitionsZNode { } def decode(bytes: Array[Byte]): Either[JsonProcessingException, collection.Map[TopicPartition, Seq[Int]]] = - Json.parseBytesAs[LegacyPartitionAssignment](bytes).right.map { partitionAssignment => + Json.parseBytesAs[LegacyPartitionAssignment](bytes).map { partitionAssignment => partitionAssignment.partitions.asScala.iterator.map { replicaAssignment => new TopicPartition(replicaAssignment.topic, replicaAssignment.partition) -> replicaAssignment.replicas.asScala }.toMap @@ -624,7 +624,7 @@ case object LiteralAclChangeStore extends ZkAclChangeStore { if (resource.patternType != PatternType.LITERAL) throw new IllegalArgumentException("Only literal resource patterns can be encoded") - val legacyName = resource.resourceType + AclEntry.ResourceSeparator + resource.name + val legacyName = resource.resourceType.toString + AclEntry.ResourceSeparator + resource.name legacyName.getBytes(UTF_8) } diff --git a/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala b/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala index c0f8d13fa2f..02a533d23a8 100755 --- a/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala +++ b/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala @@ -35,7 +35,7 @@ import org.apache.zookeeper.data.{ACL, Stat} import org.apache.zookeeper._ import org.apache.zookeeper.client.ZKClientConfig -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.Seq import scala.collection.mutable.Set diff --git a/core/src/main/scala/org/apache/zookeeper/ZooKeeperMainWithTlsSupportForKafka.scala b/core/src/main/scala/org/apache/zookeeper/ZooKeeperMainWithTlsSupportForKafka.scala index 6278d0ef05c..eb29361b6ef 100644 --- a/core/src/main/scala/org/apache/zookeeper/ZooKeeperMainWithTlsSupportForKafka.scala +++ b/core/src/main/scala/org/apache/zookeeper/ZooKeeperMainWithTlsSupportForKafka.scala @@ -23,7 +23,7 @@ import org.apache.zookeeper.cli.CommandNotFoundException import org.apache.zookeeper.cli.MalformedCommandException import org.apache.zookeeper.client.ZKClientConfig -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ object ZooKeeperMainWithTlsSupportForKafka { val zkTlsConfigFileOption = "-zk-tls-config-file" @@ -70,7 +70,7 @@ class ZooKeeperMainWithTlsSupportForKafka(args: Array[String], val zkClientConfi def kafkaTlsUsage(): Unit = { System.err.println("ZooKeeper -server host:port [-zk-tls-config-file ] cmd args") - asScalaSet(ZooKeeperMain.commandMap.keySet).toList.sorted.foreach(cmd => + ZooKeeperMain.commandMap.keySet.asScala.toList.sorted.foreach(cmd => System.err.println(s"\t$cmd ${ZooKeeperMain.commandMap.get(cmd)}")) } diff --git a/core/src/test/scala/integration/kafka/admin/ReassignPartitionsIntegrationTest.scala b/core/src/test/scala/integration/kafka/admin/ReassignPartitionsIntegrationTest.scala index 0993bfe5f57..717a272b60c 100644 --- a/core/src/test/scala/integration/kafka/admin/ReassignPartitionsIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/admin/ReassignPartitionsIntegrationTest.scala @@ -36,7 +36,7 @@ import org.junit.Assert.{assertEquals, assertFalse, assertTrue} import org.junit.{After, Rule, Test} import scala.collection.Map -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.{Seq, mutable} class ReassignPartitionsIntegrationTest extends ZooKeeperTestHarness { diff --git a/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala b/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala index fcd1402126c..aa31173dfdf 100644 --- a/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala @@ -29,7 +29,7 @@ import kafka.server.{BaseRequestTest, KafkaConfig} import org.junit.Assert._ import org.junit.Before -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable.{ArrayBuffer, Buffer} import org.apache.kafka.clients.producer.KafkaProducer import org.apache.kafka.common.errors.WakeupException @@ -323,11 +323,11 @@ abstract class AbstractConsumerTest extends BaseRequestTest { partitionsToAssign: Set[TopicPartition]) extends ShutdownableThread("daemon-consumer-assignment", false) { - def this(consumer: Consumer[Array[Byte], Array[Byte]], topicsToSubscribe: List[String]) { + def this(consumer: Consumer[Array[Byte], Array[Byte]], topicsToSubscribe: List[String]) = { this(consumer, topicsToSubscribe, Set.empty[TopicPartition]) } - def this(consumer: Consumer[Array[Byte], Array[Byte]], partitionsToAssign: Set[TopicPartition]) { + def this(consumer: Consumer[Array[Byte], Array[Byte]], partitionsToAssign: Set[TopicPartition]) = { this(consumer, List.empty[String], partitionsToAssign) } diff --git a/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala index 4bf4fc26932..33260031b4c 100644 --- a/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala @@ -31,7 +31,7 @@ import org.junit.{After, Before, Rule, Test} import org.junit.rules.Timeout import org.scalatest.Assertions.intercept -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** * Tests AdminClient calls when the broker is configured with policies like AlterConfigPolicy, CreateTopicPolicy, etc. diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index b932f462b43..1b57cff1fb5 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -57,7 +57,7 @@ import org.junit.Assert._ import org.junit.{After, Before, Test} import org.scalatest.Assertions.intercept -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable import scala.collection.mutable.Buffer diff --git a/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala index 77dcb6e202c..685bb6153e2 100644 --- a/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala @@ -33,7 +33,7 @@ import org.junit.Assert._ import org.junit.rules.Timeout import org.junit.{After, Before, Rule, Test} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.Seq import scala.compat.java8.OptionConverters._ diff --git a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala index 2b0700b38f8..e458b59f91a 100644 --- a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala @@ -22,7 +22,7 @@ import org.apache.kafka.common.internals.Topic import org.junit.Test import org.junit.Assert._ -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.Seq /** diff --git a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala index f1696115769..9797e098e2f 100644 --- a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala @@ -36,7 +36,7 @@ import org.junit.Assert._ import org.junit.{After, Before, Test} import org.scalatest.Assertions.fail -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable.Buffer import scala.concurrent.ExecutionException diff --git a/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala b/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala index 4d9d23e70f9..60d48df6dfd 100644 --- a/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala @@ -30,7 +30,7 @@ import org.apache.kafka.common.security.auth.KafkaPrincipal import org.junit.Assert._ import org.junit.{Before, Test} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ abstract class BaseQuotaTest extends IntegrationTestHarness { @@ -314,10 +314,10 @@ abstract class QuotaTestClients(topic: String, val overrideProducerRequestQuota = quota(quotaManagers.request, userPrincipal, producerClientId) val overrideConsumerRequestQuota = quota(quotaManagers.request, userPrincipal, consumerClientId) - assertEquals(s"ClientId $producerClientId of user $userPrincipal must have producer quota", Quota.upperBound(producerQuota), overrideProducerQuota) - assertEquals(s"ClientId $consumerClientId of user $userPrincipal must have consumer quota", Quota.upperBound(consumerQuota), overrideConsumerQuota) - assertEquals(s"ClientId $producerClientId of user $userPrincipal must have request quota", Quota.upperBound(requestQuota), overrideProducerRequestQuota) - assertEquals(s"ClientId $consumerClientId of user $userPrincipal must have request quota", Quota.upperBound(requestQuota), overrideConsumerRequestQuota) + assertEquals(s"ClientId $producerClientId of user $userPrincipal must have producer quota", Quota.upperBound(producerQuota.toDouble), overrideProducerQuota) + assertEquals(s"ClientId $consumerClientId of user $userPrincipal must have consumer quota", Quota.upperBound(consumerQuota.toDouble), overrideConsumerQuota) + assertEquals(s"ClientId $producerClientId of user $userPrincipal must have request quota", Quota.upperBound(requestQuota.toDouble), overrideProducerRequestQuota) + assertEquals(s"ClientId $consumerClientId of user $userPrincipal must have request quota", Quota.upperBound(requestQuota.toDouble), overrideConsumerRequestQuota) } } } diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala index 337b71458cc..24942fa3002 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala @@ -29,7 +29,7 @@ import org.apache.kafka.common.requests.{FindCoordinatorRequest, FindCoordinator import org.junit.Assert._ import org.junit.{After, Ignore, Test} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.{Seq, mutable} /** diff --git a/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala b/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala index 091a17852eb..1fdee6525e9 100644 --- a/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala +++ b/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala @@ -39,7 +39,7 @@ import org.junit.Assert._ import org.junit.{After, Before, Test} import scala.collection.mutable.ArrayBuffer -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class CustomQuotaCallbackTest extends IntegrationTestHarness with SaslSetup { @@ -159,7 +159,7 @@ class CustomQuotaCallbackTest extends IntegrationTestHarness with SaslSetup { val newProps = new Properties newProps.put(GroupedUserQuotaCallback.DefaultProduceQuotaProp, "8000") newProps.put(GroupedUserQuotaCallback.DefaultFetchQuotaProp, "2500") - TestUtils.alterConfigs(servers, adminClient, newProps, perBrokerConfig = false) + TestUtils.incrementalAlterConfigs(servers, adminClient, newProps, perBrokerConfig = false) user.waitForQuotaUpdate(8000, 2500, defaultRequestQuota) user.produceConsume(expectProduceThrottle = true, expectConsumeThrottle = true) @@ -365,8 +365,8 @@ class GroupedUserQuotaCallback extends ClientQuotaCallback with Reconfigurable w } override def reconfigure(configs: util.Map[String, _]): Unit = { - configValue(configs, DefaultProduceQuotaProp).foreach(value => quotas(ClientQuotaType.PRODUCE).put("", value)) - configValue(configs, DefaultFetchQuotaProp).foreach(value => quotas(ClientQuotaType.FETCH).put("", value)) + configValue(configs, DefaultProduceQuotaProp).foreach(value => quotas(ClientQuotaType.PRODUCE).put("", value.toDouble)) + configValue(configs, DefaultFetchQuotaProp).foreach(value => quotas(ClientQuotaType.FETCH).put("", value.toDouble)) customQuotasUpdated.values.foreach(_.set(true)) } diff --git a/core/src/test/scala/integration/kafka/api/DelegationTokenEndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/DelegationTokenEndToEndAuthorizationTest.scala index d9b6ed6ef18..c9ef92a732f 100644 --- a/core/src/test/scala/integration/kafka/api/DelegationTokenEndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/DelegationTokenEndToEndAuthorizationTest.scala @@ -29,7 +29,7 @@ import org.apache.kafka.common.security.scram.internals.ScramMechanism import org.apache.kafka.common.security.token.delegation.DelegationToken import org.junit.Before -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class DelegationTokenEndToEndAuthorizationTest extends EndToEndAuthorizationTest { diff --git a/core/src/test/scala/integration/kafka/api/DescribeAuthorizedOperationsTest.scala b/core/src/test/scala/integration/kafka/api/DescribeAuthorizedOperationsTest.scala index 55b9b897d66..3be1865b873 100644 --- a/core/src/test/scala/integration/kafka/api/DescribeAuthorizedOperationsTest.scala +++ b/core/src/test/scala/integration/kafka/api/DescribeAuthorizedOperationsTest.scala @@ -30,7 +30,7 @@ import org.apache.kafka.server.authorizer.Authorizer import org.junit.Assert.{assertEquals, assertFalse, assertNull} import org.junit.{After, Before, Test} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class DescribeAuthorizedOperationsTest extends IntegrationTestHarness with SaslSetup { override val brokerCount = 1 diff --git a/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala index 6c82fa9ce54..948a43d0cb0 100644 --- a/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala @@ -43,7 +43,7 @@ import org.junit.Assert._ import org.junit.{After, Before, Test} import org.scalatest.Assertions.{assertThrows, fail, intercept} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** * The test cases here verify that a producer authorized to publish to a topic @@ -233,7 +233,7 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas private def getGauge(metricName: String) = { KafkaYammerMetrics.defaultRegistry.allMetrics.asScala - .filterKeys(k => k.getName == metricName) + .filter { case (k, _) => k.getName == metricName } .headOption .getOrElse { fail( "Unable to find metric " + metricName ) } ._2.asInstanceOf[Gauge[Double]] diff --git a/core/src/test/scala/integration/kafka/api/EndToEndClusterIdTest.scala b/core/src/test/scala/integration/kafka/api/EndToEndClusterIdTest.scala index 072c44d1a4d..89cbf338a36 100644 --- a/core/src/test/scala/integration/kafka/api/EndToEndClusterIdTest.scala +++ b/core/src/test/scala/integration/kafka/api/EndToEndClusterIdTest.scala @@ -32,7 +32,7 @@ import org.apache.kafka.test.{TestUtils => _, _} import org.junit.Assert._ import org.junit.{Before, Test} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import org.apache.kafka.test.TestUtils.isValidClusterId /** The test cases here verify the following conditions. diff --git a/core/src/test/scala/integration/kafka/api/GroupAuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/GroupAuthorizerIntegrationTest.scala index 4a4898996f4..8f52102cce6 100644 --- a/core/src/test/scala/integration/kafka/api/GroupAuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/GroupAuthorizerIntegrationTest.scala @@ -33,7 +33,7 @@ import org.junit.Assert._ import org.junit.{Before, Test} import org.scalatest.Assertions.intercept -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ object GroupAuthorizerIntegrationTest { val BrokerPrincipal = new KafkaPrincipal("Group", "broker") @@ -136,4 +136,4 @@ class GroupAuthorizerIntegrationTest extends BaseRequestTest { TestUtils.pollUntilAtLeastNumRecords(consumer, numRecords = 1) } -} \ No newline at end of file +} diff --git a/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala index e29867b8869..af846283d81 100644 --- a/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala @@ -21,7 +21,7 @@ import org.apache.kafka.common.TopicPartition import org.junit.Test import org.junit.Assert._ -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import java.util.Properties import org.apache.kafka.common.internals.Topic diff --git a/core/src/test/scala/integration/kafka/api/MetricsTest.scala b/core/src/test/scala/integration/kafka/api/MetricsTest.scala index a851b4ee02d..5e92eca7afc 100644 --- a/core/src/test/scala/integration/kafka/api/MetricsTest.scala +++ b/core/src/test/scala/integration/kafka/api/MetricsTest.scala @@ -30,7 +30,7 @@ import org.junit.{After, Before, Test} import org.junit.Assert._ import org.scalatest.Assertions.fail -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class MetricsTest extends IntegrationTestHarness with SaslSetup { diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index e1c20e1324d..6cb33fa42f7 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -45,7 +45,7 @@ import org.junit.Assert._ import org.junit.{After, Before, Ignore, Test} import org.scalatest.Assertions.intercept -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.Seq import scala.concurrent.duration.Duration import scala.concurrent.{Await, Future} @@ -171,7 +171,9 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { client = Admin.create(createConfig()) val topic = "topic" val leaderByPartition = createTopic(topic, numPartitions = 10, replicationFactor = 1) - val partitionsByBroker = leaderByPartition.groupBy { case (_, leaderId) => leaderId }.mapValues(_.keys.toSeq) + val partitionsByBroker = leaderByPartition.groupBy { case (_, leaderId) => leaderId }.map { case (k, v) => + k -> v.keys.toSeq + } val brokers = (0 until brokerCount).map(Integer.valueOf) val logDirInfosByBroker = client.describeLogDirs(brokers.asJava).all.get @@ -179,7 +181,9 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { val server = servers.find(_.config.brokerId == brokerId).get val expectedPartitions = partitionsByBroker(brokerId) val logDirInfos = logDirInfosByBroker.get(brokerId) - val replicaInfos = logDirInfos.asScala.flatMap { case (_, logDirInfo) => logDirInfo.replicaInfos.asScala }.filterKeys(_.topic == topic) + val replicaInfos = logDirInfos.asScala.flatMap { case (_, logDirInfo) => + logDirInfo.replicaInfos.asScala + }.filter { case (k, _) => k.topic == topic } assertEquals(expectedPartitions.toSet, replicaInfos.keys.map(_.partition).toSet) logDirInfos.asScala.foreach { case (logDir, logDirInfo) => diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index 9df468a34c5..0b3390e1d31 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -33,7 +33,7 @@ import org.junit.Assert._ import org.junit.Test import org.scalatest.Assertions.intercept -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable.Buffer import kafka.server.QuotaType import kafka.server.KafkaServer @@ -579,7 +579,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertTrue(consumer.poll(Duration.ofMillis(50)).isEmpty) consumer.seekToBeginning(List(tp).asJava) - assertEquals(0, consumer.position(tp), 0) + assertEquals(0L, consumer.position(tp)) consumeAndVerifyRecords(consumer, numRecords = 1, startingOffset = 0) consumer.seek(tp, mid) @@ -597,7 +597,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertTrue(consumer.poll(Duration.ofMillis(50)).isEmpty) consumer.seekToBeginning(List(tp2).asJava) - assertEquals(0, consumer.position(tp2), 0) + assertEquals(0L, consumer.position(tp2)) consumeAndVerifyRecords(consumer, numRecords = 1, startingOffset = 0, tp = tp2) consumer.seek(tp2, mid) @@ -1511,7 +1511,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { consumer.assign(List(tp2).asJava) awaitNonEmptyRecords(consumer, tp2) - assertNull(consumer.metrics.get(new MetricName(tp + ".records-lag", "consumer-fetch-manager-metrics", "", tags))) + assertNull(consumer.metrics.get(new MetricName(tp.toString + ".records-lag", "consumer-fetch-manager-metrics", "", tags))) assertNull(consumer.metrics.get(new MetricName("records-lag", "consumer-fetch-manager-metrics", "", tags))) } diff --git a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala index d65c43e71b5..a8788a44724 100755 --- a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala @@ -19,7 +19,7 @@ package kafka.api.test import java.util.{Collection, Collections, Properties} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import org.junit.runners.Parameterized import org.junit.runner.RunWith import org.junit.runners.Parameterized.Parameters diff --git a/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala b/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala index 2dcf489187c..03c5f48493d 100644 --- a/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala @@ -17,7 +17,7 @@ import java.time.Duration import java.util.Collections import java.util.concurrent.{ExecutionException, TimeUnit} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import org.apache.kafka.clients.admin.{Admin, AdminClientConfig} import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} diff --git a/core/src/test/scala/integration/kafka/api/SaslEndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/SaslEndToEndAuthorizationTest.scala index a1b867ef753..fa9f1ca47fd 100644 --- a/core/src/test/scala/integration/kafka/api/SaslEndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslEndToEndAuthorizationTest.scala @@ -24,7 +24,7 @@ import org.junit.Assert.{assertEquals, assertTrue} import org.scalatest.Assertions.fail import scala.collection.immutable.List -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ abstract class SaslEndToEndAuthorizationTest extends EndToEndAuthorizationTest { override protected def securityProtocol = SecurityProtocol.SASL_SSL diff --git a/core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala index b24937a0394..3f30f131452 100644 --- a/core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala @@ -19,7 +19,7 @@ import org.junit.{After, Before, Test} import kafka.utils.JaasTestUtils import org.apache.kafka.common.security.auth.SecurityProtocol -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class SaslMultiMechanismConsumerTest extends BaseConsumerTest with SaslSetup { private val kafkaClientSaslMechanism = "PLAIN" diff --git a/core/src/test/scala/integration/kafka/api/SaslScramSslEndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/SaslScramSslEndToEndAuthorizationTest.scala index 17a2e0eebb9..f4e27cc90b9 100644 --- a/core/src/test/scala/integration/kafka/api/SaslScramSslEndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslScramSslEndToEndAuthorizationTest.scala @@ -21,7 +21,7 @@ import kafka.zk.ConfigEntityChangeNotificationZNode import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.security.scram.internals.ScramMechanism -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import org.junit.Before class SaslScramSslEndToEndAuthorizationTest extends SaslEndToEndAuthorizationTest { diff --git a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala index 7d698e71a0b..c0319f0e3b2 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala @@ -30,7 +30,7 @@ import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.junit.Assert.{assertEquals, assertTrue} import org.junit.{After, Assert, Before, Test} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.Seq import scala.compat.java8.OptionConverters._ import scala.concurrent.ExecutionException diff --git a/core/src/test/scala/integration/kafka/api/SslAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SslAdminIntegrationTest.scala index 9b1a645a4c9..7ef7edf7909 100644 --- a/core/src/test/scala/integration/kafka/api/SslAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SslAdminIntegrationTest.scala @@ -36,7 +36,7 @@ import org.apache.kafka.server.authorizer._ import org.junit.Assert.{assertEquals, assertFalse, assertNotNull, assertTrue} import org.junit.{Assert, Test} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable object SslAdminIntegrationTest { diff --git a/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala index 861f6e1bb31..e97b878e95f 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala @@ -29,7 +29,7 @@ import org.apache.kafka.common.TopicPartition import org.junit.Assert._ import org.junit.Test -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable class TransactionsBounceTest extends KafkaServerTestHarness { diff --git a/core/src/test/scala/integration/kafka/api/TransactionsExpirationTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsExpirationTest.scala index 04922866d03..9251750081b 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsExpirationTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsExpirationTest.scala @@ -28,7 +28,7 @@ import org.apache.kafka.clients.producer.KafkaProducer import org.apache.kafka.common.errors.InvalidPidMappingException import org.junit.{After, Before, Test} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.Seq // Test class that uses a very small transaction timeout to trigger InvalidPidMapping errors diff --git a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala index 7b2a945f6e9..25bd020c1a1 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala @@ -35,7 +35,7 @@ import org.junit.Assert._ import org.junit.{After, Before, Test} import org.scalatest.Assertions.fail -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.Seq import scala.collection.mutable.Buffer import scala.concurrent.ExecutionException diff --git a/core/src/test/scala/integration/kafka/network/DynamicConnectionQuotaTest.scala b/core/src/test/scala/integration/kafka/network/DynamicConnectionQuotaTest.scala index a0807fd2a35..79401599780 100644 --- a/core/src/test/scala/integration/kafka/network/DynamicConnectionQuotaTest.scala +++ b/core/src/test/scala/integration/kafka/network/DynamicConnectionQuotaTest.scala @@ -36,7 +36,7 @@ import org.junit.Assert._ import org.junit.{After, Before, Test} import org.scalatest.Assertions.intercept -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class DynamicConnectionQuotaTest extends BaseRequestTest { @@ -166,7 +166,7 @@ class DynamicConnectionQuotaTest extends BaseRequestTest { private def reconfigureServers(newProps: Properties, perBrokerConfig: Boolean, aPropToVerify: (String, String)): Unit = { val initialConnectionCount = connectionCount val adminClient = createAdminClient() - TestUtils.alterConfigs(servers, adminClient, newProps, perBrokerConfig).all.get() + TestUtils.incrementalAlterConfigs(servers, adminClient, newProps, perBrokerConfig).all.get() waitForConfigOnServer(aPropToVerify._1, aPropToVerify._2) adminClient.close() TestUtils.waitUntilTrue(() => initialConnectionCount == connectionCount, "Admin client connection not closed") diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala index 4705b0e47a1..12548a1c742 100644 --- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala +++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala @@ -65,7 +65,7 @@ import org.scalatest.Assertions.intercept import scala.collection._ import scala.collection.mutable.ArrayBuffer -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.Seq object DynamicBrokerReconfigurationTest { @@ -996,7 +996,7 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet val config = server.config val oldSecret = "old-dynamic-config-secret" config.dynamicConfig.staticBrokerConfigs.put(KafkaConfig.PasswordEncoderOldSecretProp, oldSecret) - val passwordConfigs = props.asScala.filterKeys(DynamicBrokerConfig.isPasswordConfig) + val passwordConfigs = props.asScala.filter { case (k, _) => DynamicBrokerConfig.isPasswordConfig(k) } assertTrue("Password configs not found", passwordConfigs.nonEmpty) val passwordDecoder = createPasswordEncoder(config, config.passwordEncoderSecret) val passwordEncoder = createPasswordEncoder(config, Some(new Password(oldSecret))) @@ -1134,7 +1134,7 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet .map(e => s"${e.listenerName.value}://${e.host}:${e.port}") .mkString(",") val listenerMap = config.listenerSecurityProtocolMap - .filterKeys(listenerName => listenerName.value != securityProtocol.name) + .filter { case (listenerName, _) => listenerName.value != securityProtocol.name } .map { case (listenerName, protocol) => s"${listenerName.value}:${protocol.name}" } .mkString(",") @@ -1358,8 +1358,23 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet props.asScala.foreach { case (k, v) => waitForConfigOnServer(server, k, v) } } + private def alterConfigs(servers: Seq[KafkaServer], adminClient: Admin, props: Properties, + perBrokerConfig: Boolean): AlterConfigsResult = { + val configEntries = props.asScala.map { case (k, v) => new ConfigEntry(k, v) }.toList.asJava + val newConfig = new Config(configEntries) + val configs = if (perBrokerConfig) { + servers.map { server => + val resource = new ConfigResource(ConfigResource.Type.BROKER, server.config.brokerId.toString) + (resource, newConfig) + }.toMap.asJava + } else { + Map(new ConfigResource(ConfigResource.Type.BROKER, "") -> newConfig).asJava + } + adminClient.alterConfigs(configs) + } + private def reconfigureServers(newProps: Properties, perBrokerConfig: Boolean, aPropToVerify: (String, String), expectFailure: Boolean = false): Unit = { - val alterResult = TestUtils.alterConfigs(servers, adminClients.head, newProps, perBrokerConfig) + val alterResult = alterConfigs(servers, adminClients.head, newProps, perBrokerConfig) if (expectFailure) { val oldProps = servers.head.config.values.asScala.filter { case (k, _) => newProps.containsKey(k) } val brokerResources = if (perBrokerConfig) diff --git a/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala b/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala index 63dc456c3ff..d6414c01c0a 100644 --- a/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala +++ b/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala @@ -36,7 +36,7 @@ import org.apache.kafka.common.utils.{LogContext, MockTime} import org.junit.Assert._ import org.junit.{After, Before, Test} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class GssapiAuthenticationTest extends IntegrationTestHarness with SaslSetup { override val brokerCount = 1 diff --git a/core/src/test/scala/integration/kafka/server/ScramServerStartupTest.scala b/core/src/test/scala/integration/kafka/server/ScramServerStartupTest.scala index 70c30ee7dcb..15bbd54d0cf 100644 --- a/core/src/test/scala/integration/kafka/server/ScramServerStartupTest.scala +++ b/core/src/test/scala/integration/kafka/server/ScramServerStartupTest.scala @@ -27,7 +27,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol import org.junit.Assert._ import org.junit.Test -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** * Tests that there are no failed authentications during broker startup. This is to verify diff --git a/core/src/test/scala/kafka/security/minikdc/MiniKdc.scala b/core/src/test/scala/kafka/security/minikdc/MiniKdc.scala index 17d6cb84b63..12f832264a5 100644 --- a/core/src/test/scala/kafka/security/minikdc/MiniKdc.scala +++ b/core/src/test/scala/kafka/security/minikdc/MiniKdc.scala @@ -27,7 +27,7 @@ import java.util.{Locale, Properties, UUID} import kafka.utils.{CoreUtils, Exit, Logging} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import org.apache.commons.lang.text.StrSubstitutor import org.apache.directory.api.ldap.model.entry.{DefaultEntry, Entry} import org.apache.directory.api.ldap.model.ldif.LdifReader diff --git a/core/src/test/scala/kafka/tools/LogCompactionTester.scala b/core/src/test/scala/kafka/tools/LogCompactionTester.scala index 72463513da0..da8a3c03a17 100755 --- a/core/src/test/scala/kafka/tools/LogCompactionTester.scala +++ b/core/src/test/scala/kafka/tools/LogCompactionTester.scala @@ -34,7 +34,7 @@ import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.serialization.{ByteArraySerializer, StringDeserializer} import org.apache.kafka.common.utils.{AbstractIterator, Utils} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** * This is a torture test that runs against an existing broker @@ -302,7 +302,7 @@ object LogCompactionTester { def consumeMessages(brokerUrl: String, topics: Array[String]): Path = { val consumer = createConsumer(brokerUrl) - consumer.subscribe(topics.seq.asJava) + consumer.subscribe(topics.toSeq.asJava) val consumedFilePath = Files.createTempFile("kafka-log-cleaner-consumed-", ".txt") println(s"Logging consumed messages to $consumedFilePath") val consumedWriter: BufferedWriter = Files.newBufferedWriter(consumedFilePath, UTF_8) diff --git a/core/src/test/scala/kafka/zk/LiteralAclStoreTest.scala b/core/src/test/scala/kafka/zk/LiteralAclStoreTest.scala index 528cb46b866..2f07cc64c5d 100644 --- a/core/src/test/scala/kafka/zk/LiteralAclStoreTest.scala +++ b/core/src/test/scala/kafka/zk/LiteralAclStoreTest.scala @@ -67,10 +67,10 @@ class LiteralAclStoreTest { @Test def shouldDecodeResourceUsingTwoPartLogic(): Unit = { val resource = new ResourcePattern(GROUP, "PREFIXED:this, including the PREFIXED part, is a valid two part group name", LITERAL) - val encoded = (resource.resourceType + AclEntry.ResourceSeparator + resource.name).getBytes(UTF_8) + val encoded = (resource.resourceType.toString + AclEntry.ResourceSeparator + resource.name).getBytes(UTF_8) val actual = store.changeStore.decode(encoded) assertEquals(resource, actual) } -} \ No newline at end of file +} diff --git a/core/src/test/scala/other/kafka/ReplicationQuotasTestRig.scala b/core/src/test/scala/other/kafka/ReplicationQuotasTestRig.scala index b5795374f08..58e16ef63af 100644 --- a/core/src/test/scala/other/kafka/ReplicationQuotasTestRig.scala +++ b/core/src/test/scala/other/kafka/ReplicationQuotasTestRig.scala @@ -37,7 +37,7 @@ import org.jfree.chart.plot.PlotOrientation import org.jfree.chart.{ChartFactory, ChartFrame, JFreeChart} import org.jfree.data.xy.{XYSeries, XYSeriesCollection} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.{Map, Seq, mutable} /** @@ -128,7 +128,7 @@ object ReplicationQuotasTestRig { experimentName = config.name val brokers = (100 to 100 + config.brokers) var count = 0 - val shift = Math.round(config.brokers / 2) + val shift = Math.round(config.brokers / 2f) def nextReplicaRoundRobin(): Int = { count = count + 1 @@ -188,9 +188,9 @@ object ReplicationQuotasTestRig { //Long stats println("The replicas are " + replicas.toSeq.sortBy(_._1).map("\n" + _)) - println("This is the current replica assignment:\n" + actual.mapValues(_.replicas).toMap.toSeq) + println("This is the current replica assignment:\n" + actual.map { case (k, v) => k -> v.replicas }) println("proposed assignment is: \n" + newAssignment) - println("This is the assignment we ended up with" + actual.mapValues(_.replicas).toMap) + println("This is the assignment we ended up with" + actual.map { case (k, v) => k -> v.replicas }) //Test Stats println(s"numBrokers: ${config.brokers}") @@ -302,7 +302,7 @@ object ReplicationQuotasTestRig { val message = s"\n\n

${config.name}

" + s"

- BrokerCount: ${config.brokers}" + s"

- PartitionCount: ${config.partitions}" + - f"

- Throttle: ${config.throttle}%,.0f MB/s" + + f"

- Throttle: ${config.throttle.toDouble}%,.0f MB/s" + f"

- MsgCount: ${config.msgsPerPartition}%,.0f " + f"

- MsgSize: ${config.msgSize}%,.0f" + s"

- TargetBytesPerBrokerMB: ${config.targetBytesPerBrokerMB}

" diff --git a/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala b/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala index bf21deeae35..24a49c32d50 100755 --- a/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala +++ b/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala @@ -168,7 +168,7 @@ object TestLinearWriteSpeed { } } val elapsedSecs = (System.nanoTime - beginTest) / (1000.0*1000.0*1000.0) - println(bytesToWrite / (1024.0 * 1024.0 * elapsedSecs) + " MB per sec") + println((bytesToWrite / (1024.0 * 1024.0 * elapsedSecs)).toString + " MB per sec") scheduler.shutdown() } diff --git a/core/src/test/scala/other/kafka/TestPurgatoryPerformance.scala b/core/src/test/scala/other/kafka/TestPurgatoryPerformance.scala index 4b540a4f1a8..507c3ff0973 100644 --- a/core/src/test/scala/other/kafka/TestPurgatoryPerformance.scala +++ b/core/src/test/scala/other/kafka/TestPurgatoryPerformance.scala @@ -28,7 +28,7 @@ import kafka.utils._ import org.apache.kafka.common.utils.Time import scala.math._ -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** * This is a benchmark test of the purgatory. diff --git a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala index c3430f45c04..4a388e8055b 100644 --- a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala @@ -28,7 +28,7 @@ import org.junit.{After, Before, Test} import org.junit.Assert._ import org.apache.kafka.common.config.internals.BrokerSecurityConfigs -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class KafkaTest { diff --git a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala index cdb4c0a4220..b5a497759a5 100755 --- a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala @@ -30,7 +30,7 @@ import org.apache.kafka.common.requests.{MetadataRequest, MetadataResponse} import org.junit.Assert._ import org.junit.{Before, Test} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class AddPartitionsTest extends BaseRequestTest { @@ -53,10 +53,10 @@ class AddPartitionsTest extends BaseRequestTest { override def setUp(): Unit = { super.setUp() - createTopic(topic1, partitionReplicaAssignment = topic1Assignment.mapValues(_.replicas).toMap) - createTopic(topic2, partitionReplicaAssignment = topic2Assignment.mapValues(_.replicas).toMap) - createTopic(topic3, partitionReplicaAssignment = topic3Assignment.mapValues(_.replicas).toMap) - createTopic(topic4, partitionReplicaAssignment = topic4Assignment.mapValues(_.replicas).toMap) + createTopic(topic1, partitionReplicaAssignment = topic1Assignment.map { case (k, v) => k -> v.replicas }) + createTopic(topic2, partitionReplicaAssignment = topic2Assignment.map { case (k, v) => k -> v.replicas }) + createTopic(topic3, partitionReplicaAssignment = topic3Assignment.map { case (k, v) => k -> v.replicas }) + createTopic(topic4, partitionReplicaAssignment = topic4Assignment.map { case (k, v) => k -> v.replicas }) } @Test diff --git a/core/src/test/scala/unit/kafka/admin/AdminRackAwareTest.scala b/core/src/test/scala/unit/kafka/admin/AdminRackAwareTest.scala index 48bddd453f1..3f2c06d830e 100644 --- a/core/src/test/scala/unit/kafka/admin/AdminRackAwareTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AdminRackAwareTest.scala @@ -31,7 +31,7 @@ class AdminRackAwareTest extends RackAwareTest with Logging { val rackMap = Map(0 -> "rack1", 1 -> "rack3", 2 -> "rack3", 3 -> "rack2", 4 -> "rack2", 5 -> "rack1") val newList = AdminUtils.getRackAlternatedBrokerList(rackMap) assertEquals(List(0, 3, 1, 5, 4, 2), newList) - val anotherList = AdminUtils.getRackAlternatedBrokerList(rackMap - 5) + val anotherList = AdminUtils.getRackAlternatedBrokerList(rackMap.toMap - 5) assertEquals(List(0, 3, 1, 4, 2), anotherList) val assignment = AdminUtils.assignReplicasToBrokers(toBrokerMetadata(rackMap), 7, 3, 0, 0) val expected = Map(0 -> List(0, 3, 1), diff --git a/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala b/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala index 5a76f75c60a..a963c74dd62 100644 --- a/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala @@ -41,7 +41,7 @@ import org.junit.Test import org.scalatest.Assertions.intercept import scala.collection.{Seq, mutable} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class ConfigCommandTest extends ZooKeeperTestHarness with Logging { diff --git a/core/src/test/scala/unit/kafka/admin/ConsumerGroupCommandTest.scala b/core/src/test/scala/unit/kafka/admin/ConsumerGroupCommandTest.scala index 853b2caa7c7..3a6405234ea 100644 --- a/core/src/test/scala/unit/kafka/admin/ConsumerGroupCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/ConsumerGroupCommandTest.scala @@ -32,7 +32,7 @@ import org.apache.kafka.common.errors.WakeupException import org.apache.kafka.common.serialization.StringDeserializer import org.junit.{After, Before} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable.ArrayBuffer class ConsumerGroupCommandTest extends KafkaServerTestHarness { @@ -64,12 +64,12 @@ class ConsumerGroupCommandTest extends KafkaServerTestHarness { super.tearDown() } - def committedOffsets(topic: String = topic, group: String = group): Map[TopicPartition, Long] = { + def committedOffsets(topic: String = topic, group: String = group): collection.Map[TopicPartition, Long] = { val consumer = createNoAutoCommitConsumer(group) try { val partitions: Set[TopicPartition] = consumer.partitionsFor(topic) .asScala.toSet.map {partitionInfo : PartitionInfo => new TopicPartition(partitionInfo.topic, partitionInfo.partition)} - consumer.committed(partitions.asJava).asScala.filter(_._2 != null).mapValues(_.offset()).toMap + consumer.committed(partitions.asJava).asScala.filter(_._2 != null).map { case (k, v) => k -> v.offset } } finally { consumer.close() } diff --git a/core/src/test/scala/unit/kafka/admin/DelegationTokenCommandTest.scala b/core/src/test/scala/unit/kafka/admin/DelegationTokenCommandTest.scala index b89966f36d9..291cdc3f9be 100644 --- a/core/src/test/scala/unit/kafka/admin/DelegationTokenCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/DelegationTokenCommandTest.scala @@ -28,7 +28,7 @@ import org.junit.Assert._ import org.junit.{After, Before, Test} import org.scalatest.Assertions.intercept -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable.ListBuffer import scala.concurrent.ExecutionException diff --git a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala index dd1ea683a9b..afa571aea04 100644 --- a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala +++ b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala @@ -40,7 +40,9 @@ class DeleteTopicTest extends ZooKeeperTestHarness { var servers: Seq[KafkaServer] = Seq() val expectedReplicaAssignment = Map(0 -> List(0, 1, 2)) - val expectedReplicaFullAssignment = expectedReplicaAssignment.mapValues(ReplicaAssignment(_, List(), List())).toMap + val expectedReplicaFullAssignment = expectedReplicaAssignment.map { case (k, v) => + k -> ReplicaAssignment(v, List(), List()) + } @After override def tearDown(): Unit = { diff --git a/core/src/test/scala/unit/kafka/admin/LeaderElectionCommandTest.scala b/core/src/test/scala/unit/kafka/admin/LeaderElectionCommandTest.scala index 1b8069ac26d..3cfba5edfeb 100644 --- a/core/src/test/scala/unit/kafka/admin/LeaderElectionCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/LeaderElectionCommandTest.scala @@ -36,7 +36,7 @@ import org.junit.Assert._ import org.junit.Before import org.junit.Test -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.Seq import scala.concurrent.duration._ diff --git a/core/src/test/scala/unit/kafka/admin/PreferredReplicaLeaderElectionCommandTest.scala b/core/src/test/scala/unit/kafka/admin/PreferredReplicaLeaderElectionCommandTest.scala index 37032f17501..1fb326792cb 100644 --- a/core/src/test/scala/unit/kafka/admin/PreferredReplicaLeaderElectionCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/PreferredReplicaLeaderElectionCommandTest.scala @@ -41,7 +41,7 @@ import org.apache.kafka.test import org.junit.Assert._ import org.junit.{After, Test} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class PreferredReplicaLeaderElectionCommandTest extends ZooKeeperTestHarness with Logging { var servers: Seq[KafkaServer] = Seq() @@ -367,7 +367,7 @@ class PreferredReplicaLeaderElectionCommandTest extends ZooKeeperTestHarness wit val serverConfigs = TestUtils.createBrokerConfigs(3, zkConnect, false, rackInfo = brokerRack).map(KafkaConfig.fromProps) // create the topic adminZkClient.createTopicWithAssignment(topic, config = new Properties, expectedReplicaAssignment) - servers = serverConfigs.reverseMap(s => TestUtils.createServer(s)) + servers = serverConfigs.reverse.map(s => TestUtils.createServer(s)) // broker 2 should be the leader since it was started first val currentLeader = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, partition, oldLeaderOpt = None) // trigger preferred replica election diff --git a/core/src/test/scala/unit/kafka/admin/ReassignPartitionsUnitTest.scala b/core/src/test/scala/unit/kafka/admin/ReassignPartitionsUnitTest.scala index 6dbca36b417..a9cd8f57df6 100644 --- a/core/src/test/scala/unit/kafka/admin/ReassignPartitionsUnitTest.scala +++ b/core/src/test/scala/unit/kafka/admin/ReassignPartitionsUnitTest.scala @@ -32,8 +32,8 @@ import org.junit.function.ThrowingRunnable import org.junit.rules.Timeout import org.junit.{After, Assert, Before, Rule, Test} -import scala.collection.JavaConverters._ import scala.collection.mutable +import scala.jdk.CollectionConverters._ class ReassignPartitionsUnitTest { @Rule @@ -106,30 +106,29 @@ class ReassignPartitionsUnitTest { try { addTopics(adminClient) // Create a reassignment and test findPartitionReassignmentStates. - assertEquals(Map( - new TopicPartition("quux", 0) -> classOf[UnknownTopicOrPartitionException] - ), - alterPartitionReassignments(adminClient, Map( - new TopicPartition("foo", 0) -> Seq(0,1,3), - new TopicPartition("quux", 0) -> Seq(1,2,3))).mapValues(_.getClass).toMap) + val reassignmentResult: Map[TopicPartition, Class[_ <: Throwable]] = alterPartitionReassignments(adminClient, Map( + new TopicPartition("foo", 0) -> Seq(0,1,3), + new TopicPartition("quux", 0) -> Seq(1,2,3))).map { case (k, v) => k -> v.getClass }.toMap + assertEquals(Map(new TopicPartition("quux", 0) -> classOf[UnknownTopicOrPartitionException]), + reassignmentResult) assertEquals((Map( - new TopicPartition("foo", 0) -> new PartitionReassignmentState(Seq(0,1,2), Seq(0,1,3), false), - new TopicPartition("foo", 1) -> new PartitionReassignmentState(Seq(1,2,3), Seq(1,2,3), true) + new TopicPartition("foo", 0) -> PartitionReassignmentState(Seq(0,1,2), Seq(0,1,3), false), + new TopicPartition("foo", 1) -> PartitionReassignmentState(Seq(1,2,3), Seq(1,2,3), true) ), true), findPartitionReassignmentStates(adminClient, Seq( (new TopicPartition("foo", 0), Seq(0,1,3)), (new TopicPartition("foo", 1), Seq(1,2,3)) ))) // Cancel the reassignment and test findPartitionReassignmentStates again. - assertEquals(Map( - new TopicPartition("quux", 2) -> classOf[UnknownTopicOrPartitionException] - ), - cancelPartitionReassignments(adminClient, Set( - new TopicPartition("foo", 0), - new TopicPartition("quux", 2))).mapValues(_.getClass).toMap) + val cancelResult: Map[TopicPartition, Class[_ <: Throwable]] = cancelPartitionReassignments(adminClient, + Set(new TopicPartition("foo", 0), new TopicPartition("quux", 2))).map { case (k, v) => + k -> v.getClass + }.toMap + assertEquals(Map(new TopicPartition("quux", 2) -> classOf[UnknownTopicOrPartitionException]), + cancelResult) assertEquals((Map( - new TopicPartition("foo", 0) -> new PartitionReassignmentState(Seq(0,1,2), Seq(0,1,3), true), - new TopicPartition("foo", 1) -> new PartitionReassignmentState(Seq(1,2,3), Seq(1,2,3), true) + new TopicPartition("foo", 0) -> PartitionReassignmentState(Seq(0,1,2), Seq(0,1,3), true), + new TopicPartition("foo", 1) -> PartitionReassignmentState(Seq(1,2,3), Seq(1,2,3), true) ), false), findPartitionReassignmentStates(adminClient, Seq( (new TopicPartition("foo", 0), Seq(0,1,3)), @@ -552,13 +551,15 @@ class ReassignPartitionsUnitTest { val adminClient = new MockAdminClient.Builder().numBrokers(4).build() try { addTopics(adminClient) - assertEquals("""No partition reassignments found.""", - curReassignmentsToString(adminClient)) - assertEquals(Map(), - alterPartitionReassignments(adminClient, Map( + assertEquals("No partition reassignments found.", curReassignmentsToString(adminClient)) + val reassignmentResult: Map[TopicPartition, Class[_ <: Throwable]] = alterPartitionReassignments(adminClient, + Map( new TopicPartition("foo", 1) -> Seq(4,5,3), new TopicPartition("foo", 0) -> Seq(0,1,4,2), - new TopicPartition("bar", 0) -> Seq(2,3))).mapValues(_.getClass).toMap) + new TopicPartition("bar", 0) -> Seq(2,3) + ) + ).map { case (k, v) => k -> v.getClass }.toMap + assertEquals(Map(), reassignmentResult) assertEquals(Seq("Current partition reassignments:", "bar-0: replicas: 2,3,0. removing: 0.", "foo-0: replicas: 0,1,2. adding: 4.", diff --git a/core/src/test/scala/unit/kafka/admin/ResetConsumerGroupOffsetTest.scala b/core/src/test/scala/unit/kafka/admin/ResetConsumerGroupOffsetTest.scala index 158787c104c..781586f3112 100644 --- a/core/src/test/scala/unit/kafka/admin/ResetConsumerGroupOffsetTest.scala +++ b/core/src/test/scala/unit/kafka/admin/ResetConsumerGroupOffsetTest.scala @@ -26,7 +26,7 @@ import org.apache.kafka.test import org.junit.Assert._ import org.junit.Test -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.Seq class TimeConversionTests { @@ -328,7 +328,7 @@ class ResetConsumerGroupOffsetTest extends ConsumerGroupCommandTest { val tp1 = new TopicPartition(topic1, 0) val tp2 = new TopicPartition(topic2, 0) - val allResetOffsets = resetOffsets(consumerGroupCommand)(group).mapValues(_.offset).toMap + val allResetOffsets = resetOffsets(consumerGroupCommand)(group).map { case (k, v) => k -> v.offset } assertEquals(Map(tp1 -> 0L, tp2 -> 0L), allResetOffsets) assertEquals(Map(tp1 -> 0L), committedOffsets(topic1)) assertEquals(Map(tp2 -> 0L), committedOffsets(topic2)) @@ -356,11 +356,11 @@ class ResetConsumerGroupOffsetTest extends ConsumerGroupCommandTest { val tp1 = new TopicPartition(topic1, 1) val tp2 = new TopicPartition(topic2, 1) - val allResetOffsets = resetOffsets(consumerGroupCommand)(group).mapValues(_.offset).toMap + val allResetOffsets = resetOffsets(consumerGroupCommand)(group).map { case (k, v) => k -> v.offset } assertEquals(Map(tp1 -> 0, tp2 -> 0), allResetOffsets) - assertEquals(priorCommittedOffsets1 + (tp1 -> 0L), committedOffsets(topic1)) - assertEquals(priorCommittedOffsets2 + (tp2 -> 0L), committedOffsets(topic2)) + assertEquals(priorCommittedOffsets1.toMap + (tp1 -> 0L), committedOffsets(topic1)) + assertEquals(priorCommittedOffsets2.toMap + (tp2 -> 0L), committedOffsets(topic2)) adminZkClient.deleteTopic(topic1) adminZkClient.deleteTopic(topic2) @@ -386,12 +386,12 @@ class ResetConsumerGroupOffsetTest extends ConsumerGroupCommandTest { val bw = new BufferedWriter(new FileWriter(file)) bw.write(consumerGroupCommand.exportOffsetsToCsv(exportedOffsets)) bw.close() - assertEquals(Map(tp0 -> 2L, tp1 -> 2L), exportedOffsets(group).mapValues(_.offset).toMap) + assertEquals(Map(tp0 -> 2L, tp1 -> 2L), exportedOffsets(group).map { case (k, v) => k -> v.offset }) val cgcArgsExec = buildArgsForGroup(group, "--all-topics", "--from-file", file.getCanonicalPath, "--dry-run") val consumerGroupCommandExec = getConsumerGroupService(cgcArgsExec) val importedOffsets = consumerGroupCommandExec.resetOffsets() - assertEquals(Map(tp0 -> 2L, tp1 -> 2L), importedOffsets(group).mapValues(_.offset).toMap) + assertEquals(Map(tp0 -> 2L, tp1 -> 2L), importedOffsets(group).map { case (k, v) => k -> v.offset }) adminZkClient.deleteTopic(topic) } @@ -427,21 +427,21 @@ class ResetConsumerGroupOffsetTest extends ConsumerGroupCommandTest { val bw = new BufferedWriter(new FileWriter(file)) bw.write(consumerGroupCommand.exportOffsetsToCsv(exportedOffsets)) bw.close() - assertEquals(Map(t1p0 -> 2L, t1p1 -> 2L), exportedOffsets(group1).mapValues(_.offset).toMap) - assertEquals(Map(t2p0 -> 2L, t2p1 -> 2L), exportedOffsets(group2).mapValues(_.offset).toMap) + assertEquals(Map(t1p0 -> 2L, t1p1 -> 2L), exportedOffsets(group1).map { case (k, v) => k -> v.offset }) + assertEquals(Map(t2p0 -> 2L, t2p1 -> 2L), exportedOffsets(group2).map { case (k, v) => k -> v.offset }) // Multiple --group's offset import val cgcArgsExec = buildArgsForGroups(Seq(group1, group2), "--all-topics", "--from-file", file.getCanonicalPath, "--dry-run") val consumerGroupCommandExec = getConsumerGroupService(cgcArgsExec) val importedOffsets = consumerGroupCommandExec.resetOffsets() - assertEquals(Map(t1p0 -> 2L, t1p1 -> 2L), importedOffsets(group1).mapValues(_.offset).toMap) - assertEquals(Map(t2p0 -> 2L, t2p1 -> 2L), importedOffsets(group2).mapValues(_.offset).toMap) + assertEquals(Map(t1p0 -> 2L, t1p1 -> 2L), importedOffsets(group1).map { case (k, v) => k -> v.offset }) + assertEquals(Map(t2p0 -> 2L, t2p1 -> 2L), importedOffsets(group2).map { case (k, v) => k -> v.offset }) // Single --group offset import using "group,topic,partition,offset" csv format val cgcArgsExec2 = buildArgsForGroup(group1, "--all-topics", "--from-file", file.getCanonicalPath, "--dry-run") val consumerGroupCommandExec2 = getConsumerGroupService(cgcArgsExec2) val importedOffsets2 = consumerGroupCommandExec2.resetOffsets() - assertEquals(Map(t1p0 -> 2L, t1p1 -> 2L), importedOffsets2(group1).mapValues(_.offset).toMap) + assertEquals(Map(t1p0 -> 2L, t1p1 -> 2L), importedOffsets2(group1).map { case (k, v) => k -> v.offset }) adminZkClient.deleteTopic(topic) } @@ -511,7 +511,8 @@ class ResetConsumerGroupOffsetTest extends ConsumerGroupCommandTest { (group, partitionInfo) <- resetOffsetsResultByGroup } { val priorOffsets = committedOffsets(topic = topic, group = group) - assertEquals(expectedOffsets(topic), partitionInfo.filter(partitionInfo => partitionInfo._1.topic() == topic).mapValues(_.offset).toMap) + assertEquals(expectedOffsets(topic), + partitionInfo.filter(partitionInfo => partitionInfo._1.topic() == topic).map { case (k, v) => k -> v.offset }) assertEquals(if (dryRun) priorOffsets else expectedOffsets(topic), committedOffsets(topic = topic, group = group)) } } finally { diff --git a/core/src/test/scala/unit/kafka/admin/TopicCommandWithAdminClientTest.scala b/core/src/test/scala/unit/kafka/admin/TopicCommandWithAdminClientTest.scala index cdeb49a1208..3f95038e7d9 100644 --- a/core/src/test/scala/unit/kafka/admin/TopicCommandWithAdminClientTest.scala +++ b/core/src/test/scala/unit/kafka/admin/TopicCommandWithAdminClientTest.scala @@ -37,7 +37,7 @@ import org.junit.rules.TestName import org.junit.{After, Before, Rule, Test} import org.scalatest.Assertions.{fail, intercept} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.Seq import scala.concurrent.ExecutionException import scala.util.Random diff --git a/core/src/test/scala/unit/kafka/cluster/AssignmentStateTest.scala b/core/src/test/scala/unit/kafka/cluster/AssignmentStateTest.scala index b218bf72974..96bbdb60083 100644 --- a/core/src/test/scala/unit/kafka/cluster/AssignmentStateTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/AssignmentStateTest.scala @@ -24,7 +24,7 @@ import org.junit.runner.RunWith import org.junit.runners.Parameterized import org.junit.runners.Parameterized.Parameters -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ object AssignmentStateTest extends AbstractPartitionTest { @@ -90,7 +90,6 @@ class AssignmentStateTest(isr: List[Integer], replicas: List[Integer], @Test def testPartitionAssignmentStatus(): Unit = { - val controllerId = 0 val controllerEpoch = 3 val leaderState = new LeaderAndIsrPartitionState() diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala index c8a7d33a5d2..78e13ebff0b 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala @@ -35,7 +35,7 @@ import org.junit.{After, Before, Test} import org.mockito.ArgumentMatchers import org.mockito.Mockito.{mock, when} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** * Verifies that slow appends to log don't block request threads processing replica fetch requests. @@ -237,7 +237,6 @@ class PartitionLockTest extends Logging { partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints) - val controllerId = 0 val controllerEpoch = 0 val replicas = (0 to numReplicaFetchers).map(i => Integer.valueOf(brokerId + i)).toList.asJava val isr = replicas diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala index 62ec28b2733..08cbd7d75e8 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala @@ -45,7 +45,7 @@ import org.mockito.ArgumentMatchers import org.mockito.invocation.InvocationOnMock import unit.kafka.cluster.AbstractPartitionTest -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable.ListBuffer class PartitionTest extends AbstractPartitionTest { @@ -1510,7 +1510,7 @@ class PartitionTest extends AbstractPartitionTest { "AtMinIsr") def getMetric(metric: String): Option[Metric] = { - KafkaYammerMetrics.defaultRegistry().allMetrics().asScala.filterKeys { metricName => + KafkaYammerMetrics.defaultRegistry().allMetrics().asScala.filter { case (metricName, _) => metricName.getName == metric && metricName.getType == "Partition" }.headOption.map(_._2) } diff --git a/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala index 38f0d29fe1f..5ae89392495 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala @@ -34,7 +34,7 @@ import org.junit.Assert._ import org.junit.Test import org.scalatest.Assertions -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable import scala.collection.mutable.ListBuffer @@ -699,7 +699,7 @@ class ControllerChannelManagerTest { Broker(brokerId, Seq(endpoint), rack = None) -> 1L }.toMap - context.setLiveBrokerAndEpochs(brokerEpochs) + context.setLiveBrokers(brokerEpochs) // Simple round-robin replica assignment var leaderIndex = 0 diff --git a/core/src/test/scala/unit/kafka/controller/ControllerContextTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerContextTest.scala index 0923b1860de..96240b25ea5 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerContextTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerContextTest.scala @@ -46,7 +46,7 @@ class ControllerContextTest { Broker(brokerId, Seq(endpoint), rack = None) -> 1L }.toMap - context.setLiveBrokerAndEpochs(brokerEpochs) + context.setLiveBrokers(brokerEpochs) // Simple round-robin replica assignment var leaderIndex = 0 diff --git a/core/src/test/scala/unit/kafka/controller/ControllerEventManagerTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerEventManagerTest.scala index 4aa02ccf97a..8a213ae77b1 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerEventManagerTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerEventManagerTest.scala @@ -31,7 +31,7 @@ import org.apache.kafka.common.utils.MockTime import org.junit.Assert.{assertEquals, assertTrue, fail} import org.junit.{After, Test} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable class ControllerEventManagerTest { @@ -111,7 +111,9 @@ class ControllerEventManagerTest { } // The metric should not already exist - assertTrue(KafkaYammerMetrics.defaultRegistry.allMetrics.asScala.filterKeys(_.getMBeanName == metricName).values.isEmpty) + assertTrue(KafkaYammerMetrics.defaultRegistry.allMetrics.asScala.filter { case (k, _) => + k.getMBeanName == metricName + }.values.isEmpty) controllerEventManager = new ControllerEventManager(0, eventProcessor, time, controllerStats.rateAndTimeMetrics) @@ -124,8 +126,9 @@ class ControllerEventManagerTest { TestUtils.waitUntilTrue(() => processedEvents.get() == 2, "Timed out waiting for processing of all events") - val queueTimeHistogram = KafkaYammerMetrics.defaultRegistry.allMetrics.asScala.filterKeys(_.getMBeanName == metricName).values.headOption - .getOrElse(fail(s"Unable to find metric $metricName")).asInstanceOf[Histogram] + val queueTimeHistogram = KafkaYammerMetrics.defaultRegistry.allMetrics.asScala.filter { case (k, _) => + k.getMBeanName == metricName + }.values.headOption.getOrElse(fail(s"Unable to find metric $metricName")).asInstanceOf[Histogram] assertEquals(2, queueTimeHistogram.count) assertEquals(0, queueTimeHistogram.min, 0.01) @@ -179,8 +182,9 @@ class ControllerEventManagerTest { } private def timer(metricName: String): Timer = { - KafkaYammerMetrics.defaultRegistry.allMetrics.asScala.filterKeys(_.getMBeanName == metricName).values.headOption - .getOrElse(fail(s"Unable to find metric $metricName")).asInstanceOf[Timer] + KafkaYammerMetrics.defaultRegistry.allMetrics.asScala.filter { case (k, _) => + k.getMBeanName == metricName + }.values.headOption.getOrElse(fail(s"Unable to find metric $metricName")).asInstanceOf[Timer] } } diff --git a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala index 771c81c162f..c4b5f478d19 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala @@ -35,7 +35,7 @@ import kafka.utils.LogCaptureAppender import org.apache.kafka.common.metrics.KafkaMetric import org.scalatest.Assertions.fail -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable import scala.collection.Seq import scala.util.{Failure, Success, Try} @@ -292,7 +292,7 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness { val assignment = Map(tp.partition -> Seq(controllerId)) val reassignment = Map(tp -> ReplicaAssignment(Seq(otherBrokerId), List(), List())) TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers) - zkClient.createPartitionReassignment(reassignment.mapValues(_.replicas).toMap) + zkClient.createPartitionReassignment(reassignment.map { case (k, v) => k -> v.replicas }) waitForPartitionState(tp, firstControllerEpoch, otherBrokerId, LeaderAndIsr.initialLeaderEpoch + 3, "failed to get expected partition state after partition reassignment") TestUtils.waitUntilTrue(() => zkClient.getFullReplicaAssignmentForTopics(Set(tp.topic)) == reassignment, @@ -334,7 +334,7 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness { TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers) servers(otherBrokerId).shutdown() servers(otherBrokerId).awaitShutdown() - zkClient.createPartitionReassignment(reassignment.mapValues(_.replicas).toMap) + zkClient.createPartitionReassignment(reassignment.map { case (k, v) => k -> v.replicas }) waitForPartitionState(tp, firstControllerEpoch, controllerId, LeaderAndIsr.initialLeaderEpoch + 1, "failed to get expected partition state during partition reassignment with offline replica") servers(otherBrokerId).startup() @@ -450,7 +450,7 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness { val partition = 0 // create brokers val serverConfigs = TestUtils.createBrokerConfigs(3, zkConnect, false).map(KafkaConfig.fromProps) - servers = serverConfigs.reverseMap(s => TestUtils.createServer(s)) + servers = serverConfigs.reverse.map(s => TestUtils.createServer(s)) // create the topic TestUtils.createTopic(zkClient, topic, partitionReplicaAssignment = expectedReplicaAssignment, servers = servers) @@ -494,7 +494,7 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness { def testControllerRejectControlledShutdownRequestWithStaleBrokerEpoch(): Unit = { // create brokers val serverConfigs = TestUtils.createBrokerConfigs(2, zkConnect, false).map(KafkaConfig.fromProps) - servers = serverConfigs.reverseMap(s => TestUtils.createServer(s)) + servers = serverConfigs.reverse.map(s => TestUtils.createServer(s)) val controller = getController().kafkaController val otherBroker = servers.find(e => e.config.brokerId != controller.config.brokerId).get @@ -698,8 +698,9 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness { } private def timer(metricName: String): Timer = { - KafkaYammerMetrics.defaultRegistry.allMetrics.asScala.filterKeys(_.getMBeanName == metricName).values.headOption - .getOrElse(fail(s"Unable to find metric $metricName")).asInstanceOf[Timer] + KafkaYammerMetrics.defaultRegistry.allMetrics.asScala.filter { case (k, _) => + k.getMBeanName == metricName + }.values.headOption.getOrElse(fail(s"Unable to find metric $metricName")).asInstanceOf[Timer] } private def getController(): KafkaServer = { diff --git a/core/src/test/scala/unit/kafka/controller/PartitionStateMachineTest.scala b/core/src/test/scala/unit/kafka/controller/PartitionStateMachineTest.scala index edf59837b28..ae5bf48e922 100644 --- a/core/src/test/scala/unit/kafka/controller/PartitionStateMachineTest.scala +++ b/core/src/test/scala/unit/kafka/controller/PartitionStateMachineTest.scala @@ -81,7 +81,7 @@ class PartitionStateMachineTest { @Test def testNewPartitionToOnlinePartitionTransition(): Unit = { - controllerContext.setLiveBrokerAndEpochs(Map(TestUtils.createBrokerAndEpoch(brokerId, "host", 0))) + controllerContext.setLiveBrokers(Map(TestUtils.createBrokerAndEpoch(brokerId, "host", 0))) controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(Seq(brokerId))) controllerContext.putPartitionState(partition, NewPartition) val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(LeaderAndIsr(brokerId, List(brokerId)), controllerEpoch) @@ -103,7 +103,7 @@ class PartitionStateMachineTest { @Test def testNewPartitionToOnlinePartitionTransitionZooKeeperClientExceptionFromCreateStates(): Unit = { - controllerContext.setLiveBrokerAndEpochs(Map(TestUtils.createBrokerAndEpoch(brokerId, "host", 0))) + controllerContext.setLiveBrokers(Map(TestUtils.createBrokerAndEpoch(brokerId, "host", 0))) controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(Seq(brokerId))) controllerContext.putPartitionState(partition, NewPartition) val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(LeaderAndIsr(brokerId, List(brokerId)), controllerEpoch) @@ -123,7 +123,7 @@ class PartitionStateMachineTest { @Test def testNewPartitionToOnlinePartitionTransitionErrorCodeFromCreateStates(): Unit = { - controllerContext.setLiveBrokerAndEpochs(Map(TestUtils.createBrokerAndEpoch(brokerId, "host", 0))) + controllerContext.setLiveBrokers(Map(TestUtils.createBrokerAndEpoch(brokerId, "host", 0))) controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(Seq(brokerId))) controllerContext.putPartitionState(partition, NewPartition) val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(LeaderAndIsr(brokerId, List(brokerId)), controllerEpoch) @@ -157,7 +157,7 @@ class PartitionStateMachineTest { @Test def testOnlinePartitionToOnlineTransition(): Unit = { - controllerContext.setLiveBrokerAndEpochs(Map(TestUtils.createBrokerAndEpoch(brokerId, "host", 0))) + controllerContext.setLiveBrokers(Map(TestUtils.createBrokerAndEpoch(brokerId, "host", 0))) controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(Seq(brokerId))) controllerContext.putPartitionState(partition, OnlinePartition) val leaderAndIsr = LeaderAndIsr(brokerId, List(brokerId)) @@ -187,7 +187,7 @@ class PartitionStateMachineTest { @Test def testOnlinePartitionToOnlineTransitionForControlledShutdown(): Unit = { val otherBrokerId = brokerId + 1 - controllerContext.setLiveBrokerAndEpochs(Map( + controllerContext.setLiveBrokers(Map( TestUtils.createBrokerAndEpoch(brokerId, "host", 0), TestUtils.createBrokerAndEpoch(otherBrokerId, "host", 0))) controllerContext.shuttingDownBrokerIds.add(brokerId) @@ -246,7 +246,7 @@ class PartitionStateMachineTest { @Test def testOfflinePartitionToOnlinePartitionTransition(): Unit = { - controllerContext.setLiveBrokerAndEpochs(Map(TestUtils.createBrokerAndEpoch(brokerId, "host", 0))) + controllerContext.setLiveBrokers(Map(TestUtils.createBrokerAndEpoch(brokerId, "host", 0))) controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(Seq(brokerId))) controllerContext.putPartitionState(partition, OfflinePartition) val leaderAndIsr = LeaderAndIsr(LeaderAndIsr.NoLeader, List(brokerId)) @@ -289,7 +289,7 @@ class PartitionStateMachineTest { * elected. */ val leaderBrokerId = brokerId + 1 - controllerContext.setLiveBrokerAndEpochs(Map(TestUtils.createBrokerAndEpoch(brokerId, "host", 0))) + controllerContext.setLiveBrokers(Map(TestUtils.createBrokerAndEpoch(brokerId, "host", 0))) controllerContext.updatePartitionFullReplicaAssignment( partition, ReplicaAssignment(Seq(leaderBrokerId, brokerId)) @@ -351,7 +351,7 @@ class PartitionStateMachineTest { @Test def testOfflinePartitionToOnlinePartitionTransitionZooKeeperClientExceptionFromStateLookup(): Unit = { - controllerContext.setLiveBrokerAndEpochs(Map(TestUtils.createBrokerAndEpoch(brokerId, "host", 0))) + controllerContext.setLiveBrokers(Map(TestUtils.createBrokerAndEpoch(brokerId, "host", 0))) controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(Seq(brokerId))) controllerContext.putPartitionState(partition, OfflinePartition) val leaderAndIsr = LeaderAndIsr(LeaderAndIsr.NoLeader, List(brokerId)) @@ -376,7 +376,7 @@ class PartitionStateMachineTest { @Test def testOfflinePartitionToOnlinePartitionTransitionErrorCodeFromStateLookup(): Unit = { - controllerContext.setLiveBrokerAndEpochs(Map(TestUtils.createBrokerAndEpoch(brokerId, "host", 0))) + controllerContext.setLiveBrokers(Map(TestUtils.createBrokerAndEpoch(brokerId, "host", 0))) controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(Seq(brokerId))) controllerContext.putPartitionState(partition, OfflinePartition) val leaderAndIsr = LeaderAndIsr(LeaderAndIsr.NoLeader, List(brokerId)) @@ -438,7 +438,7 @@ class PartitionStateMachineTest { partition -> leaderAndIsr.newLeaderAndIsr(brokerId, List(brokerId)) }.toMap EasyMock.expect(mockZkClient.updateLeaderAndIsr(updatedLeaderAndIsr, controllerEpoch, controllerContext.epochZkVersion)) - .andReturn(UpdateLeaderAndIsrResult(updatedLeaderAndIsr.mapValues(Right(_)).toMap, Seq.empty)) + .andReturn(UpdateLeaderAndIsrResult(updatedLeaderAndIsr.map { case (k, v) => k -> Right(v) }, Seq.empty)) } prepareMockToUpdateLeaderAndIsr() } @@ -449,7 +449,7 @@ class PartitionStateMachineTest { */ @Test def testUpdatingOfflinePartitionsCount(): Unit = { - controllerContext.setLiveBrokerAndEpochs(Map(TestUtils.createBrokerAndEpoch(brokerId, "host", 0))) + controllerContext.setLiveBrokers(Map(TestUtils.createBrokerAndEpoch(brokerId, "host", 0))) val partitionIds = Seq(0, 1, 2, 3) val topic = "test" diff --git a/core/src/test/scala/unit/kafka/controller/ReplicaStateMachineTest.scala b/core/src/test/scala/unit/kafka/controller/ReplicaStateMachineTest.scala index ae5d67e73c2..e79d2f581e5 100644 --- a/core/src/test/scala/unit/kafka/controller/ReplicaStateMachineTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ReplicaStateMachineTest.scala @@ -65,7 +65,7 @@ class ReplicaStateMachineTest { val endpoint1 = new EndPoint("localhost", 9997, new ListenerName("blah"), SecurityProtocol.PLAINTEXT) val liveBrokerEpochs = Map(Broker(brokerId, Seq(endpoint1), rack = None) -> 1L) - controllerContext.setLiveBrokerAndEpochs(liveBrokerEpochs) + controllerContext.setLiveBrokers(liveBrokerEpochs) controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(Seq(brokerId))) assertEquals(None, controllerContext.replicaStates.get(replica)) replicaStateMachine.startup() @@ -87,7 +87,7 @@ class ReplicaStateMachineTest { val endpoint1 = new EndPoint("localhost", 9997, new ListenerName("blah"), SecurityProtocol.PLAINTEXT) val liveBrokerEpochs = Map(Broker(brokerId, Seq(endpoint1), rack = None) -> 1L) - controllerContext.setLiveBrokerAndEpochs(liveBrokerEpochs) + controllerContext.setLiveBrokers(liveBrokerEpochs) controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(Seq(shutdownBrokerId))) assertEquals(None, controllerContext.replicaStates.get(offlineReplica)) replicaStateMachine.startup() @@ -148,7 +148,7 @@ class ReplicaStateMachineTest { val endpoint1 = new EndPoint("localhost", 9997, new ListenerName("blah"), SecurityProtocol.PLAINTEXT) val liveBrokerEpochs = Map(Broker(brokerId, Seq(endpoint1), rack = None) -> 1L) - controllerContext.setLiveBrokerAndEpochs(liveBrokerEpochs) + controllerContext.setLiveBrokers(liveBrokerEpochs) controllerContext.putReplicaState(replica, NewReplica) EasyMock.expect(mockControllerBrokerRequestBatch.newBatch()) EasyMock.expect(mockControllerBrokerRequestBatch.addStopReplicaRequestForBrokers(EasyMock.eq(Seq(brokerId)), EasyMock.eq(partition), EasyMock.eq(false))) diff --git a/core/src/test/scala/unit/kafka/controller/TopicDeletionManagerTest.scala b/core/src/test/scala/unit/kafka/controller/TopicDeletionManagerTest.scala index b1b8c2471e4..d9f52e100b8 100644 --- a/core/src/test/scala/unit/kafka/controller/TopicDeletionManagerTest.scala +++ b/core/src/test/scala/unit/kafka/controller/TopicDeletionManagerTest.scala @@ -169,7 +169,7 @@ class TopicDeletionManagerTest { assertEquals(offlineReplicas, controllerContext.replicasInState("foo", OfflineReplica)) // Broker 2 comes back online and deletion is resumed - controllerContext.addLiveBrokersAndEpochs(Map(offlineBroker -> (lastEpoch + 1L))) + controllerContext.addLiveBrokers(Map(offlineBroker -> (lastEpoch + 1L))) deletionManager.resumeDeletionForTopics(Set("foo")) assertEquals(onlineReplicas, controllerContext.replicasInState("foo", ReplicaDeletionSuccessful)) @@ -227,7 +227,7 @@ class TopicDeletionManagerTest { // Broker 2 is restarted. The offline replicas remain ineligable // (TODO: this is probably not desired) - controllerContext.addLiveBrokersAndEpochs(Map(offlineBroker -> (lastEpoch + 1L))) + controllerContext.addLiveBrokers(Map(offlineBroker -> (lastEpoch + 1L))) deletionManager.resumeDeletionForTopics(Set("foo")) assertEquals(Set("foo"), controllerContext.topicsToBeDeleted) assertEquals(Set("foo"), controllerContext.topicsWithDeletionStarted) @@ -255,7 +255,7 @@ class TopicDeletionManagerTest { SecurityProtocol.PLAINTEXT) Broker(brokerId, Seq(endpoint), rack = None) -> 1L }.toMap - context.setLiveBrokerAndEpochs(brokerEpochs) + context.setLiveBrokers(brokerEpochs) // Simple round-robin replica assignment var leaderIndex = 0 diff --git a/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala b/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala index c39a23f2b96..c7145ceca88 100644 --- a/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala @@ -36,7 +36,7 @@ import org.easymock.EasyMock import org.junit.{After, Before} import scala.collection._ -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ abstract class AbstractCoordinatorConcurrencyTest[M <: CoordinatorMember] { diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala index e66e318f5b0..55f0d1e1bf4 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala @@ -44,7 +44,7 @@ import org.junit.Assert._ import org.junit.{After, Assert, Before, Test} import org.scalatest.Assertions.intercept -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.{Seq, mutable} import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration.Duration diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala index c0a6269fc32..e9d6780695d 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala @@ -48,7 +48,7 @@ import org.junit.Assert.{assertEquals, assertFalse, assertNull, assertTrue} import org.junit.{Before, Test} import org.scalatest.Assertions.fail -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection._ class GroupMetadataManagerTest { diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataTest.scala index a4b34c8ad6e..569a9cdf199 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataTest.scala @@ -26,7 +26,7 @@ import org.apache.kafka.common.utils.Time import org.junit.Assert._ import org.junit.{Before, Test} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** * Test group state transitions and other GroupMetadata functionality diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala index abd7fcaf3c7..1be496986d5 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala @@ -37,7 +37,7 @@ import org.easymock.{EasyMock, IAnswer} import org.junit.Assert._ import org.junit.{After, Before, Test} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.{Map, mutable} class TransactionCoordinatorConcurrencyTest extends AbstractCoordinatorConcurrencyTest[Transaction] { diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionLogTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionLogTest.scala index e9bdf3110a8..52b0c33046b 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionLogTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionLogTest.scala @@ -24,7 +24,7 @@ import org.junit.Assert.assertEquals import org.junit.Test import org.scalatest.Assertions.intercept -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class TransactionLogTest { diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerChannelManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerChannelManagerTest.scala index 838654db3a2..46d9c9def1b 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerChannelManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerChannelManagerTest.scala @@ -32,7 +32,7 @@ import org.easymock.{Capture, EasyMock} import org.junit.Assert._ import org.junit.Test -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable class TransactionMarkerChannelManagerTest { @@ -414,11 +414,11 @@ class TransactionMarkerChannelManagerTest { def shouldCreateMetricsOnStarting(): Unit = { val metrics = KafkaYammerMetrics.defaultRegistry.allMetrics.asScala - assertEquals(1, metrics - .filterKeys(_.getMBeanName == "kafka.coordinator.transaction:type=TransactionMarkerChannelManager,name=UnknownDestinationQueueSize") - .size) - assertEquals(1, metrics - .filterKeys(_.getMBeanName == "kafka.coordinator.transaction:type=TransactionMarkerChannelManager,name=LogAppendRetryQueueSize") - .size) + assertEquals(1, metrics.filter { case (k, _) => + k.getMBeanName == "kafka.coordinator.transaction:type=TransactionMarkerChannelManager,name=UnknownDestinationQueueSize" + }.size) + assertEquals(1, metrics.filter { case (k, _) => + k.getMBeanName == "kafka.coordinator.transaction:type=TransactionMarkerChannelManager,name=LogAppendRetryQueueSize" + }.size) } } diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMetadataTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMetadataTest.scala index 85ee263e174..d6dcb21886d 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMetadataTest.scala @@ -22,6 +22,7 @@ import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.RecordBatch import org.junit.Assert._ import org.junit.Test +import org.scalatest.Assertions import scala.collection.mutable @@ -46,8 +47,7 @@ class TransactionMetadataTest { topicPartitions = mutable.Set.empty, txnLastUpdateTimestamp = time.milliseconds()) - val transitMetadata = txnMetadata.prepareIncrementProducerEpoch(30000, None, - time.milliseconds()).right.get + val transitMetadata = prepareSuccessfulIncrementProducerEpoch(txnMetadata, None) txnMetadata.completeTransitionTo(transitMetadata) assertEquals(producerId, txnMetadata.producerId) assertEquals(0, txnMetadata.producerEpoch) @@ -69,7 +69,7 @@ class TransactionMetadataTest { topicPartitions = mutable.Set.empty, txnLastUpdateTimestamp = time.milliseconds()) - val transitMetadata = txnMetadata.prepareIncrementProducerEpoch(30000, None, time.milliseconds()).right.get + val transitMetadata = prepareSuccessfulIncrementProducerEpoch(txnMetadata, None) txnMetadata.completeTransitionTo(transitMetadata) assertEquals(producerId, txnMetadata.producerId) assertEquals(producerEpoch + 1, txnMetadata.producerEpoch) @@ -111,7 +111,8 @@ class TransactionMetadataTest { txnLastUpdateTimestamp = time.milliseconds()) // let new time be smaller - val transitMetadata = txnMetadata.prepareIncrementProducerEpoch(30000, Option(producerEpoch), time.milliseconds() - 1).right.get + val transitMetadata = prepareSuccessfulIncrementProducerEpoch(txnMetadata, Option(producerEpoch), + Some(time.milliseconds() - 1)) txnMetadata.completeTransitionTo(transitMetadata) assertEquals(producerId, txnMetadata.producerId) assertEquals(producerEpoch + 1, txnMetadata.producerEpoch) @@ -197,7 +198,7 @@ class TransactionMetadataTest { txnLastUpdateTimestamp = time.milliseconds()) // let new time be smaller - var transitMetadata = txnMetadata.prepareAbortOrCommit(PrepareCommit, time.milliseconds() - 1) + val transitMetadata = txnMetadata.prepareAbortOrCommit(PrepareCommit, time.milliseconds() - 1) txnMetadata.completeTransitionTo(transitMetadata) assertEquals(PrepareCommit, txnMetadata.state) assertEquals(producerId, txnMetadata.producerId) @@ -223,7 +224,7 @@ class TransactionMetadataTest { txnLastUpdateTimestamp = time.milliseconds()) // let new time be smaller - var transitMetadata = txnMetadata.prepareAbortOrCommit(PrepareAbort, time.milliseconds() - 1) + val transitMetadata = txnMetadata.prepareAbortOrCommit(PrepareAbort, time.milliseconds() - 1) txnMetadata.completeTransitionTo(transitMetadata) assertEquals(PrepareAbort, txnMetadata.state) assertEquals(producerId, txnMetadata.producerId) @@ -249,7 +250,7 @@ class TransactionMetadataTest { txnLastUpdateTimestamp = time.milliseconds()) // let new time be smaller - var transitMetadata = txnMetadata.prepareComplete(time.milliseconds() - 1) + val transitMetadata = txnMetadata.prepareComplete(time.milliseconds() - 1) txnMetadata.completeTransitionTo(transitMetadata) assertEquals(CompleteCommit, txnMetadata.state) assertEquals(producerId, txnMetadata.producerId) @@ -275,7 +276,7 @@ class TransactionMetadataTest { txnLastUpdateTimestamp = time.milliseconds()) // let new time be smaller - var transitMetadata = txnMetadata.prepareComplete(time.milliseconds() - 1) + val transitMetadata = txnMetadata.prepareComplete(time.milliseconds() - 1) txnMetadata.completeTransitionTo(transitMetadata) assertEquals(CompleteAbort, txnMetadata.state) assertEquals(producerId, txnMetadata.producerId) @@ -386,8 +387,7 @@ class TransactionMetadataTest { topicPartitions = mutable.Set.empty, txnLastUpdateTimestamp = time.milliseconds()) - val transitMetadata = txnMetadata.prepareIncrementProducerEpoch(30000, Some(producerEpoch), - time.milliseconds()).right.get + val transitMetadata = prepareSuccessfulIncrementProducerEpoch(txnMetadata, Some(producerEpoch)) txnMetadata.completeTransitionTo(transitMetadata) assertEquals(producerId, txnMetadata.producerId) assertEquals(0, txnMetadata.producerEpoch) @@ -409,8 +409,7 @@ class TransactionMetadataTest { topicPartitions = mutable.Set.empty, txnLastUpdateTimestamp = time.milliseconds()) - val transitMetadata = txnMetadata.prepareIncrementProducerEpoch(30000, Some(producerEpoch), - time.milliseconds()).right.get + val transitMetadata = prepareSuccessfulIncrementProducerEpoch(txnMetadata, Some(producerEpoch)) txnMetadata.completeTransitionTo(transitMetadata) assertEquals(producerId, txnMetadata.producerId) assertEquals(producerEpoch + 1, txnMetadata.producerEpoch) @@ -433,8 +432,7 @@ class TransactionMetadataTest { topicPartitions = mutable.Set.empty, txnLastUpdateTimestamp = time.milliseconds()) - val transitMetadata = txnMetadata.prepareIncrementProducerEpoch(30000, Some(lastProducerEpoch), - time.milliseconds()).right.get + val transitMetadata = prepareSuccessfulIncrementProducerEpoch(txnMetadata, Some(lastProducerEpoch)) txnMetadata.completeTransitionTo(transitMetadata) assertEquals(producerId, txnMetadata.producerId) assertEquals(producerEpoch, txnMetadata.producerEpoch) @@ -457,9 +455,9 @@ class TransactionMetadataTest { topicPartitions = mutable.Set.empty, txnLastUpdateTimestamp = time.milliseconds()) - val error = txnMetadata.prepareIncrementProducerEpoch(30000, Some((lastProducerEpoch - 1).toShort), - time.milliseconds()).left.get - assertEquals(Errors.INVALID_PRODUCER_EPOCH, error) + val result = txnMetadata.prepareIncrementProducerEpoch(30000, Some((lastProducerEpoch - 1).toShort), + time.milliseconds()) + assertEquals(Left(Errors.INVALID_PRODUCER_EPOCH), result) } private def testRotateProducerIdInOngoingState(state: TransactionState): Unit = { @@ -479,5 +477,12 @@ class TransactionMetadataTest { txnMetadata.prepareProducerIdRotation(newProducerId, 30000, time.milliseconds(), recordLastEpoch = false) } + private def prepareSuccessfulIncrementProducerEpoch(txnMetadata: TransactionMetadata, + expectedProducerEpoch: Option[Short], + now: Option[Long] = None): TxnTransitMetadata = { + val result = txnMetadata.prepareIncrementProducerEpoch(30000, expectedProducerEpoch, + now.getOrElse(time.milliseconds())) + result.getOrElse(Assertions.fail(s"prepareIncrementProducerEpoch failed with $result")) + } } diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala index fcf98906f42..e02f46d6d4b 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala @@ -39,7 +39,7 @@ import org.junit.Assert.{assertEquals, assertFalse, assertTrue} import org.junit.{After, Before, Test} import org.scalatest.Assertions.fail -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.{Map, mutable} class TransactionStateManagerTest { diff --git a/core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala b/core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala index c4fea92d43d..11f148837ee 100644 --- a/core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala +++ b/core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala @@ -22,7 +22,7 @@ import java.util.Properties import kafka.server.KafkaConfig import kafka.utils.{Logging, TestUtils} -import scala.collection.JavaConverters.mapAsScalaMapConverter +import scala.jdk.CollectionConverters._ import org.scalatest.Assertions.fail import org.junit.{Before, Test} import com.yammer.metrics.core.Gauge @@ -124,7 +124,7 @@ class MetricsDuringTopicCreationDeletionTest extends KafkaServerTestHarness with private def getGauge(metricName: String) = { KafkaYammerMetrics.defaultRegistry.allMetrics.asScala - .filterKeys(k => k.getName.endsWith(metricName)) + .filter { case (k, _) => k.getName.endsWith(metricName) } .headOption .getOrElse { fail( "Unable to find metric " + metricName ) } ._2.asInstanceOf[Gauge[Int]] diff --git a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala index 468fe31d41d..6fcdd4c6958 100755 --- a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala @@ -17,12 +17,12 @@ package kafka.integration -import org.apache.kafka.common.config.ConfigException +import org.apache.kafka.common.config.{ConfigException, ConfigResource} import org.junit.{After, Before, Test} import scala.util.Random -import scala.collection.JavaConverters._ -import scala.collection.Seq +import scala.jdk.CollectionConverters._ +import scala.collection.{Map, Seq} import org.apache.log4j.{Level, Logger} import java.util.Properties import java.util.concurrent.ExecutionException @@ -36,7 +36,7 @@ import org.apache.kafka.common.errors.TimeoutException import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.serialization.StringDeserializer -import org.apache.kafka.clients.admin.{Admin, AdminClientConfig} +import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, AlterConfigsResult, Config, ConfigEntry} import org.junit.Assert._ import org.scalatest.Assertions.intercept @@ -334,7 +334,7 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness { val adminClient = createAdminClient() val newProps = new Properties newProps.put(KafkaConfig.UncleanLeaderElectionEnableProp, "true") - TestUtils.alterTopicConfigs(adminClient, topic, newProps).all.get + alterTopicConfigs(adminClient, topic, newProps).all.get adminClient.close() // wait until new leader is (uncleanly) elected @@ -347,6 +347,13 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness { assertEquals(List("first", "third"), consumeAllMessages(topic, 2)) } + 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 newConfig = new Config(configEntries) + val configs = Map(new ConfigResource(ConfigResource.Type.TOPIC, topic) -> newConfig).asJava + adminClient.alterConfigs(configs) + } + private def createAdminClient(): Admin = { val config = new Properties val bootstrapServers = TestUtils.bootstrapServers(servers, new ListenerName("PLAINTEXT")) diff --git a/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala b/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala index 8e4093ea83b..6e7ae74fca6 100755 --- a/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala +++ b/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala @@ -30,7 +30,7 @@ import java.util.{Collection, Properties} import kafka.server.{BrokerTopicStats, FetchLogEnd, LogDirFailureChannel} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ @RunWith(value = classOf[Parameterized]) class BrokerCompressionTest(messageCompression: String, brokerCompression: String) { diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala index 52c4d7e53f5..1ac9e62a7b1 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -28,8 +28,8 @@ import org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS import org.junit.Assert._ import org.junit.{After, Test} -import scala.collection.JavaConverters.mapAsScalaMapConverter -import scala.collection.{Iterable, JavaConverters, Seq} +import scala.collection.{Iterable, Seq} +import scala.jdk.CollectionConverters._ /** * This is an integration test that tests the fully integrated log cleaner @@ -90,7 +90,7 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K private def getGauge[T](filter: MetricName => Boolean): Gauge[T] = { KafkaYammerMetrics.defaultRegistry.allMetrics.asScala - .filterKeys(filter(_)) + .filter { case (k, _) => filter(k) } .headOption .getOrElse { fail(s"Unable to find metric") } .asInstanceOf[(Any, Gauge[Any])] @@ -178,7 +178,6 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K } private def readFromLog(log: Log): Iterable[(Int, Int)] = { - import JavaConverters._ for (segment <- log.logSegments; record <- segment.log.records.asScala) yield { val key = TestUtils.readString(record.key).toInt val value = TestUtils.readString(record.value).toInt diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala index 0232e5772d6..485111164e9 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala @@ -27,6 +27,7 @@ import org.junit.runners.Parameterized import org.junit.runners.Parameterized.Parameters import scala.collection._ +import scala.jdk.CollectionConverters._ /** * This is an integration test that tests the fully integrated log cleaner @@ -98,8 +99,6 @@ class LogCleanerLagIntegrationTest(compressionCodecName: String) extends Abstrac } private def readFromLog(log: Log): Iterable[(Int, Int)] = { - import JavaConverters._ - for (segment <- log.logSegments; record <- segment.log.records.asScala) yield { val key = TestUtils.readString(record.key).toInt val value = TestUtils.readString(record.value).toInt diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala index 5e2620ca0b7..e5cf6ee2c84 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala @@ -640,11 +640,11 @@ class LogCleanerManagerTest extends Logging { private def createCleanerManager(log: Log): LogCleanerManager = { val logs = new Pool[TopicPartition, Log]() logs.put(topicPartition, log) - new LogCleanerManager(Array(logDir), logs, null) + new LogCleanerManager(Seq(logDir), logs, null) } private def createCleanerManagerMock(pool: Pool[TopicPartition, Log]): LogCleanerManagerMock = { - new LogCleanerManagerMock(Array(logDir), pool, null) + new LogCleanerManagerMock(Seq(logDir), pool, null) } private def createLog(segmentSize: Int, diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala index 815405319df..f89a26a1335 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala @@ -34,6 +34,7 @@ import org.junit.runners.Parameterized import org.junit.runners.Parameterized.Parameters import scala.collection._ +import scala.jdk.CollectionConverters._ /** * This is an integration test that tests the fully integrated log cleaner @@ -296,7 +297,6 @@ class LogCleanerParameterizedIntegrationTest(compressionCodec: String) extends A } private def readFromLog(log: Log): Iterable[(Int, String, Long)] = { - import JavaConverters._ for (segment <- log.logSegments; deepLogEntry <- segment.log.records.asScala) yield { val key = TestUtils.readString(deepLogEntry.key).toInt val value = TestUtils.readString(deepLogEntry.value) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 9c0108eb75b..cf0fb54959d 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -35,7 +35,7 @@ import org.junit.Assert._ import org.junit.{After, Test} import org.scalatest.Assertions.{assertThrows, fail, intercept} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection._ /** @@ -1517,8 +1517,6 @@ class LogCleanerTest { */ @Test def testClientHandlingOfCorruptMessageSet(): Unit = { - import JavaConverters._ - val keys = 1 until 10 val offset = 50 val set = keys zip (offset until offset + keys.size) diff --git a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala index a29e7e5415c..5595c9018d2 100644 --- a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala @@ -26,7 +26,7 @@ import org.apache.kafka.common.utils.{MockTime, Time, Utils} import org.junit.Assert._ import org.junit.{After, Before, Test} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection._ class LogSegmentTest { diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index 6d147b4acab..762edd20900 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -46,7 +46,7 @@ import org.junit.{After, Before, Test} import org.scalatest.Assertions import scala.collection.{Iterable, mutable} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable.ListBuffer import org.scalatest.Assertions.{assertThrows, intercept, withClue} @@ -1373,9 +1373,9 @@ class LogTest { producerId: Long, epoch: Short, offset: Long = 0L, - coordinatorEpoch: Int = 0, + coordinatorEpoch: Int, partitionLeaderEpoch: Int = 0, - timestamp: Long = mockTime.milliseconds()): MemoryRecords = { + timestamp: Long): MemoryRecords = { val marker = new EndTransactionMarker(controlRecordType, coordinatorEpoch) MemoryRecords.withEndTransactionMarker(offset, timestamp, partitionLeaderEpoch, producerId, epoch, marker) } @@ -3119,7 +3119,7 @@ class LogTest { def testParseTopicPartitionNameForMissingPartition(): Unit = { val topic = "test_topic" val partition = "" - val dir = new File(logDir + topicPartitionName(topic, partition)) + val dir = new File(logDir.getPath + topicPartitionName(topic, partition)) try { Log.parseTopicPartitionName(dir) fail("KafkaException should have been thrown for dir: " + dir.getCanonicalPath) @@ -3159,14 +3159,14 @@ class LogTest { @Test def testParseTopicPartitionNameForExistingInvalidDir(): Unit = { - val dir1 = new File(logDir + "/non_kafka_dir") + val dir1 = new File(logDir.getPath + "/non_kafka_dir") try { Log.parseTopicPartitionName(dir1) fail("KafkaException should have been thrown for dir: " + dir1.getCanonicalPath) } catch { case _: KafkaException => // should only throw KafkaException } - val dir2 = new File(logDir + "/non_kafka_dir-delete") + val dir2 = new File(logDir.getPath + "/non_kafka_dir-delete") try { Log.parseTopicPartitionName(dir2) fail("KafkaException should have been thrown for dir: " + dir2.getCanonicalPath) diff --git a/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala b/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala index 5de188f8214..7d2738b91ae 100644 --- a/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala @@ -35,7 +35,7 @@ import org.junit.Assert._ import org.junit.Test import org.scalatest.Assertions.{assertThrows, intercept} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class LogValidatorTest { @@ -1287,7 +1287,6 @@ class LogValidatorTest { RecordBatch.MAGIC_VALUE_V0, CompressionType.GZIP, CompressionType.GZIP) } - e.recordErrors.foreach(e => println(e.batchIndex + " " + e.message)) assertTrue(e.invalidException.isInstanceOf[InvalidRecordException]) assertTrue(e.recordErrors.nonEmpty) // recordsWithInvalidInnerMagic creates 20 records diff --git a/core/src/test/scala/unit/kafka/log/OffsetMapTest.scala b/core/src/test/scala/unit/kafka/log/OffsetMapTest.scala index 5fa9389442d..4ac0e7b1124 100644 --- a/core/src/test/scala/unit/kafka/log/OffsetMapTest.scala +++ b/core/src/test/scala/unit/kafka/log/OffsetMapTest.scala @@ -82,7 +82,7 @@ object OffsetMapTest { val start = System.nanoTime val map = test.validateMap(size, load) val ellapsedMs = (System.nanoTime - start) / 1000.0 / 1000.0 - println(map.size + " entries in map of size " + map.slots + " in " + ellapsedMs + " ms") + println(s"${map.size} entries in map of size ${map.slots} in $ellapsedMs ms") println("Collision rate: %.1f%%".format(100*map.collisionRate)) } } diff --git a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala index 80bcbc71034..7d8643fff08 100644 --- a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala +++ b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala @@ -29,7 +29,7 @@ import kafka.server._ import kafka.utils._ import scala.collection._ -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import kafka.log.LogConfig import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.metrics.JmxReporter diff --git a/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala b/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala index 7f570e62398..ce19491bbe5 100644 --- a/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala +++ b/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala @@ -37,7 +37,7 @@ import org.easymock.EasyMock._ import org.junit.Assert._ import org.junit._ -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class RequestChannelTest { diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 3b84d6f0e94..57ecf280bd9 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -54,7 +54,7 @@ import org.junit.Assert._ import org.junit._ import org.scalatest.Assertions.fail -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.util.control.ControlThrowable @@ -1016,7 +1016,7 @@ class SocketServerTest { } @Test - def testClientDisconnectionWithOutstandingReceivesProcessedUntilFailedSend() { + def testClientDisconnectionWithOutstandingReceivesProcessedUntilFailedSend(): Unit = { val serverMetrics = new Metrics @volatile var selector: TestableSelector = null val overrideServer = new SocketServer(KafkaConfig.fromProps(props), serverMetrics, Time.SYSTEM, credentialProvider) { @@ -1135,7 +1135,7 @@ class SocketServerTest { // legacy metrics not tagged val yammerMetricsNames = KafkaYammerMetrics.defaultRegistry.allMetrics.asScala - .filterKeys(_.getType.equals("Processor")) + .filter { case (k, _) => k.getType.equals("Processor") } .collect { case (k, _: Gauge[_]) => k } assertFalse(yammerMetricsNames.isEmpty) @@ -1683,8 +1683,9 @@ class SocketServerTest { private def verifyAcceptorBlockedPercent(listenerName: String, expectBlocked: Boolean): Unit = { val blockedPercentMetricMBeanName = "kafka.network:type=Acceptor,name=AcceptorBlockedPercent,listener=PLAINTEXT" - val blockedPercentMetrics = KafkaYammerMetrics.defaultRegistry.allMetrics.asScala - .filterKeys(_.getMBeanName == blockedPercentMetricMBeanName).values + val blockedPercentMetrics = KafkaYammerMetrics.defaultRegistry.allMetrics.asScala.filter { case (k, _) => + k.getMBeanName == blockedPercentMetricMBeanName + }.values assertEquals(1, blockedPercentMetrics.size) val blockedPercentMetric = blockedPercentMetrics.head.asInstanceOf[Meter] val blockedPercent = blockedPercentMetric.meanRate diff --git a/core/src/test/scala/unit/kafka/security/auth/ZkAuthorizationTest.scala b/core/src/test/scala/unit/kafka/security/auth/ZkAuthorizationTest.scala index 2ef3354fbe3..ed00823b243 100644 --- a/core/src/test/scala/unit/kafka/security/auth/ZkAuthorizationTest.scala +++ b/core/src/test/scala/unit/kafka/security/auth/ZkAuthorizationTest.scala @@ -37,7 +37,7 @@ import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.Time -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.Seq class ZkAuthorizationTest extends ZooKeeperTestHarness with Logging { @@ -131,7 +131,7 @@ class ZkAuthorizationTest extends ZooKeeperTestHarness with Logging { // Test that can update persistent nodes val updatedAssignment = assignment - new TopicPartition(topic1, 2) - zkClient.setTopicAssignment(topic1, updatedAssignment.mapValues { case (v) => ReplicaAssignment(v, List(), List()) }.toMap) + zkClient.setTopicAssignment(topic1, updatedAssignment.map { case (k, v) => k -> ReplicaAssignment(v, List(), List()) }) assertEquals(updatedAssignment.size, zkClient.getTopicPartitionCount(topic1).get) } diff --git a/core/src/test/scala/unit/kafka/security/authorizer/AclAuthorizerTest.scala b/core/src/test/scala/unit/kafka/security/authorizer/AclAuthorizerTest.scala index e885bf5ffcc..38b257a4ce5 100644 --- a/core/src/test/scala/unit/kafka/security/authorizer/AclAuthorizerTest.scala +++ b/core/src/test/scala/unit/kafka/security/authorizer/AclAuthorizerTest.scala @@ -50,7 +50,7 @@ import org.junit.Assert._ import org.junit.{After, Before, Test} import org.scalatest.Assertions.intercept -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable import scala.compat.java8.OptionConverters._ diff --git a/core/src/test/scala/unit/kafka/security/authorizer/AclEntryTest.scala b/core/src/test/scala/unit/kafka/security/authorizer/AclEntryTest.scala index cb0db78f115..261b24fec56 100644 --- a/core/src/test/scala/unit/kafka/security/authorizer/AclEntryTest.scala +++ b/core/src/test/scala/unit/kafka/security/authorizer/AclEntryTest.scala @@ -25,7 +25,7 @@ import org.apache.kafka.common.security.auth.KafkaPrincipal import org.junit.{Assert, Test} import org.scalatestplus.junit.JUnitSuite -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class AclEntryTest extends JUnitSuite { diff --git a/core/src/test/scala/unit/kafka/security/token/delegation/DelegationTokenManagerTest.scala b/core/src/test/scala/unit/kafka/security/token/delegation/DelegationTokenManagerTest.scala index 14c2cd586d7..6d5aaa4cd81 100644 --- a/core/src/test/scala/unit/kafka/security/token/delegation/DelegationTokenManagerTest.scala +++ b/core/src/test/scala/unit/kafka/security/token/delegation/DelegationTokenManagerTest.scala @@ -43,7 +43,7 @@ import org.apache.kafka.server.authorizer._ import org.junit.Assert._ import org.junit.{After, Before, Test} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.compat.java8.OptionConverters._ import scala.collection.mutable.Buffer diff --git a/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala index 007add7cbff..3685500fd5d 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala @@ -21,7 +21,7 @@ import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.requests.{ApiVersionsRequest, ApiVersionsResponse} import org.junit.Assert._ -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ abstract class AbstractApiVersionsRequestTest extends BaseRequestTest { diff --git a/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala index 8ca2c707b2d..42d5d021c82 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala @@ -29,7 +29,7 @@ import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests._ import org.junit.Assert.{assertEquals, assertFalse, assertNotNull, assertTrue} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ abstract class AbstractCreateTopicsRequestTest extends BaseRequestTest { diff --git a/core/src/test/scala/unit/kafka/server/AbstractFetcherManagerTest.scala b/core/src/test/scala/unit/kafka/server/AbstractFetcherManagerTest.scala index 211bf56d6e5..260bbd79fce 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractFetcherManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractFetcherManagerTest.scala @@ -25,7 +25,7 @@ import org.easymock.EasyMock import org.junit.{Before, Test} import org.junit.Assert._ -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class AbstractFetcherManagerTest { @@ -35,7 +35,7 @@ class AbstractFetcherManagerTest { } private def getMetricValue(name: String): Any = { - KafkaYammerMetrics.defaultRegistry.allMetrics.asScala.filterKeys(_.getName == name).values.headOption.get. + KafkaYammerMetrics.defaultRegistry.allMetrics.asScala.filter { case (k, _) => k.getName == name }.values.headOption.get. asInstanceOf[Gauge[Int]].value() } diff --git a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala index 4fca8a444c1..4a5633d1009 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala @@ -38,7 +38,7 @@ import org.apache.kafka.common.utils.Time import org.junit.Assert._ import org.junit.{Before, Test} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.{mutable, Map, Set} import scala.util.Random import org.scalatest.Assertions.assertThrows @@ -720,7 +720,7 @@ class AbstractFetcherThreadTest { val fetcher = new MockFetcherThread { override def fetchEpochEndOffsets(partitions: Map[TopicPartition, EpochData]): Map[TopicPartition, EpochEndOffset] = { val unrequestedTp = new TopicPartition("topic2", 0) - super.fetchEpochEndOffsets(partitions) + (unrequestedTp -> new EpochEndOffset(0, 0)) + super.fetchEpochEndOffsets(partitions).toMap + (unrequestedTp -> new EpochEndOffset(0, 0)) } } diff --git a/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestTest.scala b/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestTest.scala index a314b0de8fa..20ba68c890b 100644 --- a/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestTest.scala @@ -25,7 +25,7 @@ import org.apache.kafka.common.requests.{AddPartitionsToTxnRequest, AddPartition import org.junit.Assert._ import org.junit.{Before, Test} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class AddPartitionsToTxnRequestTest extends BaseRequestTest { private val topic1 = "foobartopic" diff --git a/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala index cf436a9fc73..236e60ed6ba 100644 --- a/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala @@ -26,7 +26,7 @@ import org.apache.kafka.common.requests.{AlterReplicaLogDirsRequest, AlterReplic import org.junit.Assert._ import org.junit.Test -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable import scala.util.Random diff --git a/core/src/test/scala/unit/kafka/server/BrokerEpochIntegrationTest.scala b/core/src/test/scala/unit/kafka/server/BrokerEpochIntegrationTest.scala index 23836dd66d8..fd776622b8e 100755 --- a/core/src/test/scala/unit/kafka/server/BrokerEpochIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/server/BrokerEpochIntegrationTest.scala @@ -35,7 +35,7 @@ import org.apache.kafka.common.utils.Time import org.junit.Assert._ import org.junit.{After, Before, Test} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class BrokerEpochIntegrationTest extends ZooKeeperTestHarness { val brokerId1 = 0 @@ -120,7 +120,7 @@ class BrokerEpochIntegrationTest extends ZooKeeperTestHarness { val nodes = brokerAndEpochs.keys.map(_.node(listenerName)) val controllerContext = new ControllerContext - controllerContext.setLiveBrokerAndEpochs(brokerAndEpochs) + controllerContext.setLiveBrokers(brokerAndEpochs) val metrics = new Metrics val controllerChannelManager = new ControllerChannelManager(controllerContext, controllerConfig, Time.SYSTEM, metrics, new StateChangeLogger(controllerId, inControllerContext = true, None)) diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala index a7041633027..9820933ce2c 100644 --- a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala @@ -91,7 +91,7 @@ class ClientQuotaManagerTest { clientMetrics.updateQuota(client2.configUser, client2.configClientId, client2.sanitizedConfigClientId, Some(new Quota(4000, true))) assertEquals("Default producer quota should be " + config.quotaBytesPerSecondDefault, - config.quotaBytesPerSecondDefault, clientMetrics.quota(randomClient.user, randomClient.clientId).bound, 0.0) + config.quotaBytesPerSecondDefault.toDouble, clientMetrics.quota(randomClient.user, randomClient.clientId).bound, 0.0) assertEquals("Should return the overridden value (2000)", 2000, clientMetrics.quota(client1.user, client1.clientId).bound, 0.0) assertEquals("Should return the overridden value (4000)", 4000, clientMetrics.quota(client2.user, client2.clientId).bound, 0.0) @@ -193,10 +193,11 @@ class ClientQuotaManagerTest { } private def checkQuota(quotaManager: ClientQuotaManager, user: String, clientId: String, expectedBound: Long, value: Int, expectThrottle: Boolean): Unit = { - assertEquals(expectedBound, quotaManager.quota(user, clientId).bound, 0.0) + assertEquals(expectedBound.toDouble, quotaManager.quota(user, clientId).bound, 0.0) val session = Session(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, user), InetAddress.getLocalHost) val expectedMaxValueInQuotaWindow = - if (expectedBound < Long.MaxValue) config.quotaWindowSizeSeconds * (config.numQuotaSamples - 1) * expectedBound else Double.MaxValue + if (expectedBound < Long.MaxValue) config.quotaWindowSizeSeconds * (config.numQuotaSamples - 1) * expectedBound.toDouble + else Double.MaxValue assertEquals(expectedMaxValueInQuotaWindow, quotaManager.getMaxValueInQuotaWindow(session, clientId), 0.01) val throttleTimeMs = maybeRecord(quotaManager, user, clientId, value * config.numQuotaSamples) diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala index 047ff710523..623462d3f65 100644 --- a/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala @@ -26,7 +26,7 @@ import org.junit.Test import java.util.concurrent.{ExecutionException, TimeUnit} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class ClientQuotasRequestTest extends BaseRequestTest { private val ConsumerByteRateProp = DynamicConfig.Client.ConsumerByteRateOverrideProp @@ -272,7 +272,7 @@ class ClientQuotasRequestTest extends BaseRequestTest { def testDescribeClientQuotasMatchPartial(): Unit = { setupDescribeClientQuotasMatchTest() - def testMatchEntities(filter: ClientQuotaFilter, expectedMatchSize: Int, partition: ClientQuotaEntity => Boolean) { + def testMatchEntities(filter: ClientQuotaFilter, expectedMatchSize: Int, partition: ClientQuotaEntity => Boolean): Unit = { val result = describeClientQuotas(filter) val (expectedMatches, expectedNonMatches) = matchEntities.partition(e => partition(e._1)) assertEquals(expectedMatchSize, expectedMatches.size) // for test verification @@ -359,7 +359,7 @@ class ClientQuotasRequestTest extends BaseRequestTest { } @Test - def testClientQuotasUnsupportedEntityTypes() { + def testClientQuotasUnsupportedEntityTypes(): Unit = { val entity = new ClientQuotaEntity(Map(("other" -> "name")).asJava) try { verifyDescribeEntityQuotas(entity, Map()) diff --git a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala index 37794ad3d52..245f13f9aca 100644 --- a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala @@ -26,7 +26,7 @@ import org.apache.kafka.common.requests.CreateTopicsRequest import org.junit.Assert._ import org.junit.Test -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class CreateTopicsRequestTest extends AbstractCreateTopicsRequestTest { diff --git a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala index aef408041c2..e97c04bb768 100644 --- a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala +++ b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala @@ -27,7 +27,7 @@ import org.apache.kafka.server.policy.CreateTopicPolicy import org.apache.kafka.server.policy.CreateTopicPolicy.RequestMetadata import org.junit.Test -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class CreateTopicsRequestWithPolicyTest extends AbstractCreateTopicsRequestTest { import CreateTopicsRequestWithPolicyTest._ diff --git a/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala b/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala index 212ef80e099..c29dfec7b4d 100644 --- a/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala +++ b/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala @@ -29,7 +29,7 @@ import org.junit.{After, Before, Test} import org.junit.Assert._ import org.scalatest.Assertions.intercept -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class DelayedOperationTest { diff --git a/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsOnPlainTextTest.scala b/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsOnPlainTextTest.scala index e6bdd0e9a6e..6c01f21327d 100644 --- a/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsOnPlainTextTest.scala +++ b/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsOnPlainTextTest.scala @@ -24,7 +24,7 @@ import org.apache.kafka.common.errors.UnsupportedByAuthenticationException import org.junit.{After, Before, Test} import org.scalatest.Assertions.intercept -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.concurrent.ExecutionException class DelegationTokenRequestsOnPlainTextTest extends BaseRequestTest { diff --git a/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsTest.scala b/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsTest.scala index b1fc8e08baa..178bd277470 100644 --- a/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsTest.scala +++ b/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsTest.scala @@ -28,7 +28,7 @@ import org.junit.Assert._ import org.junit.{After, Before, Test} import org.scalatest.Assertions.intercept -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.concurrent.ExecutionException class DelegationTokenRequestsTest extends BaseRequestTest with SaslSetup { diff --git a/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsWithDisableTokenFeatureTest.scala b/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsWithDisableTokenFeatureTest.scala index 30a5241e204..4d26aef61ae 100644 --- a/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsWithDisableTokenFeatureTest.scala +++ b/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsWithDisableTokenFeatureTest.scala @@ -26,7 +26,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol import org.junit.{After, Before, Test} import org.scalatest.Assertions.intercept -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.concurrent.ExecutionException class DelegationTokenRequestsWithDisableTokenFeatureTest extends BaseRequestTest with SaslSetup { diff --git a/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestTest.scala b/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestTest.scala index 987c2bf4d76..9a18582c12b 100644 --- a/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestTest.scala @@ -27,7 +27,7 @@ import org.apache.kafka.common.requests.{DeleteTopicsRequest, DeleteTopicsRespon import org.junit.Assert._ import org.junit.Test -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class DeleteTopicsRequestTest extends BaseRequestTest { diff --git a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala index 31df6c772b3..164bacb9fb8 100755 --- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala @@ -33,7 +33,7 @@ import org.junit.Assert._ import org.junit.Test import org.scalatest.Assertions.intercept -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.Set class DynamicBrokerConfigTest { diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index 04afa41f2dd..cf518659a0e 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -32,7 +32,7 @@ import kafka.zk.ConfigEntityChangeNotificationZNode import org.apache.kafka.common.TopicPartition import scala.collection.{Map, Seq} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class DynamicConfigChangeTest extends KafkaServerTestHarness { def generateConfigs = List(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, zkConnect))) diff --git a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala index b8374f2435d..73527a67b21 100755 --- a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala @@ -34,7 +34,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol import org.junit.Assert._ import org.junit.Test -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class EdgeCaseRequestTest extends KafkaServerTestHarness { diff --git a/core/src/test/scala/unit/kafka/server/FetchRequestMaxBytesTest.scala b/core/src/test/scala/unit/kafka/server/FetchRequestMaxBytesTest.scala index 1ead2acf594..8203dfcfa55 100644 --- a/core/src/test/scala/unit/kafka/server/FetchRequestMaxBytesTest.scala +++ b/core/src/test/scala/unit/kafka/server/FetchRequestMaxBytesTest.scala @@ -28,7 +28,7 @@ import org.apache.kafka.common.requests.FetchRequest.PartitionData import org.apache.kafka.common.requests.{FetchRequest, FetchResponse} import org.junit.{Assert, Test} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** * This test verifies that the KIP-541 broker-level FetchMaxBytes configuration is honored. diff --git a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala index af6f8984428..048794bd12f 100644 --- a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala @@ -33,7 +33,7 @@ import org.apache.kafka.common.{IsolationLevel, TopicPartition} import org.junit.Assert._ import org.junit.Test -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.Seq import scala.util.Random diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 1b4fe54d639..5364ee8651d 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -66,7 +66,7 @@ import org.easymock.{Capture, EasyMock, IAnswer} import org.junit.Assert.{assertArrayEquals, assertEquals, assertNull, assertTrue} import org.junit.{After, Test} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.{Map, Seq, mutable} class KafkaApisTest { diff --git a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala index 01423a39f61..a3eb5d75728 100755 --- a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala @@ -19,7 +19,7 @@ package kafka.server import org.apache.kafka.common.TopicPartition -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import kafka.api.LeaderAndIsr import org.apache.kafka.common.requests._ import org.junit.Assert._ @@ -134,7 +134,7 @@ class LeaderElectionTest extends ZooKeeperTestHarness { val nodes = brokerAndEpochs.keys.map(_.node(listenerName)) val controllerContext = new ControllerContext - controllerContext.setLiveBrokerAndEpochs(brokerAndEpochs) + controllerContext.setLiveBrokers(brokerAndEpochs) val metrics = new Metrics val controllerChannelManager = new ControllerChannelManager(controllerContext, controllerConfig, Time.SYSTEM, metrics, new StateChangeLogger(controllerId, inControllerContext = true, None)) diff --git a/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala b/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala index 6e579c1f6b7..920176ed647 100644 --- a/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala @@ -25,7 +25,7 @@ import org.apache.kafka.common.{IsolationLevel, TopicPartition} import org.junit.Assert._ import org.junit.Test -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class ListOffsetsRequestTest extends BaseRequestTest { diff --git a/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala b/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala index f00008811ad..8ac65b46b17 100644 --- a/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala @@ -33,7 +33,7 @@ import org.junit.Assert.{assertEquals, assertFalse, assertTrue} import org.junit.{Before, Test} import org.scalatest.Assertions.fail -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** * Test whether clients can producer and consume when there is log directory failure diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala index ae0d8b487f5..bcb637829d8 100755 --- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala @@ -31,7 +31,7 @@ import org.easymock.{EasyMock, IAnswer} import org.junit.Assert._ import org.junit.Test -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class LogOffsetTest extends BaseRequestTest { diff --git a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala index ce804e81ed2..d25293644a8 100644 --- a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala +++ b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala @@ -29,7 +29,7 @@ import org.junit.Test import org.junit.Assert._ import org.scalatest.Assertions -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class MetadataCacheTest { val brokerEpoch = 0L diff --git a/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala b/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala index ecb3f7e3af8..a754d0591b9 100644 --- a/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala @@ -31,7 +31,7 @@ import org.junit.Assert._ import org.junit.{Before, Test} import org.scalatest.Assertions.intercept -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.Seq class MetadataRequestTest extends BaseRequestTest { diff --git a/core/src/test/scala/unit/kafka/server/OffsetsForLeaderEpochRequestTest.scala b/core/src/test/scala/unit/kafka/server/OffsetsForLeaderEpochRequestTest.scala index d0d84aaad25..8fc9cdbab11 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetsForLeaderEpochRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetsForLeaderEpochRequestTest.scala @@ -25,7 +25,7 @@ import org.apache.kafka.common.requests.{OffsetsForLeaderEpochRequest, OffsetsFo import org.junit.Assert._ import org.junit.Test -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class OffsetsForLeaderEpochRequestTest extends BaseRequestTest { diff --git a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala index da3f419f76c..d3a871a0439 100644 --- a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala @@ -32,7 +32,7 @@ import org.junit.Assert._ import org.junit.Test import org.scalatest.Assertions.fail -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** * Subclasses of `BaseProduceSendRequestTest` exercise the producer and produce request/response. This class diff --git a/core/src/test/scala/unit/kafka/server/ReplicaAlterLogDirsThreadTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaAlterLogDirsThreadTest.scala index 17fed26a16a..482eafd8c7a 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaAlterLogDirsThreadTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaAlterLogDirsThreadTest.scala @@ -32,7 +32,7 @@ import org.easymock.{Capture, CaptureType, EasyMock, IExpectationSetters} import org.junit.Assert._ import org.junit.Test -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.{Map, Seq} class ReplicaAlterLogDirsThreadTest { diff --git a/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala index 65a8d57fabb..1ab2054137e 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala @@ -37,7 +37,7 @@ import org.easymock.{Capture, CaptureType} import org.junit.Assert._ import org.junit.{After, Test} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.{Map, mutable} class ReplicaFetcherThreadTest { diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala index 7d0d17c3f83..21b8c1a6c8e 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala @@ -33,7 +33,7 @@ import EasyMock._ import org.junit.Assert._ import org.junit.{After, Test} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class ReplicaManagerQuotasTest { val configs = TestUtils.createBrokerConfigs(2, TestUtils.MockZkConnect).map(KafkaConfig.fromProps(_, new Properties())) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 4b600204bcf..0aca3488c99 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -51,7 +51,7 @@ import org.junit.Assert._ import org.junit.{After, Before, Test} import org.mockito.Mockito -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.{Map, Seq} class ReplicaManagerTest { diff --git a/core/src/test/scala/unit/kafka/server/ReplicationQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicationQuotaManagerTest.scala index 16c1ab7e2c7..c8c508108cd 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicationQuotaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicationQuotaManagerTest.scala @@ -25,7 +25,7 @@ import org.apache.kafka.common.utils.MockTime import org.junit.Assert.{assertEquals, assertFalse, assertTrue} import org.junit.{After, Test} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class ReplicationQuotaManagerTest { private val time = new MockTime diff --git a/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala index 45836651927..24b33e8a618 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala @@ -31,7 +31,7 @@ import org.apache.kafka.common.TopicPartition import org.junit.Assert._ import org.junit.{After, Test} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** * This is the main test which ensure Replication Quotas work correctly. @@ -43,7 +43,7 @@ import scala.collection.JavaConverters._ * Anything over 100MB/s tends to fail as this is the non-throttled replication rate */ class ReplicationQuotasTest extends ZooKeeperTestHarness { - def percentError(percent: Int, value: Long): Long = Math.round(value * percent / 100) + def percentError(percent: Int, value: Long): Long = Math.round(value * percent / 100.0) val msg100KB = new Array[Byte](100000) var brokers: Seq[KafkaServer] = null diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index 6cc794567ad..6941d32d9ef 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -45,7 +45,7 @@ import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, A import org.junit.Assert._ import org.junit.{After, Before, Test} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable.ListBuffer class RequestQuotaTest extends BaseRequestTest { diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala index 33e45668003..6bf49e3f548 100755 --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -40,7 +40,7 @@ import org.apache.kafka.common.utils.Time import org.junit.{Before, Test} import org.junit.Assert._ -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.reflect.ClassTag class ServerShutdownTest extends ZooKeeperTestHarness { @@ -226,7 +226,7 @@ class ServerShutdownTest extends ZooKeeperTestHarness { val brokerAndEpochs = Map((new Broker(1, "localhost", serverSocket.getLocalPort, listenerName, securityProtocol), 0L)) val controllerConfig = KafkaConfig.fromProps(TestUtils.createBrokerConfig(controllerId, zkConnect)) val controllerContext = new ControllerContext - controllerContext.setLiveBrokerAndEpochs(brokerAndEpochs) + controllerContext.setLiveBrokers(brokerAndEpochs) controllerChannelManager = new ControllerChannelManager(controllerContext, controllerConfig, Time.SYSTEM, metrics, new StateChangeLogger(controllerId, inControllerContext = true, None)) controllerChannelManager.startup() diff --git a/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala b/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala index 39795809587..2448e5058a2 100755 --- a/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala @@ -42,7 +42,7 @@ class ServerStartupTest extends ZooKeeperTestHarness { val zookeeperChroot = "/kafka-chroot-for-unittest" val props = TestUtils.createBrokerConfig(brokerId, zkConnect) val zooKeeperConnect = props.get("zookeeper.connect") - props.put("zookeeper.connect", zooKeeperConnect + zookeeperChroot) + props.put("zookeeper.connect", zooKeeperConnect.toString + zookeeperChroot) server = TestUtils.createServer(KafkaConfig.fromProps(props)) val pathExists = zkClient.pathExists(zookeeperChroot) diff --git a/core/src/test/scala/unit/kafka/server/StopReplicaRequestTest.scala b/core/src/test/scala/unit/kafka/server/StopReplicaRequestTest.scala index c7ab708f054..faf0eca5736 100644 --- a/core/src/test/scala/unit/kafka/server/StopReplicaRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/StopReplicaRequestTest.scala @@ -24,7 +24,7 @@ import org.apache.kafka.common.requests._ import org.junit.Assert._ import org.junit.Test -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class StopReplicaRequestTest extends BaseRequestTest { override val logDirCount = 2 diff --git a/core/src/test/scala/unit/kafka/server/epoch/EpochDrivenReplicationProtocolAcceptanceTest.scala b/core/src/test/scala/unit/kafka/server/epoch/EpochDrivenReplicationProtocolAcceptanceTest.scala index 8694590d8c3..3d636cda9fd 100644 --- a/core/src/test/scala/unit/kafka/server/epoch/EpochDrivenReplicationProtocolAcceptanceTest.scala +++ b/core/src/test/scala/unit/kafka/server/epoch/EpochDrivenReplicationProtocolAcceptanceTest.scala @@ -36,7 +36,7 @@ import org.apache.kafka.common.serialization.ByteArrayDeserializer import org.junit.Assert.{assertEquals, assertTrue} import org.junit.{After, Before, Test} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable.{ListBuffer => Buffer} import scala.collection.Seq diff --git a/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala b/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala index 26de772a62f..8b35149587d 100644 --- a/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala @@ -35,7 +35,7 @@ import org.junit.Assert._ import org.junit.{After, Test} import org.apache.kafka.common.requests.{EpochEndOffset, OffsetsForLeaderEpochRequest, OffsetsForLeaderEpochResponse} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.Map import scala.collection.mutable.ListBuffer @@ -274,8 +274,9 @@ class LeaderEpochIntegrationTest extends ZooKeeperTestHarness with Logging { private[epoch] class TestFetcherThread(sender: BlockingSend) extends Logging { def leaderOffsetsFor(partitions: Map[TopicPartition, Int]): Map[TopicPartition, EpochEndOffset] = { - val partitionData = partitions.mapValues( - new OffsetsForLeaderEpochRequest.PartitionData(Optional.empty(), _)).toMap + val partitionData = partitions.map { case (k, v) => + k -> new OffsetsForLeaderEpochRequest.PartitionData(Optional.empty(), v) + } val request = OffsetsForLeaderEpochRequest.Builder.forFollower( ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion, partitionData.asJava, 1) diff --git a/core/src/test/scala/unit/kafka/tools/ConsoleConsumerTest.scala b/core/src/test/scala/unit/kafka/tools/ConsoleConsumerTest.scala index a33c07680ee..ccbf0780ae9 100644 --- a/core/src/test/scala/unit/kafka/tools/ConsoleConsumerTest.scala +++ b/core/src/test/scala/unit/kafka/tools/ConsoleConsumerTest.scala @@ -33,7 +33,7 @@ import ArgumentMatchers._ import org.junit.Assert._ import org.junit.{Before, Test} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class ConsoleConsumerTest { diff --git a/core/src/test/scala/unit/kafka/tools/MirrorMakerTest.scala b/core/src/test/scala/unit/kafka/tools/MirrorMakerTest.scala index de3016bedc9..0afdf093032 100644 --- a/core/src/test/scala/unit/kafka/tools/MirrorMakerTest.scala +++ b/core/src/test/scala/unit/kafka/tools/MirrorMakerTest.scala @@ -19,7 +19,7 @@ package kafka.tools import kafka.consumer.BaseConsumerRecord import org.apache.kafka.common.record.{RecordBatch, TimestampType} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import org.junit.Assert._ import org.junit.Test diff --git a/core/src/test/scala/unit/kafka/utils/CoreUtilsTest.scala b/core/src/test/scala/unit/kafka/utils/CoreUtilsTest.scala index dace3a5bb23..c6bcd46496b 100755 --- a/core/src/test/scala/unit/kafka/utils/CoreUtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/CoreUtilsTest.scala @@ -31,7 +31,7 @@ import org.junit.Test import org.apache.kafka.common.utils.Utils import org.slf4j.event.Level -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable import scala.concurrent.duration.Duration import scala.concurrent.{Await, ExecutionContext, Future} @@ -103,7 +103,7 @@ class CoreUtilsTest extends Logging { assertEquals(2, itl.next()) assertEquals(1, itl.next()) assertEquals(2, itl.next()) - assertFalse(itl.hasDefiniteSize) + assertFalse(itl.isEmpty) val s = Set(1, 2) val its = CoreUtils.circularIterator(s) diff --git a/core/src/test/scala/unit/kafka/utils/JsonTest.scala b/core/src/test/scala/unit/kafka/utils/JsonTest.scala index 4e41bb36b05..4b7f94ae226 100644 --- a/core/src/test/scala/unit/kafka/utils/JsonTest.scala +++ b/core/src/test/scala/unit/kafka/utils/JsonTest.scala @@ -27,7 +27,7 @@ import kafka.utils.json.JsonValue import org.junit.Assert._ import org.junit.Test -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.Map object JsonTest { @@ -140,15 +140,12 @@ class JsonTest { val result = Json.parseStringAs[TestObject](s"""{"foo": "$foo", "bar": $bar}""") - assertTrue(result.isRight) - assertEquals(TestObject(foo, bar), result.right.get) + assertEquals(Right(TestObject(foo, bar)), result) } @Test def testParseToWithInvalidJson() = { val result = Json.parseStringAs[TestObject]("{invalid json}") - - assertTrue(result.isLeft) - assertEquals(classOf[JsonParseException], result.left.get.getClass) + assertEquals(Left(classOf[JsonParseException]), result.left.map(_.getClass)) } } diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 871d81c314c..edbed3140bd 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -64,7 +64,7 @@ import org.apache.zookeeper.data.ACL import org.junit.Assert._ import org.scalatest.Assertions.fail -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable.{ArrayBuffer, ListBuffer} import scala.collection.{Map, Seq, mutable} import scala.compat.java8.OptionConverters._ @@ -1120,7 +1120,6 @@ object TestUtils extends Logging { } checkpoints.forall(checkpointsPerLogDir => !checkpointsPerLogDir.contains(tp)) }), "Cleaner offset for deleted partition should have been removed") - import scala.collection.JavaConverters._ waitUntilTrue(() => servers.forall(server => server.config.logDirs.forall { logDir => topicPartitions.forall { tp => @@ -1456,7 +1455,7 @@ object TestUtils extends Logging { } def resetToCommittedPositions(consumer: KafkaConsumer[Array[Byte], Array[Byte]]): Unit = { - val committed = consumer.committed(consumer.assignment).asScala.filter(_._2 != null).mapValues(_.offset) + val committed = consumer.committed(consumer.assignment).asScala.filter(_._2 != null).map { case (k, v) => k -> v.offset } consumer.assignment.asScala.foreach { topicPartition => if (committed.contains(topicPartition)) @@ -1466,21 +1465,6 @@ object TestUtils extends Logging { } } - def alterConfigs(servers: Seq[KafkaServer], adminClient: Admin, props: Properties, - perBrokerConfig: Boolean): AlterConfigsResult = { - val configEntries = props.asScala.map { case (k, v) => new ConfigEntry(k, v) }.toList.asJava - val newConfig = new Config(configEntries) - val configs = if (perBrokerConfig) { - servers.map { server => - val resource = new ConfigResource(ConfigResource.Type.BROKER, server.config.brokerId.toString) - (resource, newConfig) - }.toMap.asJava - } else { - Map(new ConfigResource(ConfigResource.Type.BROKER, "") -> newConfig).asJava - } - adminClient.alterConfigs(configs) - } - def incrementalAlterConfigs(servers: Seq[KafkaServer], adminClient: Admin, props: Properties, perBrokerConfig: Boolean, opType: OpType = OpType.SET): AlterConfigsResult = { val configEntries = props.asScala.map { case (k, v) => new AlterConfigOp(new ConfigEntry(k, v), opType) }.toList.asJavaCollection @@ -1495,13 +1479,6 @@ object TestUtils extends Logging { adminClient.incrementalAlterConfigs(configs) } - def alterTopicConfigs(adminClient: Admin, topic: String, topicConfigs: Properties): AlterConfigsResult = { - val configEntries = topicConfigs.asScala.map { case (k, v) => new ConfigEntry(k, v) }.toList.asJava - val newConfig = new Config(configEntries) - val configs = Map(new ConfigResource(ConfigResource.Type.TOPIC, topic) -> newConfig).asJava - adminClient.alterConfigs(configs) - } - def assertLeader(client: Admin, topicPartition: TopicPartition, expectedLeader: Int): Unit = { waitForLeaderToBecome(client, topicPartition, Some(expectedLeader)) } @@ -1631,7 +1608,7 @@ object TestUtils extends Logging { def meterCount(metricName: String): Long = { KafkaYammerMetrics.defaultRegistry.allMetrics.asScala - .filterKeys(_.getMBeanName.endsWith(metricName)) + .filter { case (k, _) => k.getMBeanName.endsWith(metricName) } .values .headOption .getOrElse(fail(s"Unable to find metric $metricName")) diff --git a/core/src/test/scala/unit/kafka/zk/AdminZkClientTest.scala b/core/src/test/scala/unit/kafka/zk/AdminZkClientTest.scala index 1f46d939196..02a0403db3c 100644 --- a/core/src/test/scala/unit/kafka/zk/AdminZkClientTest.scala +++ b/core/src/test/scala/unit/kafka/zk/AdminZkClientTest.scala @@ -38,7 +38,7 @@ import org.junit.Assert._ import org.junit.{After, Test} import org.scalatest.Assertions.intercept -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.{Map, Seq, immutable} class AdminZkClientTest extends ZooKeeperTestHarness with Logging with RackAwareTest { @@ -88,7 +88,7 @@ class AdminZkClientTest extends ZooKeeperTestHarness with Logging with RackAware 1 -> List(1, 2, 3)) adminZkClient.createTopicWithAssignment("test", topicConfig, assignment) val found = zkClient.getPartitionAssignmentForTopics(Set("test")) - assertEquals(assignment.mapValues(ReplicaAssignment(_, List(), List())).toMap, found("test")) + assertEquals(assignment.map { case (k, v) => k -> ReplicaAssignment(v, List(), List()) }, found("test")) } @Test diff --git a/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala b/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala index f619a32845b..a0cc718dc9f 100644 --- a/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala +++ b/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala @@ -35,7 +35,7 @@ import org.junit.Assert._ import org.junit.{After, Before, Test} import org.scalatest.Assertions.intercept -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable.ArrayBuffer import scala.collection.{Seq, mutable} import scala.util.Random @@ -209,7 +209,7 @@ class KafkaZkClientTest extends ZooKeeperTestHarness { val updatedAssignment = assignment - new TopicPartition(topic1, 2) - zkClient.setTopicAssignment(topic1, updatedAssignment.mapValues { case v => ReplicaAssignment(v, List(), List()) }.toMap) + zkClient.setTopicAssignment(topic1, updatedAssignment.map { case (k, v) => k -> ReplicaAssignment(v, List(), List()) }) assertEquals(updatedAssignment.size, zkClient.getTopicPartitionCount(topic1).get) // add second topic @@ -864,10 +864,11 @@ class KafkaZkClientTest extends ZooKeeperTestHarness { val initialLeaderIsrAndControllerEpochs: Map[TopicPartition, LeaderIsrAndControllerEpoch] = leaderIsrAndControllerEpochs(0, 0) - val initialLeaderIsrs: Map[TopicPartition, LeaderAndIsr] = initialLeaderIsrAndControllerEpochs.mapValues(_.leaderAndIsr).toMap + val initialLeaderIsrs: Map[TopicPartition, LeaderAndIsr] = + initialLeaderIsrAndControllerEpochs.map { case (k, v) => k -> v.leaderAndIsr } private def leaderIsrs(state: Int, zkVersion: Int): Map[TopicPartition, LeaderAndIsr] = - leaderIsrAndControllerEpochs(state, zkVersion).mapValues(_.leaderAndIsr).toMap + leaderIsrAndControllerEpochs(state, zkVersion).map { case (k, v) => k -> v.leaderAndIsr } private def checkUpdateLeaderAndIsrResult( expectedSuccessfulPartitions: Map[TopicPartition, LeaderAndIsr], diff --git a/core/src/test/scala/unit/kafka/zk/ReassignPartitionsZNodeTest.scala b/core/src/test/scala/unit/kafka/zk/ReassignPartitionsZNodeTest.scala index 3ae237b2a8a..5b9e3e4d95f 100644 --- a/core/src/test/scala/unit/kafka/zk/ReassignPartitionsZNodeTest.scala +++ b/core/src/test/scala/unit/kafka/zk/ReassignPartitionsZNodeTest.scala @@ -22,6 +22,7 @@ import com.fasterxml.jackson.core.JsonProcessingException import org.apache.kafka.common.TopicPartition import org.junit.Assert._ import org.junit.Test +import org.scalatest.Assertions class ReassignPartitionsZNodeTest { @@ -42,15 +43,14 @@ class ReassignPartitionsZNodeTest { @Test def testDecodeInvalidJson(): Unit = { val result = ReassignPartitionsZNode.decode("invalid json".getBytes) - assertTrue(result.isLeft) - assertTrue(result.left.get.isInstanceOf[JsonProcessingException]) + val exception = result.left.getOrElse(Assertions.fail(s"decode should have failed, result $result")) + assertTrue(exception.isInstanceOf[JsonProcessingException]) } @Test def testDecodeValidJson(): Unit = { val result = ReassignPartitionsZNode.decode(reassignmentJson.getBytes) - assertTrue(result.isRight) - val assignmentMap = result.right.get - assertEquals(Seq(replica1, replica2), assignmentMap(new TopicPartition(topic, partition1))) + val replicas = result.map(assignmentMap => assignmentMap(new TopicPartition(topic, partition1))) + assertEquals(Right(Seq(replica1, replica2)), replicas) } } diff --git a/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala b/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala index 3dcbf147931..dfd5e002b56 100755 --- a/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala +++ b/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala @@ -27,7 +27,7 @@ import org.apache.kafka.test.IntegrationTest import org.junit.experimental.categories.Category import scala.collection.Set -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import org.apache.kafka.clients.producer.KafkaProducer import org.apache.kafka.clients.consumer.internals.AbstractCoordinator import kafka.controller.ControllerEventManager diff --git a/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala b/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala index 2f2e546c3a2..1792ac0acbf 100644 --- a/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala +++ b/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala @@ -37,7 +37,7 @@ import org.junit.Assert.{assertArrayEquals, assertEquals, assertFalse, assertTru import org.junit.{After, Before, Test} import org.scalatest.Assertions.{fail, intercept} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class ZooKeeperClientTest extends ZooKeeperTestHarness { private val mockPath = "/foo" @@ -151,7 +151,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testExistsExistingZNode(): Unit = { - import scala.collection.JavaConverters._ val createResponse = zooKeeperClient.handleRequest(CreateRequest(mockPath, Array.empty[Byte], ZooDefs.Ids.OPEN_ACL_UNSAFE.asScala, CreateMode.PERSISTENT)) assertEquals("Response code for create should be OK", Code.OK, createResponse.resultCode) @@ -167,7 +166,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testGetDataExistingZNode(): Unit = { - import scala.collection.JavaConverters._ val data = bytes val createResponse = zooKeeperClient.handleRequest(CreateRequest(mockPath, data, ZooDefs.Ids.OPEN_ACL_UNSAFE.asScala, CreateMode.PERSISTENT)) @@ -185,7 +183,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testSetDataExistingZNode(): Unit = { - import scala.collection.JavaConverters._ val data = bytes val createResponse = zooKeeperClient.handleRequest(CreateRequest(mockPath, Array.empty[Byte], ZooDefs.Ids.OPEN_ACL_UNSAFE.asScala, CreateMode.PERSISTENT)) @@ -205,7 +202,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testGetAclExistingZNode(): Unit = { - import scala.collection.JavaConverters._ val createResponse = zooKeeperClient.handleRequest(CreateRequest(mockPath, Array.empty[Byte], ZooDefs.Ids.OPEN_ACL_UNSAFE.asScala, CreateMode.PERSISTENT)) assertEquals("Response code for create should be OK", Code.OK, createResponse.resultCode) val getAclResponse = zooKeeperClient.handleRequest(GetAclRequest(mockPath)) @@ -215,7 +211,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testSetAclNonExistentZNode(): Unit = { - import scala.collection.JavaConverters._ val setAclResponse = zooKeeperClient.handleRequest(SetAclRequest(mockPath, ZooDefs.Ids.OPEN_ACL_UNSAFE.asScala, -1)) assertEquals("Response code should be NONODE", Code.NONODE, setAclResponse.resultCode) } @@ -228,7 +223,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testGetChildrenExistingZNode(): Unit = { - import scala.collection.JavaConverters._ val createResponse = zooKeeperClient.handleRequest(CreateRequest(mockPath, Array.empty[Byte], ZooDefs.Ids.OPEN_ACL_UNSAFE.asScala, CreateMode.PERSISTENT)) assertEquals("Response code for create should be OK", Code.OK, createResponse.resultCode) @@ -239,7 +233,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testGetChildrenExistingZNodeWithChildren(): Unit = { - import scala.collection.JavaConverters._ val child1 = "child1" val child2 = "child2" val child1Path = mockPath + "/" + child1 @@ -261,7 +254,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testPipelinedGetData(): Unit = { - import scala.collection.JavaConverters._ val createRequests = (1 to 3).map(x => CreateRequest("/" + x, (x * 2).toString.getBytes, ZooDefs.Ids.OPEN_ACL_UNSAFE.asScala, CreateMode.PERSISTENT)) val createResponses = createRequests.map(zooKeeperClient.handleRequest) createResponses.foreach(createResponse => assertEquals("Response code for create should be OK", Code.OK, createResponse.resultCode)) @@ -277,7 +269,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testMixedPipeline(): Unit = { - import scala.collection.JavaConverters._ val createResponse = zooKeeperClient.handleRequest(CreateRequest(mockPath, Array.empty[Byte], ZooDefs.Ids.OPEN_ACL_UNSAFE.asScala, CreateMode.PERSISTENT)) assertEquals("Response code for create should be OK", Code.OK, createResponse.resultCode) @@ -291,7 +282,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testZNodeChangeHandlerForCreation(): Unit = { - import scala.collection.JavaConverters._ val znodeChangeHandlerCountDownLatch = new CountDownLatch(1) val zNodeChangeHandler = new ZNodeChangeHandler { override def handleCreation(): Unit = { @@ -311,7 +301,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testZNodeChangeHandlerForDeletion(): Unit = { - import scala.collection.JavaConverters._ val znodeChangeHandlerCountDownLatch = new CountDownLatch(1) val zNodeChangeHandler = new ZNodeChangeHandler { override def handleDeletion(): Unit = { @@ -333,7 +322,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testZNodeChangeHandlerForDataChange(): Unit = { - import scala.collection.JavaConverters._ val znodeChangeHandlerCountDownLatch = new CountDownLatch(1) val zNodeChangeHandler = new ZNodeChangeHandler { override def handleDataChange(): Unit = { @@ -433,7 +421,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testZNodeChildChangeHandlerForChildChange(): Unit = { - import scala.collection.JavaConverters._ val zNodeChildChangeHandlerCountDownLatch = new CountDownLatch(1) val zNodeChildChangeHandler = new ZNodeChildChangeHandler { override def handleChildChange(): Unit = { @@ -459,7 +446,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testZNodeChildChangeHandlerForChildChangeNotTriggered(): Unit = { - import scala.collection.JavaConverters._ val zNodeChildChangeHandlerCountDownLatch = new CountDownLatch(1) val zNodeChildChangeHandler = new ZNodeChildChangeHandler { override def handleChildChange(): Unit = { diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index cd2f15925eb..400b8ad06a8 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -103,7 +103,7 @@ versions += [ powermock: "2.0.5", reflections: "0.9.12", rocksDB: "5.18.4", - scalaCollectionCompat: "2.1.3", + scalaCollectionCompat: "2.1.4", scalafmt: "1.5.1", scalaJava8Compat : "0.9.0", scalatest: "3.0.8", diff --git a/gradle/spotbugs-exclude.xml b/gradle/spotbugs-exclude.xml index a7174001f78..86cc464f529 100644 --- a/gradle/spotbugs-exclude.xml +++ b/gradle/spotbugs-exclude.xml @@ -175,6 +175,13 @@ For a detailed description of spotbugs bug categories, see https://spotbugs.read + + + + + + + diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java index 5e5b8cc4a17..4bf6890729e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java @@ -1303,7 +1303,7 @@ public class KStreamImplTest { final KStream stream1 = source1.filter((key, value) -> true) .filterNot((key, value) -> false); - final KStream stream2 = stream1.mapValues(Integer::new); + final KStream stream2 = stream1.mapValues(s -> Integer.valueOf(s)); final KStream stream3 = source2.flatMapValues((ValueMapper>) value -> Collections.singletonList(Integer.valueOf(value))); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java index f782d454fee..40096efdc67 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java @@ -91,7 +91,7 @@ public class KTableImplTest { final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); table1.toStream().process(supplier); - final KTable table2 = table1.mapValues(Integer::new); + final KTable table2 = table1.mapValues(s -> Integer.valueOf(s)); table2.toStream().process(supplier); final KTable table3 = table2.filter((key, value) -> (value % 2) == 0); @@ -274,7 +274,7 @@ public class KTableImplTest { builder.table(topic2, consumed); final KTableImpl table1Mapped = - (KTableImpl) table1.mapValues(Integer::new); + (KTableImpl) table1.mapValues(s -> Integer.valueOf(s)); table1Mapped.filter((key, value) -> (value % 2) == 0); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { @@ -294,7 +294,7 @@ public class KTableImplTest { (KTableImpl) builder.table(topic2, consumed); final KTableImpl table1Mapped = - (KTableImpl) table1.mapValues(Integer::new); + (KTableImpl) table1.mapValues(s -> Integer.valueOf(s)); final KTableImpl table1MappedFiltered = (KTableImpl) table1Mapped.filter((key, value) -> (value % 2) == 0); table2.join(table1MappedFiltered, (v1, v2) -> v1 + v2); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java index 8493f9976bb..28b03420df2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java @@ -180,7 +180,7 @@ public class KTableMapValuesTest { (KTableImpl) builder.table(topic1, consumed); final KTableImpl table2 = (KTableImpl) table1.mapValues( - Integer::new, + s -> Integer.valueOf(s), Materialized.>as(storeName2) .withValueSerde(Serdes.Integer())); final KTableImpl table3 = @@ -189,7 +189,7 @@ public class KTableMapValuesTest { Materialized.>as(storeName3) .withValueSerde(Serdes.Integer())); final KTableImpl table4 = - (KTableImpl) table1.mapValues(Integer::new); + (KTableImpl) table1.mapValues(s -> Integer.valueOf(s)); assertEquals(storeName2, table2.queryableStoreName()); assertEquals(storeName3, table3.queryableStoreName()); @@ -206,7 +206,7 @@ public class KTableMapValuesTest { final KTableImpl table1 = (KTableImpl) builder.table(topic1, consumed); final KTableImpl table2 = - (KTableImpl) table1.mapValues(Integer::new); + (KTableImpl) table1.mapValues(s -> Integer.valueOf(s)); final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); final Topology topology = builder.build().addProcessor("proc", supplier, table2.name); @@ -247,7 +247,7 @@ public class KTableMapValuesTest { final KTableImpl table1 = (KTableImpl) builder.table(topic1, consumed); final KTableImpl table2 = - (KTableImpl) table1.mapValues(Integer::new); + (KTableImpl) table1.mapValues(s -> Integer.valueOf(s)); table2.enableSendingOldValues(); final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapterTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapterTest.java index 31e9a95d56c..22e2210eb2b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapterTest.java @@ -175,7 +175,7 @@ public class RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapterTest { parameters[i] = new WriteBufferManager(1L, new LRUCache(1L)); break; default: - parameters[i] = parameterTypes[i].newInstance(); + parameters[i] = parameterTypes[i].getConstructor().newInstance(); } } @@ -279,7 +279,7 @@ public class RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapterTest { parameters[i] = new PlainTableConfig(); break; default: - parameters[i] = parameterTypes[i].newInstance(); + parameters[i] = parameterTypes[i].getConstructor().newInstance(); } } diff --git a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/FunctionConversions.scala b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/FunctionConversions.scala index ed8278d17b0..e6852c49a5d 100644 --- a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/FunctionConversions.scala +++ b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/FunctionConversions.scala @@ -21,7 +21,7 @@ package org.apache.kafka.streams.scala import org.apache.kafka.streams.KeyValue import org.apache.kafka.streams.kstream._ -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import java.lang.{Iterable => JIterable} @deprecated("This object is for internal use only", since = "2.1.0") diff --git a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/FunctionsCompatConversions.scala b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/FunctionsCompatConversions.scala index 2072a6eb25b..c3c6403dd39 100644 --- a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/FunctionsCompatConversions.scala +++ b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/FunctionsCompatConversions.scala @@ -18,7 +18,7 @@ package org.apache.kafka.streams.scala import org.apache.kafka.streams.KeyValue import org.apache.kafka.streams.kstream._ -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import java.lang.{Iterable => JIterable} import org.apache.kafka.streams.processor.ProcessorContext diff --git a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/ImplicitConversions.scala b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/ImplicitConversions.scala index 214a4a2a33f..f665c351127 100644 --- a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/ImplicitConversions.scala +++ b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/ImplicitConversions.scala @@ -35,8 +35,6 @@ import org.apache.kafka.streams.kstream.{ import org.apache.kafka.streams.processor.StateStore import org.apache.kafka.streams.scala.kstream._ -import scala.language.implicitConversions - /** * Implicit conversions between the Scala wrapper objects and the underlying Java * objects. diff --git a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/StreamsBuilder.scala b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/StreamsBuilder.scala index 0c05d8f277b..1d8da073e21 100644 --- a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/StreamsBuilder.scala +++ b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/StreamsBuilder.scala @@ -28,7 +28,7 @@ import org.apache.kafka.streams.state.StoreBuilder import org.apache.kafka.streams.{StreamsBuilder => StreamsBuilderJ, Topology} import org.apache.kafka.streams.scala.kstream.{Consumed, KStream, KTable, Materialized} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** * Wraps the Java class StreamsBuilder and delegates method calls to the underlying Java object. diff --git a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KStream.scala b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KStream.scala index e8e63e84331..f0d8e62192d 100644 --- a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KStream.scala +++ b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KStream.scala @@ -45,7 +45,7 @@ import org.apache.kafka.streams.scala.FunctionsCompatConversions.{ ValueTransformerSupplierWithKeyAsJava } -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** * Wraps the Java class [[org.apache.kafka.streams.kstream.KStream KStream]] and delegates method calls to the diff --git a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/TopologyTest.scala b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/TopologyTest.scala index 7bb3049a5d4..3107db6f1f2 100644 --- a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/TopologyTest.scala +++ b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/TopologyTest.scala @@ -48,7 +48,7 @@ import org.apache.kafka.streams.{KeyValue, StreamsConfig, TopologyDescription, S import org.junit.Assert._ import org.junit._ -import _root_.scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** * Test suite that verifies that the topology built by the Java and Scala APIs match. diff --git a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/WordCountTest.scala b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/WordCountTest.scala index ed623d2b016..eb1bfb89c8f 100644 --- a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/WordCountTest.scala +++ b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/WordCountTest.scala @@ -93,7 +93,7 @@ class WordCountTest extends WordCountTestData { streams.close() - import collection.JavaConverters._ + import scala.jdk.CollectionConverters._ assertEquals(actualWordCounts.asScala.take(expectedWordCounts.size).sortBy(_.key), expectedWordCounts.sortBy(_.key)) } @@ -126,7 +126,7 @@ class WordCountTest extends WordCountTestData { streams.close() - import collection.JavaConverters._ + import scala.jdk.CollectionConverters._ assertEquals(actualWordCounts.asScala.take(expectedWordCounts.size).sortBy(_.key), expectedWordCounts.sortBy(_.key)) } @@ -140,7 +140,7 @@ class WordCountTest extends WordCountTestData { KGroupedStream => KGroupedStreamJ, _ } - import collection.JavaConverters._ + import scala.jdk.CollectionConverters._ val streamsConfiguration = getStreamsConfiguration() streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String.getClass.getName) @@ -208,7 +208,7 @@ class WordCountTest extends WordCountTestData { val linesProducerConfig: Properties = getProducerConfig() - import collection.JavaConverters._ + import scala.jdk.CollectionConverters._ IntegrationTestUtils.produceValuesSynchronously(inputTopic, inputValues.asJava, linesProducerConfig, mockTime) val consumerConfig = getConsumerConfig() diff --git a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/utils/StreamToTableJoinScalaIntegrationTestBase.scala b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/utils/StreamToTableJoinScalaIntegrationTestBase.scala index c1464ad4c9c..800ad201e53 100644 --- a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/utils/StreamToTableJoinScalaIntegrationTestBase.scala +++ b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/utils/StreamToTableJoinScalaIntegrationTestBase.scala @@ -104,7 +104,7 @@ class StreamToTableJoinScalaIntegrationTestBase extends StreamToTableJoinTestDat outputTopic: String, waitTillRecordsReceived: Boolean = true): java.util.List[KeyValue[String, Long]] = { - import collection.JavaConverters._ + import _root_.scala.jdk.CollectionConverters._ // Publish user-region information. val userRegionsProducerConfig: Properties = getUserRegionsProducerConfig()