Skip to content

Commit 4bba2c8

Browse files
authored
KAFKA-14591: Move DeleteRecordsCommand to tools (#13278)
Reviewers: Mickael Maison <[email protected]>, Federico Valeri <[email protected]>
1 parent 2e30dd8 commit 4bba2c8

21 files changed

+408
-168
lines changed

bin/kafka-delete-records.sh

+1-1
Original file line numberDiff line numberDiff line change
@@ -14,4 +14,4 @@
1414
# See the License for the specific language governing permissions and
1515
# limitations under the License.
1616

17-
exec $(dirname $0)/kafka-run-class.sh kafka.admin.DeleteRecordsCommand "$@"
17+
exec $(dirname $0)/kafka-run-class.sh org.apache.kafka.tools.DeleteRecordsCommand "$@"

bin/windows/kafka-delete-records.bat

+1-1
Original file line numberDiff line numberDiff line change
@@ -14,4 +14,4 @@ rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
1414
rem See the License for the specific language governing permissions and
1515
rem limitations under the License.
1616

17-
"%~dp0kafka-run-class.bat" kafka.admin.DeleteRecordsCommand %*
17+
"%~dp0kafka-run-class.bat" org.apache.kafka.tools.DeleteRecordsCommand %*

core/src/main/scala/kafka/admin/AdminUtils.scala

+1-1
Original file line numberDiff line numberDiff line change
@@ -18,9 +18,9 @@
1818
package kafka.admin
1919

2020
import java.util.Random
21-
2221
import kafka.utils.Logging
2322
import org.apache.kafka.common.errors.{InvalidPartitionsException, InvalidReplicationFactorException}
23+
import org.apache.kafka.server.common.AdminOperationException
2424

2525
import collection.{Map, mutable, _}
2626

core/src/main/scala/kafka/admin/DeleteRecordsCommand.scala

-137
This file was deleted.

core/src/main/scala/kafka/admin/LeaderElectionCommand.scala

+1-1
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,6 @@ package kafka.admin
1919
import java.util.Properties
2020
import java.util.concurrent.ExecutionException
2121
import joptsimple.util.EnumConverter
22-
import kafka.common.AdminCommandFailedException
2322
import kafka.utils.CoreUtils
2423
import kafka.utils.Implicits._
2524
import kafka.utils.Json
@@ -31,6 +30,7 @@ import org.apache.kafka.common.errors.ClusterAuthorizationException
3130
import org.apache.kafka.common.errors.ElectionNotNeededException
3231
import org.apache.kafka.common.errors.TimeoutException
3332
import org.apache.kafka.common.utils.Utils
33+
import org.apache.kafka.server.common.{AdminCommandFailedException, AdminOperationException}
3434
import org.apache.kafka.server.util.{CommandDefaultOptions, CommandLineUtils}
3535

3636
import scala.jdk.CollectionConverters._

core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala

+1-1
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,6 @@ package kafka.admin
1919
import java.util
2020
import java.util.Optional
2121
import java.util.concurrent.ExecutionException
22-
import kafka.common.AdminCommandFailedException
2322
import kafka.server.DynamicConfig
2423
import kafka.utils.{CoreUtils, Exit, Json, Logging}
2524
import kafka.utils.Implicits._
@@ -30,6 +29,7 @@ import org.apache.kafka.common.config.ConfigResource
3029
import org.apache.kafka.common.errors.{ReplicaNotAvailableException, UnknownTopicOrPartitionException}
3130
import org.apache.kafka.common.utils.{Time, Utils}
3231
import org.apache.kafka.common.{KafkaException, KafkaFuture, TopicPartition, TopicPartitionReplica}
32+
import org.apache.kafka.server.common.{AdminCommandFailedException, AdminOperationException}
3333
import org.apache.kafka.server.util.{CommandDefaultOptions, CommandLineUtils}
3434
import org.apache.kafka.storage.internals.log.LogConfig
3535

core/src/main/scala/kafka/admin/TopicCommand.scala

+1-1
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,6 @@ package kafka.admin
2020
import java.util
2121
import java.util.{Collections, Optional, Properties}
2222
import joptsimple._
23-
import kafka.common.AdminCommandFailedException
2423
import kafka.utils._
2524
import org.apache.kafka.clients.CommonClientConfigs
2625
import org.apache.kafka.clients.admin.CreatePartitionsOptions
@@ -33,6 +32,7 @@ import org.apache.kafka.common.config.{ConfigResource, TopicConfig}
3332
import org.apache.kafka.common.errors.{ClusterAuthorizationException, TopicExistsException, UnsupportedVersionException}
3433
import org.apache.kafka.common.internals.Topic
3534
import org.apache.kafka.common.utils.Utils
35+
import org.apache.kafka.server.common.{AdminCommandFailedException, AdminOperationException}
3636
import org.apache.kafka.server.util.{CommandDefaultOptions, CommandLineUtils}
3737
import org.apache.kafka.storage.internals.log.LogConfig
3838
import org.apache.kafka.server.util.TopicFilter.IncludeList

core/src/main/scala/kafka/controller/KafkaController.scala

+1-2
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,6 @@ package kafka.controller
1919
import com.yammer.metrics.core.Timer
2020

2121
import java.util.concurrent.TimeUnit
22-
import kafka.admin.AdminOperationException
2322
import kafka.api._
2423
import kafka.common._
2524
import kafka.cluster.Broker
@@ -45,7 +44,7 @@ import org.apache.kafka.common.protocol.Errors
4544
import org.apache.kafka.common.requests.{AbstractControlRequest, ApiError, LeaderAndIsrResponse, UpdateFeaturesRequest, UpdateMetadataResponse}
4645
import org.apache.kafka.common.utils.{Time, Utils}
4746
import org.apache.kafka.metadata.LeaderRecoveryState
48-
import org.apache.kafka.server.common.ProducerIdsBlock
47+
import org.apache.kafka.server.common.{AdminOperationException, ProducerIdsBlock}
4948
import org.apache.kafka.server.metrics.KafkaMetricsGroup
5049
import org.apache.kafka.server.util.KafkaScheduler
5150
import org.apache.zookeeper.KeeperException

core/src/main/scala/kafka/server/ZkAdminManager.scala

+2-1
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,7 @@ package kafka.server
1818

1919
import java.util
2020
import java.util.Properties
21-
import kafka.admin.{AdminOperationException, AdminUtils}
21+
import kafka.admin.AdminUtils
2222
import kafka.common.TopicAlreadyMarkedForDeletionException
2323
import kafka.server.ConfigAdminManager.{prepareIncrementalConfigs, toLoggableProps}
2424
import kafka.server.DynamicConfig.QuotaConfigs
@@ -47,6 +47,7 @@ import org.apache.kafka.common.requests.CreateTopicsRequest._
4747
import org.apache.kafka.common.requests.{AlterConfigsRequest, ApiError}
4848
import org.apache.kafka.common.security.scram.internals.{ScramCredentialUtils, ScramFormatter}
4949
import org.apache.kafka.common.utils.Sanitizer
50+
import org.apache.kafka.server.common.AdminOperationException
5051
import org.apache.kafka.storage.internals.log.LogConfig
5152

5253
import scala.collection.{Map, mutable, _}

core/src/main/scala/kafka/zk/AdminZkClient.scala

+2-1
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@
1717
package kafka.zk
1818

1919
import java.util.Properties
20-
import kafka.admin.{AdminOperationException, AdminUtils, BrokerMetadata, RackAwareMode}
20+
import kafka.admin.{AdminUtils, BrokerMetadata, RackAwareMode}
2121
import kafka.common.TopicAlreadyMarkedForDeletionException
2222
import kafka.controller.ReplicaAssignment
2323
import kafka.server.{ConfigEntityName, ConfigType, DynamicConfig}
@@ -26,6 +26,7 @@ import kafka.utils.Implicits._
2626
import org.apache.kafka.common.{TopicPartition, Uuid}
2727
import org.apache.kafka.common.errors._
2828
import org.apache.kafka.common.internals.Topic
29+
import org.apache.kafka.server.common.AdminOperationException
2930
import org.apache.kafka.storage.internals.log.LogConfig
3031
import org.apache.zookeeper.KeeperException.NodeExistsException
3132

core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala

+1
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@ import org.apache.kafka.clients.admin.{Admin, NewPartitions, NewTopic}
2626
import org.apache.kafka.common.errors.InvalidReplicaAssignmentException
2727
import org.apache.kafka.common.requests.MetadataResponse.TopicMetadata
2828
import org.apache.kafka.common.requests.{MetadataRequest, MetadataResponse}
29+
import org.apache.kafka.server.common.AdminOperationException
2930
import org.junit.jupiter.api.Assertions._
3031
import org.junit.jupiter.api.{BeforeEach, TestInfo}
3132
import org.junit.jupiter.params.ParameterizedTest

core/src/test/scala/unit/kafka/admin/LeaderElectionCommandErrorTest.scala

+1-1
Original file line numberDiff line numberDiff line change
@@ -16,8 +16,8 @@
1616
*/
1717
package kafka.admin
1818

19-
import kafka.common.AdminCommandFailedException
2019
import org.apache.kafka.common.errors.TimeoutException
20+
import org.apache.kafka.server.common.AdminCommandFailedException
2121
import org.junit.jupiter.api.Assertions._
2222
import org.junit.jupiter.api.Test
2323

core/src/test/scala/unit/kafka/admin/LeaderElectionCommandTest.scala

+1-2
Original file line numberDiff line numberDiff line change
@@ -18,8 +18,6 @@ package kafka.admin
1818

1919
import java.nio.charset.StandardCharsets
2020
import java.nio.file.{Files, Path}
21-
22-
import kafka.common.AdminCommandFailedException
2321
import kafka.server.IntegrationTestUtils.createTopic
2422
import kafka.server.{KafkaConfig, KafkaServer}
2523
import kafka.test.annotation.{ClusterTest, ClusterTestDefaults, Type}
@@ -29,6 +27,7 @@ import kafka.utils.TestUtils
2927
import org.apache.kafka.clients.admin.AdminClientConfig
3028
import org.apache.kafka.common.TopicPartition
3129
import org.apache.kafka.common.errors.UnknownTopicOrPartitionException
30+
import org.apache.kafka.server.common.AdminCommandFailedException
3231
import org.junit.jupiter.api.Assertions._
3332
import org.junit.jupiter.api.extension.ExtendWith
3433
import org.junit.jupiter.api.{BeforeEach, Tag}

core/src/test/scala/unit/kafka/admin/ReassignPartitionsUnitTest.scala

+1-1
Original file line numberDiff line numberDiff line change
@@ -20,12 +20,12 @@ package kafka.admin
2020
import java.util.concurrent.ExecutionException
2121
import java.util.{Arrays, Collections}
2222
import kafka.admin.ReassignPartitionsCommand._
23-
import kafka.common.AdminCommandFailedException
2423
import kafka.utils.Exit
2524
import org.apache.kafka.clients.admin.{Config, MockAdminClient, PartitionReassignment}
2625
import org.apache.kafka.common.config.ConfigResource
2726
import org.apache.kafka.common.errors.{InvalidReplicationFactorException, UnknownTopicOrPartitionException}
2827
import org.apache.kafka.common.{Node, TopicPartition, TopicPartitionInfo, TopicPartitionReplica}
28+
import org.apache.kafka.server.common.{AdminCommandFailedException, AdminOperationException}
2929
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertThrows, assertTrue}
3030
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, Timeout}
3131

core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala

+1-1
Original file line numberDiff line numberDiff line change
@@ -17,13 +17,13 @@
1717
package kafka.admin
1818

1919
import kafka.admin.TopicCommand.{PartitionDescription, TopicCommandOptions, TopicService}
20-
import kafka.common.AdminCommandFailedException
2120
import kafka.utils.Exit
2221
import org.apache.kafka.clients.admin.{Admin, AdminClientTestUtils, CreatePartitionsOptions, CreateTopicsOptions, DeleteTopicsOptions, NewPartitions, NewTopic, PartitionReassignment, TopicDescription}
2322
import org.apache.kafka.common.Node
2423
import org.apache.kafka.common.TopicPartitionInfo
2524
import org.apache.kafka.common.errors.ThrottlingQuotaExceededException
2625
import org.apache.kafka.common.protocol.Errors
26+
import org.apache.kafka.server.common.{AdminCommandFailedException, AdminOperationException}
2727
import org.junit.jupiter.api.Assertions._
2828
import org.junit.jupiter.api.Test
2929
import org.mockito.ArgumentMatcher

core/src/test/scala/unit/kafka/server/DynamicConfigTest.scala

+1-1
Original file line numberDiff line numberDiff line change
@@ -16,11 +16,11 @@
1616
*/
1717
package kafka.server
1818

19-
import kafka.admin.AdminOperationException
2019
import kafka.utils.CoreUtils._
2120
import kafka.server.QuorumTestHarness
2221
import org.apache.kafka.common.config._
2322
import org.apache.kafka.common.config.internals.QuotaConfigs
23+
import org.apache.kafka.server.common.AdminOperationException
2424
import org.junit.jupiter.api.Assertions.assertThrows
2525
import org.junit.jupiter.api.Test
2626

core/src/test/scala/unit/kafka/zk/AdminZkClientTest.scala

+1
Original file line numberDiff line numberDiff line change
@@ -31,6 +31,7 @@ import org.apache.kafka.common.config.TopicConfig
3131
import org.apache.kafka.common.config.internals.QuotaConfigs
3232
import org.apache.kafka.common.errors.{InvalidReplicaAssignmentException, InvalidTopicException, TopicExistsException}
3333
import org.apache.kafka.common.metrics.Quota
34+
import org.apache.kafka.server.common.AdminOperationException
3435
import org.apache.kafka.storage.internals.log.LogConfig
3536
import org.apache.kafka.test.{TestUtils => JTestUtils}
3637
import org.junit.jupiter.api.Assertions._
Original file line numberDiff line numberDiff line change
@@ -1,10 +1,10 @@
1-
/**
1+
/*
22
* Licensed to the Apache Software Foundation (ASF) under one or more
3-
* contributor license agreements. See the NOTICE file distributed with
3+
* contributor license agreements. See the NOTICE file distributed with
44
* this work for additional information regarding copyright ownership.
55
* The ASF licenses this file to You under the Apache License, Version 2.0
66
* (the "License"); you may not use this file except in compliance with
7-
* the License. You may obtain a copy of the License at
7+
* the License. You may obtain a copy of the License at
88
*
99
* http://www.apache.org/licenses/LICENSE-2.0
1010
*
@@ -15,9 +15,14 @@
1515
* limitations under the License.
1616
*/
1717

18-
package kafka.admin
18+
package org.apache.kafka.server.common;
1919

20-
class AdminOperationException(val error: String, cause: Throwable) extends RuntimeException(error, cause) {
21-
def this(error: Throwable) = this(error.getMessage, error)
22-
def this(msg: String) = this(msg, null)
23-
}
20+
public class AdminCommandFailedException extends RuntimeException {
21+
public AdminCommandFailedException(String message) {
22+
super(message);
23+
}
24+
25+
public AdminCommandFailedException(String message, Throwable cause) {
26+
super(message, cause);
27+
}
28+
}

0 commit comments

Comments
 (0)