Skip to content

Commit 58ec37d

Browse files
committed
KAFKA-14588: Move ConfigCommand to tools module
Rewrite ConfigCommand in Java and move it to the tools module
1 parent b908dd7 commit 58ec37d

File tree

12 files changed

+1137
-894
lines changed

12 files changed

+1137
-894
lines changed

bin/kafka-configs.sh

Lines changed: 1 addition & 1 deletion
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.ConfigCommand "$@"
17+
exec $(dirname $0)/kafka-run-class.sh org.apache.kafka.tools.ConfigCommand "$@"

bin/windows/kafka-configs.bat

Lines changed: 1 addition & 1 deletion
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.ConfigCommand %*
17+
"%~dp0kafka-run-class.bat" org.apache.kafka.tools.ConfigCommand %*

checkstyle/import-control-core.xml

Lines changed: 0 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -46,7 +46,6 @@
4646
<allow pkg="com.yammer.metrics"/>
4747

4848
<subpackage name="tools">
49-
<allow pkg="kafka.admin" />
5049
<allow pkg="joptsimple" />
5150
</subpackage>
5251

@@ -67,22 +66,6 @@
6766
<allow pkg="org.apache.kafka" />
6867
</subpackage>
6968

70-
<subpackage name="admin">
71-
<allow pkg="kafka.admin"/>
72-
<allow pkg="kafka.cluster"/>
73-
<allow pkg="kafka.log"/>
74-
<allow pkg="kafka.server"/>
75-
<allow pkg="org.apache.kafka.coordinator.group"/>
76-
<allow pkg="org.apache.kafka.metadata"/>
77-
<allow pkg="org.apache.kafka.security"/>
78-
<allow pkg="org.apache.kafka.storage.internals.checkpoint"/>
79-
<allow pkg="org.apache.kafka.storage.internals.log"/>
80-
<allow pkg="org.apache.kafka.test"/>
81-
<allow pkg="org.apache.logging.log4j"/>
82-
<allow pkg="org.apache.kafka.common.test"/>
83-
<allow pkg="org.apache.kafka.admin"/>
84-
</subpackage>
85-
8669
<subpackage name="security">
8770
<allow pkg="org.apache.commons" />
8871
<allow pkg="org.apache.directory" />

checkstyle/import-control.xml

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -293,7 +293,6 @@
293293
<allow pkg="org.apache.kafka.network" />
294294
<allow pkg="org.apache.kafka.raft" />
295295
<allow pkg="org.apache.kafka.server.util" />
296-
<allow pkg="kafka.admin" />
297296
<allow pkg="kafka.server" />
298297
<allow pkg="org.apache.kafka.metadata.authorizer" />
299298
<allow pkg="org.apache.kafka.security.authorizer" />
@@ -302,6 +301,7 @@
302301
<allow pkg="org.apache.kafka.server.common" />
303302
<allow pkg="org.apache.kafka.server.log.remote.metadata.storage" />
304303
<allow pkg="org.apache.kafka.server.log.remote.storage" />
304+
<allow pkg="org.apache.kafka.server.metrics" />
305305
<allow pkg="org.apache.kafka.server.quota" />
306306
<allow pkg="org.apache.kafka.streams" />
307307
<allow pkg="org.apache.kafka.clients" />

checkstyle/suppressions.xml

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -235,11 +235,11 @@
235235
<suppress checks="ClassDataAbstractionCoupling"
236236
files="(VerifiableConsumer|DumpLogSegmentsTest).java"/>
237237
<suppress checks="CyclomaticComplexity"
238-
files="(AclCommand|ConsoleConsumer|DefaultMessageFormatter|StreamsResetter|ProducerPerformance|Agent).java"/>
238+
files="(AclCommand|ConfigCommand|ConsoleConsumer|DefaultMessageFormatter|StreamsResetter|ProducerPerformance|Agent).java"/>
239239
<suppress checks="BooleanExpressionComplexity"
240240
files="(StreamsResetter|DefaultMessageFormatter).java"/>
241241
<suppress checks="NPathComplexity"
242-
files="(AclCommand|DefaultMessageFormatter|ProducerPerformance|StreamsResetter|Agent|TransactionalMessageCopier|ReplicaVerificationTool|LineMessageReader|ConsoleProducer|DumpLogSegments).java"/>
242+
files="(AclCommand|ConfigCommand|DefaultMessageFormatter|ProducerPerformance|StreamsResetter|Agent|TransactionalMessageCopier|ReplicaVerificationTool|LineMessageReader|ConsoleProducer|DumpLogSegments).java"/>
243243
<suppress checks="ParameterNumber"
244244
files="(ProduceBenchSpec|ConsumeBenchSpec|SustainedConnectionSpec).java"/>
245245
<suppress id="dontUseSystemExit"

clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/ClientTelemetryTest.java

Lines changed: 0 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -17,8 +17,6 @@
1717

1818
package org.apache.kafka.clients.admin;
1919

20-
import kafka.admin.ConfigCommand;
21-
2220
import org.apache.kafka.clients.consumer.Consumer;
2321
import org.apache.kafka.clients.consumer.ConsumerConfig;
2422
import org.apache.kafka.clients.consumer.ConsumerRecord;
@@ -31,7 +29,6 @@
3129
import org.apache.kafka.common.MetricName;
3230
import org.apache.kafka.common.TopicPartition;
3331
import org.apache.kafka.common.Uuid;
34-
import org.apache.kafka.common.errors.InvalidConfigurationException;
3532
import org.apache.kafka.common.metrics.KafkaMetric;
3633
import org.apache.kafka.common.metrics.MetricsReporter;
3734
import org.apache.kafka.common.serialization.StringDeserializer;
@@ -46,7 +43,6 @@
4643
import java.time.Duration;
4744
import java.util.ArrayList;
4845
import java.util.Arrays;
49-
import java.util.Collection;
5046
import java.util.Collections;
5147
import java.util.HashMap;
5248
import java.util.List;
@@ -56,12 +52,10 @@
5652
import java.util.concurrent.ExecutionException;
5753
import java.util.stream.Collectors;
5854

59-
import static java.util.Arrays.asList;
6055
import static org.apache.kafka.clients.admin.AdminClientConfig.METRIC_REPORTER_CLASSES_CONFIG;
6156
import static org.junit.jupiter.api.Assertions.assertEquals;
6257
import static org.junit.jupiter.api.Assertions.assertNotNull;
6358
import static org.junit.jupiter.api.Assertions.assertNull;
64-
import static org.junit.jupiter.api.Assertions.assertThrows;
6559
import static org.junit.jupiter.api.Assertions.assertTrue;
6660

6761
public class ClientTelemetryTest {
@@ -123,18 +117,6 @@ public void testClientInstanceId(ClusterInstance clusterInstance) throws Interru
123117
}
124118
}
125119

126-
@ClusterTest(types = {Type.CO_KRAFT, Type.KRAFT})
127-
public void testIntervalMsParser(ClusterInstance clusterInstance) {
128-
List<String> alterOpts = asList("--bootstrap-server", clusterInstance.bootstrapServers(),
129-
"--alter", "--entity-type", "client-metrics", "--entity-name", "test", "--add-config", "interval.ms=bbb");
130-
try (Admin client = clusterInstance.admin()) {
131-
ConfigCommand.ConfigCommandOptions addOpts = new ConfigCommand.ConfigCommandOptions(toArray(Set.of(alterOpts)));
132-
133-
Throwable e = assertThrows(ExecutionException.class, () -> ConfigCommand.alterConfig(client, addOpts));
134-
assertTrue(e.getMessage().contains(InvalidConfigurationException.class.getSimpleName()));
135-
}
136-
}
137-
138120
@ClusterTest(types = Type.KRAFT)
139121
public void testMetrics(ClusterInstance clusterInstance) {
140122
Map<String, Object> configs = new HashMap<>();
@@ -152,10 +134,6 @@ public void testMetrics(ClusterInstance clusterInstance) {
152134
}
153135
}
154136

155-
private static String[] toArray(Collection<List<String>> lists) {
156-
return lists.stream().flatMap(List::stream).toArray(String[]::new);
157-
}
158-
159137
@SuppressWarnings("unused")
160138
public static class TelemetryExporter implements ClientTelemetryExporterProvider, MetricsReporter {
161139

0 commit comments

Comments
 (0)