@ -19,17 +19,20 @@ package org.apache.kafka.clients.admin;
import org.apache.kafka.clients.consumer.OffsetAndMetadata ;
import org.apache.kafka.clients.consumer.OffsetAndMetadata ;
import org.apache.kafka.common.ElectionType ;
import org.apache.kafka.common.ElectionType ;
import org.apache.kafka.common.KafkaException ;
import org.apache.kafka.common.KafkaFuture ;
import org.apache.kafka.common.KafkaFuture ;
import org.apache.kafka.common.Metric ;
import org.apache.kafka.common.Metric ;
import org.apache.kafka.common.MetricName ;
import org.apache.kafka.common.MetricName ;
import org.apache.kafka.common.TopicCollection ;
import org.apache.kafka.common.TopicCollection ;
import org.apache.kafka.common.TopicPartition ;
import org.apache.kafka.common.TopicPartition ;
import org.apache.kafka.common.TopicPartitionReplica ;
import org.apache.kafka.common.TopicPartitionReplica ;
import org.apache.kafka.common.Uuid ;
import org.apache.kafka.common.acl.AclBinding ;
import org.apache.kafka.common.acl.AclBinding ;
import org.apache.kafka.common.acl.AclBindingFilter ;
import org.apache.kafka.common.acl.AclBindingFilter ;
import org.apache.kafka.common.annotation.InterfaceStability ;
import org.apache.kafka.common.annotation.InterfaceStability ;
import org.apache.kafka.common.config.ConfigResource ;
import org.apache.kafka.common.config.ConfigResource ;
import org.apache.kafka.common.errors.FeatureUpdateFailedException ;
import org.apache.kafka.common.errors.FeatureUpdateFailedException ;
import org.apache.kafka.common.errors.InterruptException ;
import org.apache.kafka.common.quota.ClientQuotaAlteration ;
import org.apache.kafka.common.quota.ClientQuotaAlteration ;
import org.apache.kafka.common.quota.ClientQuotaFilter ;
import org.apache.kafka.common.quota.ClientQuotaFilter ;
import org.apache.kafka.common.requests.LeaveGroupResponse ;
import org.apache.kafka.common.requests.LeaveGroupResponse ;
@ -1660,6 +1663,33 @@ public interface Admin extends AutoCloseable {
FenceProducersResult fenceProducers ( Collection < String > transactionalIds ,
FenceProducersResult fenceProducers ( Collection < String > transactionalIds ,
FenceProducersOptions options ) ;
FenceProducersOptions options ) ;
/ * *
* Determines the client ' s unique client instance ID used for telemetry . This ID is unique to
* this specific client instance and will not change after it is initially generated .
* The ID is useful for correlating client operations with telemetry sent to the broker and
* to its eventual monitoring destinations .
* < p >
* If telemetry is enabled , this will first require a connection to the cluster to generate
* the unique client instance ID . This method waits up to { @code timeout } for the admin
* client to complete the request .
* < p >
* Client telemetry is controlled by the { @link AdminClientConfig # ENABLE_METRICS_PUSH_CONFIG }
* configuration option .
*
* @param timeout The maximum time to wait for admin client to determine its client instance ID .
* The value must be non - negative . Specifying a timeout of zero means do not
* wait for the initial request to complete if it hasn ' t already .
* @throws InterruptException If the thread is interrupted while blocked .
* @throws KafkaException If an unexpected error occurs while trying to determine the client
* instance ID , though this error does not necessarily imply the
* admin client is otherwise unusable .
* @throws IllegalArgumentException If the { @code timeout } is negative .
* @throws IllegalStateException If telemetry is not enabled ie , config ` { @code enable . metrics . push } `
* is set to ` { @code false } ` .
* @return The client ' s assigned instance id used for metrics collection .
* /
Uuid clientInstanceId ( Duration timeout ) ;
/ * *
/ * *
* Get the metrics kept by the adminClient
* Get the metrics kept by the adminClient
* /
* /