Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion bin/kafka-configs.sh
Original file line number Diff line number Diff line change
Expand Up @@ -14,4 +14,4 @@
# See the License for the specific language governing permissions and
# limitations under the License.

exec $(dirname $0)/kafka-run-class.sh kafka.admin.ConfigCommand "$@"
exec $(dirname $0)/kafka-run-class.sh org.apache.kafka.tools.ConfigCommand "$@"
2 changes: 1 addition & 1 deletion bin/windows/kafka-configs.bat
Original file line number Diff line number Diff line change
Expand Up @@ -14,4 +14,4 @@ rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
rem See the License for the specific language governing permissions and
rem limitations under the License.

"%~dp0kafka-run-class.bat" kafka.admin.ConfigCommand %*
"%~dp0kafka-run-class.bat" org.apache.kafka.tools.ConfigCommand %*
17 changes: 0 additions & 17 deletions checkstyle/import-control-core.xml
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,6 @@
<allow pkg="com.yammer.metrics"/>

<subpackage name="tools">
<allow pkg="kafka.admin" />
<allow pkg="joptsimple" />
</subpackage>

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

<subpackage name="admin">
<allow pkg="kafka.admin"/>
<allow pkg="kafka.cluster"/>
<allow pkg="kafka.log"/>
<allow pkg="kafka.server"/>
<allow pkg="org.apache.kafka.coordinator.group"/>
<allow pkg="org.apache.kafka.metadata"/>
<allow pkg="org.apache.kafka.security"/>
<allow pkg="org.apache.kafka.storage.internals.checkpoint"/>
<allow pkg="org.apache.kafka.storage.internals.log"/>
<allow pkg="org.apache.kafka.test"/>
<allow pkg="org.apache.logging.log4j"/>
<allow pkg="org.apache.kafka.common.test"/>
<allow pkg="org.apache.kafka.admin"/>
</subpackage>

<subpackage name="security">
<allow pkg="org.apache.commons" />
<allow pkg="org.apache.directory" />
Expand Down
2 changes: 1 addition & 1 deletion checkstyle/import-control.xml
Original file line number Diff line number Diff line change
Expand Up @@ -293,7 +293,6 @@
<allow pkg="org.apache.kafka.network" />
<allow pkg="org.apache.kafka.raft" />
<allow pkg="org.apache.kafka.server.util" />
<allow pkg="kafka.admin" />
<allow pkg="kafka.server" />
<allow pkg="org.apache.kafka.metadata.authorizer" />
<allow pkg="org.apache.kafka.security.authorizer" />
Expand All @@ -302,6 +301,7 @@
<allow pkg="org.apache.kafka.server.common" />
<allow pkg="org.apache.kafka.server.log.remote.metadata.storage" />
<allow pkg="org.apache.kafka.server.log.remote.storage" />
<allow pkg="org.apache.kafka.server.metrics" />
<allow pkg="org.apache.kafka.server.quota" />
<allow pkg="org.apache.kafka.streams" />
<allow pkg="org.apache.kafka.clients" />
Expand Down
4 changes: 2 additions & 2 deletions checkstyle/suppressions.xml
Original file line number Diff line number Diff line change
Expand Up @@ -235,11 +235,11 @@
<suppress checks="ClassDataAbstractionCoupling"
files="(VerifiableConsumer|DumpLogSegmentsTest).java"/>
<suppress checks="CyclomaticComplexity"
files="(AclCommand|ConsoleConsumer|DefaultMessageFormatter|StreamsResetter|ProducerPerformance|Agent).java"/>
files="(AclCommand|ConfigCommand|ConsoleConsumer|DefaultMessageFormatter|StreamsResetter|ProducerPerformance|Agent).java"/>
<suppress checks="BooleanExpressionComplexity"
files="(StreamsResetter|DefaultMessageFormatter).java"/>
<suppress checks="NPathComplexity"
files="(AclCommand|DefaultMessageFormatter|ProducerPerformance|StreamsResetter|Agent|TransactionalMessageCopier|ReplicaVerificationTool|LineMessageReader|ConsoleProducer|DumpLogSegments).java"/>
files="(AclCommand|ConfigCommand|DefaultMessageFormatter|ProducerPerformance|StreamsResetter|Agent|TransactionalMessageCopier|ReplicaVerificationTool|LineMessageReader|ConsoleProducer|DumpLogSegments).java"/>
<suppress checks="ParameterNumber"
files="(ProduceBenchSpec|ConsumeBenchSpec|SustainedConnectionSpec).java"/>
<suppress id="dontUseSystemExit"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,6 @@

package org.apache.kafka.clients.admin;

import kafka.admin.ConfigCommand;

import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerRecord;
Expand All @@ -31,7 +29,6 @@
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.errors.InvalidConfigurationException;
import org.apache.kafka.common.metrics.KafkaMetric;
import org.apache.kafka.common.metrics.MetricsReporter;
import org.apache.kafka.common.serialization.StringDeserializer;
Expand All @@ -46,7 +43,6 @@
import java.time.Duration;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
Expand All @@ -56,12 +52,10 @@
import java.util.concurrent.ExecutionException;
import java.util.stream.Collectors;

import static java.util.Arrays.asList;
import static org.apache.kafka.clients.admin.AdminClientConfig.METRIC_REPORTER_CLASSES_CONFIG;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;

public class ClientTelemetryTest {
Expand Down Expand Up @@ -123,18 +117,6 @@ public void testClientInstanceId(ClusterInstance clusterInstance) throws Interru
}
}

@ClusterTest(types = {Type.CO_KRAFT, Type.KRAFT})
public void testIntervalMsParser(ClusterInstance clusterInstance) {
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This was moved to ConfigCommandIntegrationTest

List<String> alterOpts = asList("--bootstrap-server", clusterInstance.bootstrapServers(),
"--alter", "--entity-type", "client-metrics", "--entity-name", "test", "--add-config", "interval.ms=bbb");
try (Admin client = clusterInstance.admin()) {
ConfigCommand.ConfigCommandOptions addOpts = new ConfigCommand.ConfigCommandOptions(toArray(Set.of(alterOpts)));

Throwable e = assertThrows(ExecutionException.class, () -> ConfigCommand.alterConfig(client, addOpts));
assertTrue(e.getMessage().contains(InvalidConfigurationException.class.getSimpleName()));
}
}

@ClusterTest(types = Type.KRAFT)
public void testMetrics(ClusterInstance clusterInstance) {
Map<String, Object> configs = new HashMap<>();
Expand All @@ -152,10 +134,6 @@ public void testMetrics(ClusterInstance clusterInstance) {
}
}

private static String[] toArray(Collection<List<String>> lists) {
return lists.stream().flatMap(List::stream).toArray(String[]::new);
}

@SuppressWarnings("unused")
public static class TelemetryExporter implements ClientTelemetryExporterProvider, MetricsReporter {

Expand Down
Loading
Loading