apoorvmittal10 commented on code in PR #14724: URL: https://github.com/apache/kafka/pull/14724#discussion_r1396737699
########## clients/src/main/java/org/apache/kafka/clients/ClientTelemetryReporter.java: ########## @@ -0,0 +1,914 @@ +/* + * 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.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.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.CopyOnWriteArrayList; +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; + public static final long MAX_TERMINAL_PUSH_WAIT_MS = 1000; + + private final List<MetricsCollector> collectors; + private final ClientTelemetryProvider telemetryProvider; + + private final ClientTelemetrySender clientTelemetrySender; + private Map<String, Object> rawOriginalConfig; + private KafkaMetricsCollector kafkaMetricsCollector; + + public ClientTelemetryReporter() { + this.collectors = new CopyOnWriteArrayList<>(); + this.telemetryProvider = new ClientTelemetryProvider(); + this.clientTelemetrySender = new DefaultClientTelemetrySender(); + } + + @SuppressWarnings("unchecked") + @Override + public synchronized void configure(Map<String, ?> configs) { + rawOriginalConfig = (Map<String, Object>) 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(this.rawOriginalConfig, "configure() was not called before contextChange()"); + if (!metricsContext.contextLabels().containsKey(MetricsContext.NAMESPACE)) { + log.warn("_namespace not found in metrics context. Metrics collection is disabled"); + return; + } + + collectors.forEach(MetricsCollector::stop); + + if (!telemetryProvider.validate(metricsContext, rawOriginalConfig)) { + log.warn("Validation failed for {} context {}, skip starting collectors", 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 ClientTelemetrySender telemetrySender() { + return clientTelemetrySender; + } + + private void initCollectors() { + kafkaMetricsCollector = new KafkaMetricsCollector( + TelemetryMetricNamingConvention.getClientTelemetryMetricNamingStrategy( + telemetryProvider.domain())); + collectors.add(kafkaMetricsCollector); + } + + 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 + List<MetricsCollector> collectors() { + if (kafkaMetricsCollector == null) { + return Collections.emptyList(); + } + + return Collections.singletonList(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() { + this.enabled = true; + } + + @Override + public long timeToNextUpdate(long requestTimeoutMs) { + final long now = System.currentTimeMillis(); + final ClientTelemetryState localState; + final long localLastRequestMs; + int localIntervalMs; + + try { + lock.readLock().lock(); Review Comment: Thanks for pointing out, done. -- 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