Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KAFKA-15681: Add support of client-metrics in kafka-configs.sh (KIP-714) #14632

Merged
merged 21 commits into from
Nov 28, 2023

Conversation

apoorvmittal10
Copy link
Collaborator

The PR adds support of alter/describe configs for client-metrics as defined in KIP-714

Below are the results of the commands:

Help section adds details for client-metrics:

./bin/kafka-configs.sh --help

This tool helps to manipulate and describe entity config for a topic, client, user, broker, ip or client-metrics
.
.
.
--add-config <String>                  Key Value pairs of configs to add.
                                         Square brackets can be used to group
                                         values which contain commas: 'k1=v1,
                                         k2=[v1,v2,v2],k3=v3'. The following
                                         is a list of valid configurations:
                                         For entity-type 'topics':
                                       	.
                                        .
                                        .
                                        controller_mutation_rate
                                       	producer_byte_rate
                                       	request_percentage
                                       For entity-type 'clients':
                                       	consumer_byte_rate
                                       	controller_mutation_rate
                                       	producer_byte_rate
                                       	request_percentage
                                       For entity-type 'ips':
                                       	connection_creation_rate
                                       For entity-type 'client-metrics':
                                       	interval.ms
                                       	match
                                       	metrics
                                       Entity types 'users' and 'clients' may
                                         be specified together to update
                                         config for clients of a specific
                                         user.

--entity-type <String>                 Type of entity
                                         (topics/clients/users/brokers/broker-
                                         loggers/ips/client-metrics)


--entity-name <String>                 Name of entity (topic name/client
                                         id/user principal name/broker
                                         id/ip/client metrics subscription
                                         name)

Incorrect entity type:

./bin/kafka-configs.sh --bootstrap-server localhost:9092 --entity-type client-metrics1 --describe --entity-name METRICSUB

Invalid entity type client-metrics1, the entity type must be one of topics, clients, users, brokers, ips, client-metrics, broker-loggers with a --bootstrap-server or --bootstrap-controller argument

Describe wihout entity name:

./bin/kafka-configs.sh --bootstrap-server localhost:9092 --entity-type client-metrics --describe

an entity name must be specified with --describe of client-metrics

Describe with blank entity name:

./bin/kafka-configs.sh --bootstrap-server localhost:9092 --entity-type client-metrics --describe --entity-name ""

an entity name must be specified with --describe of client-metrics

Invalid entity name. Omitted to throw exception as the describe response is further needed in alter to construct if the new subscription to be added or altered.

./bin/kafka-configs.sh --bootstrap-server localhost:9092 --entity-type client-metrics --describe --entity-name "random"

Dynamic configs for client-metric random are:

Successful alter of client-metrics:

./bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter --entity-type client-metrics --entity-name METRICSUB --add-config "metrics=org.apache.kafka.consumer.,interval.ms=60000,match=[client_software_name=kafka.python,client_software_version=1\.2\..*]"

Completed updating config for client-metric METRICSUB.

Successful describe of client-metrics:

./bin/kafka-configs.sh --bootstrap-server localhost:9092 --entity-type client-metrics --describe --entity-name METRICSUB

Dynamic configs for client-metric METRICSUB are:
  interval.ms=60000 sensitive=false synonyms={}
  match=client_software_name=kafka.python,client_software_version=1\.2\..* sensitive=false synonyms={}
  metrics=org.apache.kafka.consumer. sensitive=false synonyms={}

With Zookeper:

Without making change in ZKConfigRepository, throws UnknownServerException:

./bin/kafka-configs.sh --bootstrap-server localhost:9092 --entity-type client-metrics --describe --entity-name "random"


Dynamic configs for client-metric random are:

Error while executing config command with args '--bootstrap-server localhost:9092 --entity-type client-metrics --describe --entity-name random'
java.util.concurrent.ExecutionException: org.apache.kafka.common.errors.UnknownServerException: The server experienced an unexpected error when processing the request.
	at java.base/java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:396)
	at java.base/java.util.concurrent.CompletableFuture.get(CompletableFuture.java:2096)
	at org.apache.kafka.common.internals.KafkaFutureImpl.get(KafkaFutureImpl.java:180)
	at kafka.admin.ConfigCommand$.getResourceConfig(ConfigCommand.scala:614)
	at kafka.admin.ConfigCommand$.$anonfun$describeResourceConfig$5(ConfigCommand.scala:572)
	at kafka.admin.ConfigCommand$.$anonfun$describeResourceConfig$5$adapted(ConfigCommand.scala:564)
	at scala.collection.immutable.List.foreach(List.scala:333)
	at kafka.admin.ConfigCommand$.describeResourceConfig(ConfigCommand.scala:564)
	at kafka.admin.ConfigCommand$.describeConfig(ConfigCommand.scala:542)
	at kafka.admin.ConfigCommand$.processCommand(ConfigCommand.scala:343)
	at kafka.admin.ConfigCommand$.main(ConfigCommand.scala:97)
	at kafka.admin.ConfigCommand.main(ConfigCommand.scala)
Caused by: org.apache.kafka.common.errors.UnknownServerException: The server experienced an unexpected error when processing the request.

After the change in ZKConfigRepository:

Describe:

./bin/kafka-configs.sh --bootstrap-server localhost:9092 --entity-type client-metrics --describe --entity-name "random"


Dynamic configs for client-metric random are:

Error while executing config command with args '--bootstrap-server localhost:9092 --entity-type client-metrics --describe --entity-name random'
java.util.concurrent.ExecutionException: org.apache.kafka.common.errors.InvalidRequestException: Config type client-metrics is only supported on KRaft clusters
	at java.base/java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:396)
	at java.base/java.util.concurrent.CompletableFuture.get(CompletableFuture.java:2096)
	at org.apache.kafka.common.internals.KafkaFutureImpl.get(KafkaFutureImpl.java:180)
	at kafka.admin.ConfigCommand$.getResourceConfig(ConfigCommand.scala:614)
	at kafka.admin.ConfigCommand$.$anonfun$describeResourceConfig$5(ConfigCommand.scala:572)
	at kafka.admin.ConfigCommand$.$anonfun$describeResourceConfig$5$adapted(ConfigCommand.scala:564)
	at scala.collection.immutable.List.foreach(List.scala:333)
	at kafka.admin.ConfigCommand$.describeResourceConfig(ConfigCommand.scala:564)
	at kafka.admin.ConfigCommand$.describeConfig(ConfigCommand.scala:542)
	at kafka.admin.ConfigCommand$.processCommand(ConfigCommand.scala:343)
	at kafka.admin.ConfigCommand$.main(ConfigCommand.scala:97)
	at kafka.admin.ConfigCommand.main(ConfigCommand.scala)
Caused by: org.apache.kafka.common.errors.InvalidRequestException: Config type client-metrics is only supported on KRaft clusters

Alter:

./bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter --entity-type client-metrics --entity-name METRICSUB2 --add-config "interval.ms=1000"

Error while executing config command with args '--bootstrap-server localhost:9092 --alter --entity-type client-metrics --entity-name METRICSUB2 --add-config interval.ms=1000'
java.util.concurrent.ExecutionException: org.apache.kafka.common.errors.InvalidRequestException: Config type client-metrics is only supported on KRaft clusters
	at java.base/java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:396)
	at java.base/java.util.concurrent.CompletableFuture.get(CompletableFuture.java:2096)
	at org.apache.kafka.common.internals.KafkaFutureImpl.get(KafkaFutureImpl.java:180)
	at kafka.admin.ConfigCommand$.getResourceConfig(ConfigCommand.scala:610)
	at kafka.admin.ConfigCommand$.alterConfig(ConfigCommand.scala:450)
	at kafka.admin.ConfigCommand$.processCommand(ConfigCommand.scala:341)
	at kafka.admin.ConfigCommand$.main(ConfigCommand.scala:97)
	at kafka.admin.ConfigCommand.main(ConfigCommand.scala)
Caused by: org.apache.kafka.common.errors.InvalidRequestException: Config type client-metrics is only supported on KRaft clusters

With zookeeper as arg:

./bin/kafka-configs.sh -zookeeper localhost:9071 --describe --entity-type client-metrics --entity-name METRICSUB2

Invalid entity type client-metrics, the entity type must be one of users, brokers with a --zookeeper argument

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

@apoorvmittal10
Copy link
Collaborator Author

Build depends on PR - #14621

@mjsax mjsax added tools kip Requires or implements a KIP labels Oct 27, 2023
@apoorvmittal10
Copy link
Collaborator Author

@junrao @hachikuji @AndrewJSchofield @mjsax Please if I can get the feedback on the PR.

Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

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

@apoorvmittal10 : Thanks for the PR. Left a few comments.

@@ -536,6 +552,8 @@ object ConfigCommand extends Logging {
adminClient.listTopics(new ListTopicsOptions().listInternal(true)).names().get().asScala.toSeq
case ConfigType.Broker | BrokerLoggerConfigType =>
adminClient.describeCluster(new DescribeClusterOptions()).nodes().get().asScala.map(_.idString).toSeq :+ BrokerDefaultEntityName
case ConfigType.ClientMetrics =>
throw new InvalidRequestException("Client metrics entity-name is required")
Copy link
Contributor

Choose a reason for hiding this comment

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

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

Yes, it will be improved in KIP-1000 where empty entity-name can be used to describe all.

Copy link
Collaborator

Choose a reason for hiding this comment

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

Precisely.

Copy link
Collaborator

Choose a reason for hiding this comment

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

I've written kafka-client-metrics.sh also which will be in a new PR once this one is merged.

core/src/main/scala/kafka/server/DynamicConfig.scala Outdated Show resolved Hide resolved
}
}
ConfigCommand.alterConfig(mockAdminClient, alterOpts)
verify(describeResult).all()
Copy link
Contributor

Choose a reason for hiding this comment

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

Do we need to verify alterResult too? Also, why do we need to call all()?

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

I have added alterResult as well, thanks. all() is need as it ensures the verification of returned future is executed.

@apoorvmittal10
Copy link
Collaborator Author

@apoorvmittal10 : Thanks for the PR. Left a few comments.

Thanks for the feedback @junrao. I have addressed the comments.

Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

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

@apoorvmittal10 : Thanks for the updated PR. LGTM

JDK 17 and Scala 2.13 didn't finish. Could you trigger another test run? This can typically be done by closing the PR, waiting for 20 secs and reopening it.

@apoorvmittal10
Copy link
Collaborator Author

@apoorvmittal10 : Thanks for the updated PR. LGTM

JDK 17 and Scala 2.13 didn't finish. Could you trigger another test run? This can typically be done by closing the PR, waiting for 20 secs and reopening it.

Thanks @junrao, I have merged upstream/trunk branch to trigger the build for now.

@junrao
Copy link
Contributor

junrao commented Nov 11, 2023

@apoorvmittal10 : Thanks for rerunning the tests. Are the 34 test failures related to this PR?

@apoorvmittal10
Copy link
Collaborator Author

@apoorvmittal10 : Thanks for rerunning the tests. Are the 34 test failures related to this PR?

@junrao I looked at the failing tests and none of them seems related to the PR changes. I saw the comment and mail by @dajac and completely agree with the pain of getting the build without flaky tests for a while.

Copy link
Collaborator

@AndrewJSchofield AndrewJSchofield left a comment

Choose a reason for hiding this comment

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

Looking good. A few minor comments only.

@@ -536,6 +552,8 @@ object ConfigCommand extends Logging {
adminClient.listTopics(new ListTopicsOptions().listInternal(true)).names().get().asScala.toSeq
case ConfigType.Broker | BrokerLoggerConfigType =>
adminClient.describeCluster(new DescribeClusterOptions()).nodes().get().asScala.map(_.idString).toSeq :+ BrokerDefaultEntityName
case ConfigType.ClientMetrics =>
throw new InvalidRequestException("Client metrics entity-name is required")
Copy link
Collaborator

Choose a reason for hiding this comment

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

Precisely.

core/src/main/scala/kafka/admin/ConfigCommand.scala Outdated Show resolved Hide resolved
@@ -536,6 +552,8 @@ object ConfigCommand extends Logging {
adminClient.listTopics(new ListTopicsOptions().listInternal(true)).names().get().asScala.toSeq
case ConfigType.Broker | BrokerLoggerConfigType =>
adminClient.describeCluster(new DescribeClusterOptions()).nodes().get().asScala.map(_.idString).toSeq :+ BrokerDefaultEntityName
case ConfigType.ClientMetrics =>
throw new InvalidRequestException("Client metrics entity-name is required")
Copy link
Collaborator

Choose a reason for hiding this comment

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

I've written kafka-client-metrics.sh also which will be in a new PR once this one is merged.

@apoorvmittal10
Copy link
Collaborator Author

There are different tests failing from previous runs and none related to the PR.

@junrao
Copy link
Contributor

junrao commented Nov 14, 2023

@apoorvmittal10 : Thanks for looking into the test failures. There is an ongoing discussion on requiring a green build before merging the PR. I will need to wait for the result of that discussion before merging the PR.

@apoorvmittal10
Copy link
Collaborator Author

@apoorvmittal10 : Thanks for looking into the test failures. There is an ongoing discussion on requiring a green build before merging the PR. I will need to wait for the result of that discussion before merging the PR.

Thanks @junrao I followed the thread where discussion is going on for green builds, started by @dajac, though I am not so old with AK process but if we are targeting no test failures in PRs for Kafka then shouldn't we be aggressive in fixing those else it will delay all deliverables for 3.7. I ll wait for the decision or what's the way forward. Definitely flaky tests is a big problem with AK right now.

@apoorvmittal10
Copy link
Collaborator Author

10 Failing flaky tests in current run, some have existing jira (mostly open):

Build / JDK 11 and Scala 2.13 / testRackAwareRangeAssignor(String).quorum=kraft – integration.kafka.server.FetchFromFollowerIntegrationTest
5s - https://issues.apache.org/jira/browse/KAFKA-15020

Build / JDK 11 and Scala 2.13 / testFailureToFenceEpoch(String).quorum=kraft – org.apache.kafka.tiered.storage.integration.TransactionsWithTieredStoreTest
38s - https://issues.apache.org/jira/browse/KAFKA-14989

Build / JDK 8 and Scala 2.12 / testDynamicProducerIdExpirationMs(String).quorum=kraft – kafka.api.ProducerIdExpirationTest
32s
Build / JDK 8 and Scala 2.12 / testThrottledProducerConsumer(String).quorum=zk – kafka.api.UserClientIdQuotaTest
41s
Build / JDK 8 and Scala 2.12 / testThrottledProducerConsumer(String).quorum=kraft – kafka.api.UserClientIdQuotaTest
43s
Build / JDK 8 and Scala 2.12 / testQuotaOverrideDelete(String).quorum=zk – kafka.api.UserClientIdQuotaTest
1m 6s
Build / JDK 8 and Scala 2.12 / testAssignmentAggregation() – kafka.server.AssignmentsManagerTest
<1s
Build / JDK 8 and Scala 2.12 / testAssignmentAggregation() – kafka.server.AssignmentsManagerTest
2s
Build / JDK 8 and Scala 2.12 / shouldMigratePersistentKeyValueStoreToTimestampedKeyValueStoreUsingPapi – org.apache.kafka.streams.integration.StoreUpgradeIntegrationTest
1m 7s - https://issues.apache.org/jira/browse/KAFKA-10151
Build / JDK 8 and Scala 2.12 / [6] Type=Raft-Isolated, Name=testDescribeQuorumReplicationSuccessful, MetadataVersion=3.7-IV1, Security=PLAINTEXT – org.apache.kafka.tools.MetadataQuorumCommandTest
1m 7s - https://issues.apache.org/jira/browse/KAFKA-15104
Build / JDK 17 and Scala 2.13 / testTaskRequestWithOldStartMsGetsUpdated() – org.apache.kafka.trogdor.coordinator.CoordinatorTest
2m 0s - https://issues.apache.org/jira/browse/KAFKA-15760
Build / JDK 21 and Scala 2.13 / shouldThrowIllegalArgumentExceptionWhenCustomPartitionerReturnsMultiplePartitions() – org.apache.kafka.streams.integration.KTableKTableForeignKeyInnerJoinCustomPartitionerIntegrationTest
1m 5s - https://issues.apache.org/jira/browse/KAFKA-14454
Build / JDK 21 and Scala 2.13 / [1] Type=Raft-Combined, Name=testDescribeQuorumReplicationSuccessful, MetadataVersion=3.7-IV1, Security=PLAINTEXT – org.apache.kafka.tools.MetadataQuorumCommandTest
1m 7s - https://issues.apache.org/jira/browse/KAFKA-15104
Build / JDK 21 and Scala 2.13 / [5] Type=Raft-Combined, Name=testDescribeQuorumReplicationSuccessful, MetadataVersion=3.7-IV1, Security=PLAINTEXT – org.apache.kafka.tools.MetadataQuorumCommandTest
https://issues.apache.org/jira/browse/KAFKA-15104

@junrao
Copy link
Contributor

junrao commented Nov 20, 2023

@apoorvmittal10 : Thanks for triaging the failed tests. There is still no green build though.

@mjsax
Copy link
Member

mjsax commented Nov 21, 2023

Seems there is a compilation error on the last run:


[Error] /home/jenkins/jenkins-agent/workspace/Kafka_kafka-pr_PR-14632/core/src/main/scala/kafka/server/DynamicConfig.scala:116:47: object ClientMetricsConfigs is not a member of package kafka.metrics

[Error] /home/jenkins/jenkins-agent/workspace/Kafka_kafka-pr_PR-14632/core/src/main/scala/kafka/server/DynamicConfig.scala:116:17: private val clientConfigs in object ClientMetrics is never used

@apoorvmittal10
Copy link
Collaborator Author

Seems there is a compilation error on the last run:


[Error] /home/jenkins/jenkins-agent/workspace/Kafka_kafka-pr_PR-14632/core/src/main/scala/kafka/server/DynamicConfig.scala:116:47: object ClientMetricsConfigs is not a member of package kafka.metrics

[Error] /home/jenkins/jenkins-agent/workspace/Kafka_kafka-pr_PR-14632/core/src/main/scala/kafka/server/DynamicConfig.scala:116:17: private val clientConfigs in object ClientMetrics is never used

The change in dependent classes broke the PR build. I have resolved merge issue and triggered build.

@apoorvmittal10
Copy link
Collaborator Author

@junrao @mjsax There are still some flaky tests which fails, Can I do anything to get the build green?

@junrao
Copy link
Contributor

junrao commented Nov 21, 2023

@apoorvmittal10 :

  1. Do you know why JDK 11 and Scala 2.13 didn't build?
  2. For getting green build, it would be useful to help triage the new test failures. If we could identify the PR that introduced the failure, we could ping the author for a fix.

@apoorvmittal10
Copy link
Collaborator Author

@apoorvmittal10 :

  1. Do you know why JDK 11 and Scala 2.13 didn't build?
  2. For getting green build, it would be useful to help triage the new test failures. If we could identify the PR that introduced the failure, we could ping the author for a fix.

I ll try to find out some details regarding failing tests. Build for JDK11 sems stuck in publishing test result where the build went for more than 7 hours.

@apoorvmittal10
Copy link
Collaborator Author

apoorvmittal10 commented Nov 24, 2023

@junrao @mjsax I have investigated the test failure in the build which are reported as New Failing. Among 12 tests, 8 tests have already been reported as flaky from other developers (jira already exists), successfully reproduced the flakiness behaviour for 2 tests and reported on the recent PRs from developer (details below), remaining 2 - I cannot reproduce flakiness locally but have seen couple of already existing Jiras for same test class file hence I have created jiras for them.

New failing - 12

Created Jira: Build / JDK 11 and Scala 2.13 / testAlterSinkConnectorOffsets – org.apache.kafka.connect.integration.OffsetsApiIntegrationTest
12s: https://issues.apache.org/jira/browse/KAFKA-15892

Created Jira: Build / JDK 11 and Scala 2.13 / testResetSinkConnectorOffsetsOverriddenConsumerGroupId – org.apache.kafka.connect.integration.OffsetsApiIntegrationTest
51s: https://issues.apache.org/jira/browse/KAFKA-15891

Already existing jira: Build / JDK 11 and Scala 2.13 / testResetSinkConnectorOffsetsZombieSinkTasks – org.apache.kafka.connect.integration.OffsetsApiIntegrationTest
58s: https://issues.apache.org/jira/browse/KAFKA-15524

Already existing jira: Build / JDK 21 and Scala 2.13 / testReplicateSourceDefault() – org.apache.kafka.connect.mirror.integration.IdentityReplicationIntegrationTest
1m 50s: https://issues.apache.org/jira/browse/KAFKA-15292

Already exists multiple jiras which reports the failure from the MirrorConnectorsIntegrationBaseTest (multiple MirrorConnector...Test file extends base class), hence didn't create another jira though current MirrorConnectorsIntegrationTransactionsTest file differs from already reported: Build / JDK 21 and Scala 2.13 / testSyncTopicConfigs() – org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationTransactionsTest
1m 42s: https://issues.apache.org/jira/browse/KAFKA-14971, https://issues.apache.org/jira/browse/KAFKA-15523

Already existing jira: Build / JDK 21 and Scala 2.13 / shouldAddAndRemoveNamedTopologiesBeforeStartingAndRouteQueriesToCorrectTopology() – org.apache.kafka.streams.integration.NamedTopologyIntegrationTest
1m 9s: https://issues.apache.org/jira/browse/KAFKA-15798

Already existing jira: Build / JDK 21 and Scala 2.13 / testTaskRequestWithOldStartMsGetsUpdated() – org.apache.kafka.trogdor.coordinator.CoordinatorTest
2m 0s: https://issues.apache.org/jira/browse/KAFKA-15760

Already existing jira: Build / JDK 17 and Scala 2.13 / testTaskRequestWithOldStartMsGetsUpdated() – org.apache.kafka.trogdor.coordinator.CoordinatorTest
2m 0s: https://issues.apache.org/jira/browse/KAFKA-15760

Already existing jira: Build / JDK 8 and Scala 2.12 / testConsumptionWithBrokerFailures() – kafka.api.ConsumerBounceTest
54s: https://issues.apache.org/jira/browse/KAFKA-15146

Reported the issue on recent PR: Build / JDK 8 and Scala 2.12 / testAlwaysSendsAccumulatedOfflineDirs() – kafka.server.BrokerLifecycleManagerTest
<1s: Reported to the committer, fix has been raised for flakiness by author: #14770 (comment)

Reported the issue on recent PR: Build / JDK 8 and Scala 2.12 / testInternalTopicExists() – org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerTest
12s: Reported to the committer https://github.com/apache/kafka/pull/14755/files#r1404552551

Already existing jira, reports flakiness in the Test Class itself (not specific tests): Build / JDK 8 and Scala 2.12 / testListTopicsWithExcludeInternal(String).quorum=zk – org.apache.kafka.tools.TopicCommandIntegrationTest: https://issues.apache.org/jira/browse/KAFKA-15140

@junrao
Copy link
Contributor

junrao commented Nov 28, 2023

@apoorvmittal10 : Thanks for triaging the failed tests. Since they are unrelated to this PR, will merge it.

@junrao junrao merged commit 38f2faf into apache:trunk Nov 28, 2023
1 check failed
ex172000 pushed a commit to ex172000/kafka that referenced this pull request Dec 15, 2023
…14) (apache#14632)

The PR adds support of alter/describe configs for client-metrics as defined in KIP-714

Reviewers: Andrew Schofield <[email protected]>, Jun Rao <[email protected]>
@apoorvmittal10 apoorvmittal10 deleted the kip-714-ak-utility branch January 5, 2024 12:47
yyu1993 pushed a commit to yyu1993/kafka that referenced this pull request Feb 15, 2024
…14) (apache#14632)

The PR adds support of alter/describe configs for client-metrics as defined in KIP-714

Reviewers: Andrew Schofield <[email protected]>, Jun Rao <[email protected]>
AnatolyPopov pushed a commit to aiven/kafka that referenced this pull request Feb 16, 2024
…14) (apache#14632)

The PR adds support of alter/describe configs for client-metrics as defined in KIP-714

Reviewers: Andrew Schofield <[email protected]>, Jun Rao <[email protected]>
clolov pushed a commit to clolov/kafka that referenced this pull request Apr 5, 2024
…14) (apache#14632)

The PR adds support of alter/describe configs for client-metrics as defined in KIP-714

Reviewers: Andrew Schofield <[email protected]>, Jun Rao <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
kip Requires or implements a KIP tools
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants