From 58e5fb88e67f66f4299076eb9b1736631f09d3de Mon Sep 17 00:00:00 2001 From: dengziming Date: Thu, 17 Mar 2022 20:31:41 +0800 Subject: [PATCH] MINOR: Remove scala KafkaException --- .../scala/kafka/common/KafkaException.scala | 27 ------------------- .../scala/kafka/log/LogCleanerManager.scala | 6 ++--- .../server/BrokerMetadataCheckpoint.scala | 3 ++- .../scala/kafka/server/KafkaRaftServer.scala | 6 ++--- .../kafka/server/KafkaRaftServerTest.scala | 5 ++-- 5 files changed, 11 insertions(+), 36 deletions(-) delete mode 100644 core/src/main/scala/kafka/common/KafkaException.scala diff --git a/core/src/main/scala/kafka/common/KafkaException.scala b/core/src/main/scala/kafka/common/KafkaException.scala deleted file mode 100644 index 9c34dd9bd78b4..0000000000000 --- a/core/src/main/scala/kafka/common/KafkaException.scala +++ /dev/null @@ -1,27 +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.common - -/** - * Usage of this class is discouraged. Use org.apache.kafka.common.KafkaException instead. - * - * This class will be removed once kafka.security.auth classes are removed. - */ -class KafkaException(message: String, t: Throwable) extends RuntimeException(message, t) { - def this(message: String) = this(message, null) - def this(t: Throwable) = this("", t) -} diff --git a/core/src/main/scala/kafka/log/LogCleanerManager.scala b/core/src/main/scala/kafka/log/LogCleanerManager.scala index 8b6926b0d46d7..89d4686e2852e 100755 --- a/core/src/main/scala/kafka/log/LogCleanerManager.scala +++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala @@ -21,15 +21,15 @@ import java.io.File import java.util.concurrent.TimeUnit import java.util.concurrent.locks.ReentrantLock -import kafka.common.{KafkaException, LogCleaningAbortedException} +import kafka.common.LogCleaningAbortedException import kafka.metrics.KafkaMetricsGroup import kafka.server.LogDirFailureChannel import kafka.server.checkpoints.OffsetCheckpointFile import kafka.utils.CoreUtils._ import kafka.utils.{Logging, Pool} -import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.utils.Time +import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.errors.KafkaStorageException +import org.apache.kafka.common.utils.Time import scala.collection.{Iterable, Seq, mutable} diff --git a/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala b/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala index 0a9bfbda535f3..67d27cb052931 100755 --- a/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala +++ b/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala @@ -21,9 +21,10 @@ import java.io._ import java.nio.file.{Files, NoSuchFileException} import java.util.Properties -import kafka.common.{InconsistentBrokerMetadataException, KafkaException} +import kafka.common.InconsistentBrokerMetadataException import kafka.server.RawMetaProperties._ import kafka.utils._ +import org.apache.kafka.common.KafkaException import org.apache.kafka.common.utils.Utils import scala.collection.mutable diff --git a/core/src/main/scala/kafka/server/KafkaRaftServer.scala b/core/src/main/scala/kafka/server/KafkaRaftServer.scala index 876925c9797f3..20c87ed9e51a0 100644 --- a/core/src/main/scala/kafka/server/KafkaRaftServer.scala +++ b/core/src/main/scala/kafka/server/KafkaRaftServer.scala @@ -19,15 +19,15 @@ package kafka.server import java.io.File import java.util.concurrent.CompletableFuture -import kafka.common.{InconsistentNodeIdException, KafkaException} +import kafka.common.InconsistentNodeIdException import kafka.log.{LogConfig, UnifiedLog} import kafka.metrics.{KafkaMetricsReporter, KafkaYammerMetrics} import kafka.raft.KafkaRaftManager import kafka.server.KafkaRaftServer.{BrokerRole, ControllerRole} import kafka.utils.{CoreUtils, Logging, Mx4jLoader, VerifiableProperties} -import org.apache.kafka.common.utils.{AppInfoParser, Time} -import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.common.config.{ConfigDef, ConfigResource} +import org.apache.kafka.common.utils.{AppInfoParser, Time} +import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid} import org.apache.kafka.metadata.{KafkaConfigSchema, MetadataRecordSerde} import org.apache.kafka.raft.RaftConfig import org.apache.kafka.server.common.ApiMessageAndVersion diff --git a/core/src/test/scala/unit/kafka/server/KafkaRaftServerTest.scala b/core/src/test/scala/unit/kafka/server/KafkaRaftServerTest.scala index 82ad5427a6a4e..b4dac22c6c581 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaRaftServerTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaRaftServerTest.scala @@ -19,9 +19,10 @@ package kafka.server import java.io.File import java.nio.file.Files import java.util.Properties -import kafka.common.{InconsistentBrokerMetadataException, InconsistentNodeIdException, KafkaException} + +import kafka.common.{InconsistentBrokerMetadataException, InconsistentNodeIdException} import kafka.log.UnifiedLog -import org.apache.kafka.common.Uuid +import org.apache.kafka.common.{KafkaException, Uuid} import org.apache.kafka.common.utils.Utils import org.apache.kafka.test.TestUtils import org.junit.jupiter.api.Assertions._