philipnee commented on code in PR #14724: URL: https://github.com/apache/kafka/pull/14724#discussion_r1411338139
########## clients/src/main/java/org/apache/kafka/clients/ClientTelemetryReporter.java: ########## @@ -0,0 +1,952 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients; + +import io.opentelemetry.proto.metrics.v1.Metric; +import io.opentelemetry.proto.metrics.v1.MetricsData; +import io.opentelemetry.proto.metrics.v1.ResourceMetrics; +import io.opentelemetry.proto.metrics.v1.ScopeMetrics; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.errors.InterruptException; +import org.apache.kafka.common.message.GetTelemetrySubscriptionsRequestData; +import org.apache.kafka.common.message.GetTelemetrySubscriptionsResponseData; +import org.apache.kafka.common.message.PushTelemetryRequestData; +import org.apache.kafka.common.message.PushTelemetryResponseData; +import org.apache.kafka.common.metrics.KafkaMetric; +import org.apache.kafka.common.metrics.MetricsContext; +import org.apache.kafka.common.metrics.MetricsReporter; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.record.CompressionType; +import org.apache.kafka.common.requests.AbstractRequest; +import org.apache.kafka.common.requests.AbstractRequest.Builder; +import org.apache.kafka.common.requests.GetTelemetrySubscriptionsRequest; +import org.apache.kafka.common.requests.GetTelemetrySubscriptionsResponse; +import org.apache.kafka.common.requests.PushTelemetryRequest; +import org.apache.kafka.common.requests.PushTelemetryResponse; +import org.apache.kafka.common.telemetry.ClientTelemetryState; +import org.apache.kafka.common.telemetry.internals.ClientTelemetrySender; +import org.apache.kafka.common.telemetry.internals.ClientTelemetryUtils; +import org.apache.kafka.common.telemetry.internals.KafkaMetricsCollector; +import org.apache.kafka.common.telemetry.internals.MetricKeyable; +import org.apache.kafka.common.telemetry.internals.MetricsCollector; +import org.apache.kafka.common.telemetry.internals.MetricsEmitter; +import org.apache.kafka.common.telemetry.internals.SinglePointMetric; +import org.apache.kafka.common.telemetry.internals.TelemetryMetricNamingConvention; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.ByteBuffer; +import java.time.Duration; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.StringJoiner; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.Predicate; + +/** + * The implementation of the {@link MetricsReporter} for client telemetry which manages the life-cycle + * of the client telemetry collection process. The client telemetry reporter is responsible for + * collecting the client telemetry data and sending it to the broker. + * <p> + * + * The client telemetry reporter is configured with a {@link ClientTelemetrySender} which is + * responsible for sending the client telemetry data to the broker. The client telemetry reporter + * will attempt to fetch the telemetry subscription information from the broker and send the + * telemetry data to the broker based on the subscription information. + * <p> + * + * The full life-cycle of the metric collection process is defined by a state machine in + * {@link ClientTelemetryState}. Each state is associated with a different set of operations. + * For example, the client telemetry reporter will attempt to fetch the telemetry subscription + * from the broker when in the {@link ClientTelemetryState#SUBSCRIPTION_NEEDED} state. + * If the push operation fails, the client telemetry reporter will attempt to re-fetch the + * subscription information by setting the state back to {@link ClientTelemetryState#SUBSCRIPTION_NEEDED}. + * <p> + * + * In an unlikely scenario, if a bad state transition is detected, an + * {@link IllegalStateException} will be thrown. + * <p> + * + * The state transition follows the following steps in order: + * <ol> + * <li>{@link ClientTelemetryState#SUBSCRIPTION_NEEDED}</li> + * <li>{@link ClientTelemetryState#SUBSCRIPTION_IN_PROGRESS}</li> + * <li>{@link ClientTelemetryState#PUSH_NEEDED}</li> + * <li>{@link ClientTelemetryState#PUSH_IN_PROGRESS}</li> + * <li>{@link ClientTelemetryState#TERMINATING_PUSH_NEEDED}</li> + * <li>{@link ClientTelemetryState#TERMINATING_PUSH_IN_PROGRESS}</li> + * <li>{@link ClientTelemetryState#TERMINATED}</li> + * </ol> + * <p> + * + * For more detail in state transition, see {@link ClientTelemetryState#validateTransition}. + */ +public class ClientTelemetryReporter implements MetricsReporter { + + private static final Logger log = LoggerFactory.getLogger(ClientTelemetryReporter.class); + public static final int DEFAULT_PUSH_INTERVAL_MS = 5 * 60 * 1000; + + private final ClientTelemetryProvider telemetryProvider; + private final ClientTelemetrySender clientTelemetrySender; + private final Time time; + + private Map<String, Object> rawOriginalConfig; + private KafkaMetricsCollector kafkaMetricsCollector; + + public ClientTelemetryReporter(Time time) { + this.time = time; + telemetryProvider = new ClientTelemetryProvider(); + clientTelemetrySender = new DefaultClientTelemetrySender(); + } + + @SuppressWarnings("unchecked") + @Override + public synchronized void configure(Map<String, ?> configs) { + rawOriginalConfig = (Map<String, Object>) Objects.requireNonNull(configs); + } + + @Override + public synchronized void contextChange(MetricsContext metricsContext) { + /* + If validation succeeds: initialize the provider, start the metric collection task, + set metrics labels for services/libraries that expose metrics. + */ + Objects.requireNonNull(rawOriginalConfig, "configure() was not called before contextChange()"); + if (kafkaMetricsCollector != null) { + kafkaMetricsCollector.stop(); + } + + if (!telemetryProvider.validate(metricsContext)) { + log.warn("Validation failed for {} context {}, skip starting collectors. Metrics collection is disabled", + telemetryProvider.getClass(), metricsContext.contextLabels()); + return; + } + + if (kafkaMetricsCollector == null) { + // Initialize the provider only once. contextChange(..) can be called more than once, + // but once it's been initialized and all necessary labels are present then we don't + // re-initialize. + telemetryProvider.configure(rawOriginalConfig); + } + + telemetryProvider.contextChange(metricsContext); + + if (kafkaMetricsCollector == null) { + initCollectors(); + } + } + + @Override + public void init(List<KafkaMetric> metrics) { + // metrics collector may not have been initialized (e.g. invalid context labels) + // in which case metrics collection is disabled + if (kafkaMetricsCollector != null) { + kafkaMetricsCollector.init(metrics); + } + } + + /** + * Method is invoked whenever a metric is added/registered + */ + @Override + public void metricChange(KafkaMetric metric) { + // metrics collector may not have been initialized (e.g. invalid context labels) + // in which case metrics collection is disabled + if (kafkaMetricsCollector != null) { + kafkaMetricsCollector.metricChange(metric); + } + } + + /** + * Method is invoked whenever a metric is removed + */ + @Override + public void metricRemoval(KafkaMetric metric) { + // metrics collector may not have been initialized (e.g. invalid context labels) + // in which case metrics collection is disabled + if (kafkaMetricsCollector != null) { + kafkaMetricsCollector.metricRemoval(metric); + } + } + + @Override + public void close() { + log.debug("Stopping ClientTelemetryReporter"); + try { + clientTelemetrySender.close(); + } catch (Exception exception) { + log.error("Failed to close client telemetry reporter", exception); + } + } + + public synchronized void updateMetricsLabels(Map<String, String> labels) { + telemetryProvider.updateLabels(labels); + } + + public void initiateClose(long timeoutMs) { + log.debug("Initiate close of ClientTelemetryReporter"); + try { + clientTelemetrySender.initiateClose(timeoutMs); + } catch (Exception exception) { + log.error("Failed to initiate close of client telemetry reporter", exception); + } + } + + public ClientTelemetrySender telemetrySender() { + return clientTelemetrySender; + } + + private void initCollectors() { + kafkaMetricsCollector = new KafkaMetricsCollector( + TelemetryMetricNamingConvention.getClientTelemetryMetricNamingStrategy( + telemetryProvider.domain())); + } + + private ResourceMetrics buildMetric(Metric metric) { + return ResourceMetrics.newBuilder() + .setResource(telemetryProvider.resource()) + .addScopeMetrics(ScopeMetrics.newBuilder() + .addMetrics(metric) + .build()).build(); + } + + // Visible for testing, only for unit tests + MetricsCollector metricsCollector() { + return kafkaMetricsCollector; + } + + // Visible for testing, only for unit tests + ClientTelemetryProvider telemetryProvider() { + return telemetryProvider; + } + + class DefaultClientTelemetrySender implements ClientTelemetrySender { + + /* + These are the lower and upper bounds of the jitter that we apply to the initial push + telemetry API call. This helps to avoid a flood of requests all coming at the same time. + */ + private final static double INITIAL_PUSH_JITTER_LOWER = 0.5; + private final static double INITIAL_PUSH_JITTER_UPPER = 1.5; + + private final ReadWriteLock lock = new ReentrantReadWriteLock(); + private final Condition subscriptionLoaded = lock.writeLock().newCondition(); + private final Condition terminalPushInProgress = lock.writeLock().newCondition(); + /* + Initial state should be subscription needed which should allow issuing first telemetry + request of get telemetry subscription. + */ + private ClientTelemetryState state = ClientTelemetryState.SUBSCRIPTION_NEEDED; + + private ClientTelemetrySubscription subscription; + + /* + Last time a telemetry request was made. Initialized to 0 to indicate that no request has + been made yet. Telemetry requests, get or post, should always be made after the push interval + time has elapsed. + */ + private long lastRequestMs; + /* + Interval between telemetry requests in milliseconds. Initialized to 0 to indicate that the + interval has not yet been computed. The first get request will be immediately triggered as + soon as the client is ready. + */ + private int intervalMs; + /* + Whether the client telemetry sender is enabled or not. Initialized to true to indicate that + the client telemetry sender is enabled. This is used to disable the client telemetry sender + when the client receives unrecoverable error from broker. + */ + private boolean enabled; + + private DefaultClientTelemetrySender() { + enabled = true; + } + + @Override + public long timeToNextUpdate(long requestTimeoutMs) { + final long now = time.milliseconds(); + final ClientTelemetryState localState; + final long localLastRequestMs; + final int localIntervalMs; + + lock.readLock().lock(); + try { + if (!enabled) { + return Integer.MAX_VALUE; + } + + localState = state; + localLastRequestMs = lastRequestMs; + localIntervalMs = intervalMs; + } finally { + lock.readLock().unlock(); + } + + final long timeMs; + final String apiName; + final String msg; + + switch (localState) { + case SUBSCRIPTION_IN_PROGRESS: + case PUSH_IN_PROGRESS: + /* + We have a network request in progress. We record the time of the request + submission, so wait that amount of the time PLUS the requestTimeout that + is provided. + */ + apiName = (localState == ClientTelemetryState.SUBSCRIPTION_IN_PROGRESS) ? ApiKeys.GET_TELEMETRY_SUBSCRIPTIONS.name : ApiKeys.PUSH_TELEMETRY.name; + timeMs = requestTimeoutMs; + msg = String.format("the remaining wait time for the %s network API request, as specified by %s", apiName, CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG); + break; + case TERMINATING_PUSH_IN_PROGRESS: + timeMs = Long.MAX_VALUE; + msg = String.format("the terminating push is in progress, disabling telemetry for further requests"); + break; + case TERMINATING_PUSH_NEEDED: + timeMs = 0; + msg = String.format("the client should try to submit the final %s network API request ASAP before closing", ApiKeys.PUSH_TELEMETRY.name); + break; + case SUBSCRIPTION_NEEDED: + case PUSH_NEEDED: + apiName = (localState == ClientTelemetryState.SUBSCRIPTION_NEEDED) ? ApiKeys.GET_TELEMETRY_SUBSCRIPTIONS.name : ApiKeys.PUSH_TELEMETRY.name; + long timeRemainingBeforeRequest = localLastRequestMs + localIntervalMs - now; + if (timeRemainingBeforeRequest <= 0) { + timeMs = 0; + msg = String.format("the wait time before submitting the next %s network API request has elapsed", apiName); + } else { + timeMs = timeRemainingBeforeRequest; + msg = String.format("the client will wait before submitting the next %s network API request", apiName); + } + break; + default: + throw new IllegalStateException("Unknown telemetry state: " + localState); + } + + log.debug("For telemetry state {}, returning the value {} ms; {}", localState, timeMs, msg); + return timeMs; + } + + @Override + public Optional<AbstractRequest.Builder<?>> createRequest() { + final ClientTelemetryState localState; + final ClientTelemetrySubscription localSubscription; + + lock.readLock().lock(); + try { + localState = state; + localSubscription = subscription; + } finally { + lock.readLock().unlock(); + } + + if (localState == ClientTelemetryState.SUBSCRIPTION_NEEDED) { + return createSubscriptionRequest(localSubscription); + } else if (localState == ClientTelemetryState.PUSH_NEEDED || localState == ClientTelemetryState.TERMINATING_PUSH_NEEDED) { + return createPushRequest(localSubscription); + } + + log.warn("Cannot make telemetry request as telemetry is in state: {}", localState); + return Optional.empty(); + } + + @Override + public void handleResponse(GetTelemetrySubscriptionsResponse response) { + final long now = time.milliseconds(); + final GetTelemetrySubscriptionsResponseData data = response.data(); + + final ClientTelemetryState oldState; + final ClientTelemetrySubscription oldSubscription; + lock.readLock().lock(); + try { + oldState = state; + oldSubscription = subscription; + } finally { + lock.readLock().unlock(); + } + + Optional<Integer> errorIntervalMsOpt = ClientTelemetryUtils.maybeFetchErrorIntervalMs(data.errorCode(), + oldSubscription != null ? oldSubscription.pushIntervalMs() : -1); + // If the error code indicates that the interval ms needs to be updated as per the error + // code then update the interval ms and state so that the subscription can be retried. + if (errorIntervalMsOpt.isPresent()) { + // Update the state from SUBSCRIPTION_INR_PROGRESS to SUBSCRIPTION_NEEDED as the error + // response indicates that the subscription is not valid. + if (!maybeSetState(ClientTelemetryState.SUBSCRIPTION_NEEDED)) { + log.error("Unable to transition state after failed get telemetry subscriptions from state {}", oldState); + } + updateErrorResult(errorIntervalMsOpt.get(), now); + return; + } + + Uuid clientInstanceId = ClientTelemetryUtils.validateClientInstanceId(data.clientInstanceId()); + int intervalMs = ClientTelemetryUtils.validateIntervalMs(data.pushIntervalMs()); + Predicate<? super MetricKeyable> selector = ClientTelemetryUtils.getSelectorFromRequestedMetrics( + data.requestedMetrics()); + List<CompressionType> acceptedCompressionTypes = ClientTelemetryUtils.getCompressionTypesFromAcceptedList( + data.acceptedCompressionTypes()); + + // Check if the delta temporality has changed, if so, we need to reset the ledger tracking + // the last value sent for each metric. + if (oldSubscription != null && oldSubscription.deltaTemporality() != data.deltaTemporality()) { + log.info("Delta temporality has changed from {} to {}, resetting metric values", + oldSubscription.deltaTemporality(), data.deltaTemporality()); + if (kafkaMetricsCollector != null) { + kafkaMetricsCollector.metricsReset(); + } + } + + ClientTelemetrySubscription clientTelemetrySubscription = new ClientTelemetrySubscription( + clientInstanceId, + data.subscriptionId(), + intervalMs, + acceptedCompressionTypes, + data.deltaTemporality(), + selector); + + lock.writeLock().lock(); + try { + // This is the case if we began termination sometime after the subscription request + // was issued. We're just now getting our callback, but we need to ignore it. + if (isTerminatingState()) { + return; + } + + ClientTelemetryState newState; + if (selector == ClientTelemetryUtils.SELECTOR_NO_METRICS) { + // This is the case where no metrics are requested and/or match the filters. We need + // to wait intervalMs then retry. + newState = ClientTelemetryState.SUBSCRIPTION_NEEDED; + } else { + newState = ClientTelemetryState.PUSH_NEEDED; + } + + // If we're terminating, don't update state or set the subscription. + if (!maybeSetState(newState)) { + return; + } + + updateSubscriptionResult(clientTelemetrySubscription, now); + log.info("Client telemetry registered with client instance id: {}", subscription.clientInstanceId()); + } finally { + lock.writeLock().unlock(); + } + } + + @Override + public void handleResponse(PushTelemetryResponse response) { + final long now = time.milliseconds(); + final PushTelemetryResponseData data = response.data(); + + lock.writeLock().lock(); + try { + Optional<Integer> errorIntervalMsOpt = ClientTelemetryUtils.maybeFetchErrorIntervalMs(data.errorCode(), + subscription.pushIntervalMs()); + // If the error code indicates that the interval ms needs to be updated as per the error + // code then update the interval ms and state so that the subscription can be re-fetched, + // and the push retried. + if (errorIntervalMsOpt.isPresent()) { + // This is the case when client began termination sometime after the last push request + // was issued. Just getting the callback, hence need to ignore it. + if (isTerminatingState()) { + return; + } + + if (!maybeSetState(ClientTelemetryState.SUBSCRIPTION_NEEDED)) { + log.error("Unable to transition state after failed push telemetry from state {}", state); + } + updateErrorResult(errorIntervalMsOpt.get(), now); + return; + } + + lastRequestMs = now; + intervalMs = subscription.pushIntervalMs(); + if (!maybeSetState(ClientTelemetryState.PUSH_NEEDED)) { + log.error("Unable to transition state after successful push telemetry from state {}", state); + } + } finally { + lock.writeLock().unlock(); + } + } + + @Override + public void handleFailedGetTelemetrySubscriptionsRequest(KafkaException maybeFatalException) { + log.warn("The broker generated an error for the get telemetry network API request", maybeFatalException); + handleFailedRequest(maybeFatalException != null); + } + + @Override + public void handleFailedPushTelemetryRequest(KafkaException maybeFatalException) { + log.warn("The broker generated an error for the push telemetry network API request", maybeFatalException); + handleFailedRequest(maybeFatalException != null); + } + + @Override + public Optional<Uuid> clientInstanceId(Duration timeout) { + final long timeoutMs = timeout.toMillis(); + if (timeoutMs < 0) { + throw new IllegalArgumentException("The timeout cannot be negative for fetching client instance id."); + } + + lock.writeLock().lock(); + try { + if (subscription == null) { + // If we have a non-negative timeout and no-subscription, let's wait for one to be retrieved. + log.debug("Waiting for telemetry subscription containing the client instance ID with timeoutMillis = {} ms.", timeoutMs); + try { + if (!subscriptionLoaded.await(timeoutMs, TimeUnit.MILLISECONDS)) { + log.debug("Wait for telemetry subscription elapsed; may not have actually loaded it"); + } + } catch (InterruptedException e) { + throw new InterruptException(e); + } + } + + if (subscription == null) { + log.debug("Client instance ID could not be retrieved with timeout {}", timeout); + return Optional.empty(); + } + + Uuid clientInstanceId = subscription.clientInstanceId(); + if (clientInstanceId == null) { + log.info("Client instance ID was null in telemetry subscription while in state {}", state); + return Optional.empty(); + } + + return Optional.of(clientInstanceId); + } finally { + lock.writeLock().unlock(); + } + } + + @Override + public void close() { + log.debug("close telemetry sender for client telemetry reporter instance"); + + boolean shouldClose = false; + lock.writeLock().lock(); + try { + if (state != ClientTelemetryState.TERMINATED) { + if (maybeSetState(ClientTelemetryState.TERMINATED)) { + shouldClose = true; + } + } else { + log.debug("Ignoring subsequent close"); + } + } finally { + lock.writeLock().unlock(); + } + + if (shouldClose) { + if (kafkaMetricsCollector != null) { + kafkaMetricsCollector.stop(); + } + } + } + + @Override + public void initiateClose(long timeoutMs) { + log.debug("initiate close for client telemetry, check if terminal push required. Timeout {} ms.", timeoutMs); + + lock.writeLock().lock(); + try { + // If we never fetched a subscription, we can't really push anything. + if (lastRequestMs == 0) { + log.debug("Telemetry subscription not loaded, not attempting terminating push"); + return; + } + + if (state == ClientTelemetryState.SUBSCRIPTION_NEEDED) { + log.debug("Subscription not yet loaded, ignoring terminal push"); + return; + } + + if (isTerminatingState() || !maybeSetState(ClientTelemetryState.TERMINATING_PUSH_NEEDED)) { + log.debug("Ignoring subsequent initiateClose"); + return; + } + + try { + log.info("About to wait {} ms. for terminal telemetry push to be submitted", timeoutMs); + if (!terminalPushInProgress.await(timeoutMs, TimeUnit.MILLISECONDS)) { + log.info("Wait for terminal telemetry push to be submitted has elapsed; may not have actually sent request"); + } + } catch (InterruptedException e) { + log.warn("Error during client telemetry close", e); + } + } finally { + lock.writeLock().unlock(); + } + } + + private Optional<Builder<?>> createSubscriptionRequest(ClientTelemetrySubscription localSubscription) { + // If we've previously retrieved a subscription, it will contain the client instance ID + // that the broker assigned. Otherwise, per KIP-714, we send a special "null" UUID to + // signal to the broker that we need to have a client instance ID assigned. + Uuid clientInstanceId = (localSubscription != null) ? localSubscription.clientInstanceId() : Uuid.ZERO_UUID; + + lock.writeLock().lock(); + try { + if (isTerminatingState()) { + return Optional.empty(); + } + + if (!maybeSetState(ClientTelemetryState.SUBSCRIPTION_IN_PROGRESS)) { + return Optional.empty(); + } + } finally { + lock.writeLock().unlock(); + } + + AbstractRequest.Builder<?> requestBuilder = new GetTelemetrySubscriptionsRequest.Builder( + new GetTelemetrySubscriptionsRequestData().setClientInstanceId(clientInstanceId), true); + return Optional.of(requestBuilder); + } + + private Optional<Builder<?>> createPushRequest(ClientTelemetrySubscription localSubscription) { + if (localSubscription == null) { + log.warn("Telemetry state is {} but subscription is null; not sending telemetry", state); + if (!maybeSetState(ClientTelemetryState.SUBSCRIPTION_NEEDED)) { + log.error("Unable to transition state after failed create push telemetry from state {}", state); + } + return Optional.empty(); + } + + // Don't send a push request if we don't have the collector initialized. Re-attempt + // the push on the next interval. + if (kafkaMetricsCollector == null) { + log.warn("Cannot make telemetry request as collector is not initialized"); + // Update last accessed time for push request to be retried on next interval. + updateErrorResult(localSubscription.pushIntervalMs, time.milliseconds()); + return Optional.empty(); + } + + boolean terminating; + lock.writeLock().lock(); + try { + // We've already been terminated, or we've already issued our last push, so we + // should just exit now. + if (state == ClientTelemetryState.TERMINATED || state == ClientTelemetryState.TERMINATING_PUSH_IN_PROGRESS) { + return Optional.empty(); + } + + // Check the *actual* state (while locked) to make sure we're still in the state + // we expect to be in. + terminating = state == ClientTelemetryState.TERMINATING_PUSH_NEEDED; + if (!maybeSetState(terminating ? ClientTelemetryState.TERMINATING_PUSH_IN_PROGRESS : ClientTelemetryState.PUSH_IN_PROGRESS)) { + return Optional.empty(); + } + } finally { + lock.writeLock().unlock(); + } + + byte[] payload; + try (MetricsEmitter emitter = new ClientTelemetryEmitter(localSubscription.selector(), localSubscription.deltaTemporality())) { + emitter.init(); + kafkaMetricsCollector.collect(emitter); + payload = createPayload(emitter.emittedMetrics()); Review Comment: If you can't create a payload due to some exception, do you return an empty Optional? If so can we move the catch{ } in createPayload here? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org