-
Notifications
You must be signed in to change notification settings - Fork 4.3k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
* Add kafka poll latency metrics * Address Sam's comments [Dataflow Streaming] Use isolated windmill streams based on job settings (#32503) * Add kafka poll latency metrics * address comments * Ensure this is disabled for now until flag to enable it is explicitly passed --------- Co-authored-by: Naireen <[email protected]>
- Loading branch information
Showing
10 changed files
with
476 additions
and
7 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
131 changes: 131 additions & 0 deletions
131
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaMetrics.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,131 @@ | ||
/* | ||
* 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.beam.sdk.io.kafka; | ||
|
||
import com.google.auto.value.AutoValue; | ||
import java.time.Duration; | ||
import java.util.HashMap; | ||
import java.util.Map; | ||
import java.util.concurrent.ConcurrentLinkedQueue; | ||
import java.util.concurrent.atomic.AtomicBoolean; | ||
import org.apache.beam.sdk.metrics.Histogram; | ||
import org.apache.beam.sdk.util.Preconditions; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
/** Stores and exports metrics for a batch of Kafka Client RPCs. */ | ||
public interface KafkaMetrics { | ||
|
||
void updateSuccessfulRpcMetrics(String topic, Duration elapsedTime); | ||
|
||
void updateKafkaMetrics(); | ||
|
||
/** No-op implementation of {@code KafkaResults}. */ | ||
class NoOpKafkaMetrics implements KafkaMetrics { | ||
private NoOpKafkaMetrics() {} | ||
|
||
@Override | ||
public void updateSuccessfulRpcMetrics(String topic, Duration elapsedTime) {} | ||
|
||
@Override | ||
public void updateKafkaMetrics() {} | ||
|
||
private static NoOpKafkaMetrics singleton = new NoOpKafkaMetrics(); | ||
|
||
static NoOpKafkaMetrics getInstance() { | ||
return singleton; | ||
} | ||
} | ||
|
||
/** | ||
* Metrics of a batch of RPCs. Member variables are thread safe; however, this class does not have | ||
* atomicity across member variables. | ||
* | ||
* <p>Expected usage: A number of threads record metrics in an instance of this class with the | ||
* member methods. Afterwards, a single thread should call {@code updateStreamingInsertsMetrics} | ||
* which will export all counters metrics and RPC latency distribution metrics to the underlying | ||
* {@code perWorkerMetrics} container. Afterwards, metrics should not be written/read from this | ||
* object. | ||
*/ | ||
@AutoValue | ||
abstract class KafkaMetricsImpl implements KafkaMetrics { | ||
|
||
private static final Logger LOG = LoggerFactory.getLogger(KafkaMetricsImpl.class); | ||
|
||
static HashMap<String, Histogram> latencyHistograms = new HashMap<String, Histogram>(); | ||
|
||
abstract HashMap<String, ConcurrentLinkedQueue<Duration>> perTopicRpcLatencies(); | ||
|
||
abstract AtomicBoolean isWritable(); | ||
|
||
public static KafkaMetricsImpl create() { | ||
return new AutoValue_KafkaMetrics_KafkaMetricsImpl( | ||
new HashMap<String, ConcurrentLinkedQueue<Duration>>(), new AtomicBoolean(true)); | ||
} | ||
|
||
/** Record the rpc status and latency of a successful Kafka poll RPC call. */ | ||
@Override | ||
public void updateSuccessfulRpcMetrics(String topic, Duration elapsedTime) { | ||
if (isWritable().get()) { | ||
ConcurrentLinkedQueue<Duration> latencies = perTopicRpcLatencies().get(topic); | ||
if (latencies == null) { | ||
latencies = new ConcurrentLinkedQueue<Duration>(); | ||
latencies.add(elapsedTime); | ||
perTopicRpcLatencies().put(topic, latencies); | ||
} else { | ||
latencies.add(elapsedTime); | ||
} | ||
} | ||
} | ||
|
||
/** Record rpc latency histogram metrics for all recorded topics. */ | ||
private void recordRpcLatencyMetrics() { | ||
for (Map.Entry<String, ConcurrentLinkedQueue<Duration>> topicLatencies : | ||
perTopicRpcLatencies().entrySet()) { | ||
Histogram topicHistogram; | ||
if (latencyHistograms.containsKey(topicLatencies.getKey())) { | ||
topicHistogram = latencyHistograms.get(topicLatencies.getKey()); | ||
} else { | ||
topicHistogram = | ||
KafkaSinkMetrics.createRPCLatencyHistogram( | ||
KafkaSinkMetrics.RpcMethod.POLL, topicLatencies.getKey()); | ||
latencyHistograms.put(topicLatencies.getKey(), topicHistogram); | ||
} | ||
// update all the latencies | ||
for (Duration d : topicLatencies.getValue()) { | ||
Preconditions.checkArgumentNotNull(topicHistogram); | ||
topicHistogram.update(d.toMillis()); | ||
} | ||
} | ||
} | ||
|
||
/** | ||
* Export all metrics recorded in this instance to the underlying {@code perWorkerMetrics} | ||
* containers. This function will only report metrics once per instance. Subsequent calls to | ||
* this function will no-op. | ||
*/ | ||
@Override | ||
public void updateKafkaMetrics() { | ||
if (!isWritable().compareAndSet(true, false)) { | ||
LOG.warn("Updating stale Kafka metrics container"); | ||
return; | ||
} | ||
recordRpcLatencyMetrics(); | ||
} | ||
} | ||
} |
89 changes: 89 additions & 0 deletions
89
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaSinkMetrics.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,89 @@ | ||
/* | ||
* 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.beam.sdk.io.kafka; | ||
|
||
import org.apache.beam.sdk.metrics.DelegatingHistogram; | ||
import org.apache.beam.sdk.metrics.Histogram; | ||
import org.apache.beam.sdk.metrics.LabeledMetricNameUtils; | ||
import org.apache.beam.sdk.metrics.MetricName; | ||
import org.apache.beam.sdk.util.HistogramData; | ||
|
||
/** | ||
* Helper class to create per worker metrics for Kafka Sink stages. | ||
* | ||
* <p>Metrics will be in the namespace 'KafkaSink' and have their name formatted as: | ||
* | ||
* <p>'{baseName}-{metricLabelKey1}:{metricLabelVal1};...{metricLabelKeyN}:{metricLabelValN};' ???? | ||
*/ | ||
|
||
// TODO, refactor out common parts for BQ sink, so it can be reused with other sinks, eg, GCS? | ||
// @SuppressWarnings("unused") | ||
public class KafkaSinkMetrics { | ||
private static boolean supportKafkaMetrics = false; | ||
|
||
public static final String METRICS_NAMESPACE = "KafkaSink"; | ||
|
||
// Base Metric names | ||
private static final String RPC_LATENCY = "RpcLatency"; | ||
|
||
// Kafka Consumer Method names | ||
enum RpcMethod { | ||
POLL, | ||
} | ||
|
||
// Metric labels | ||
private static final String TOPIC_LABEL = "topic_name"; | ||
private static final String RPC_METHOD = "rpc_method"; | ||
|
||
/** | ||
* Creates an Histogram metric to record RPC latency. Metric will have name. | ||
* | ||
* <p>'RpcLatency*rpc_method:{method};topic_name:{topic};' | ||
* | ||
* @param method Kafka method associated with this metric. | ||
* @param topic Kafka topic associated with this metric. | ||
* @return Histogram with exponential buckets with a sqrt(2) growth factor. | ||
*/ | ||
public static Histogram createRPCLatencyHistogram(RpcMethod method, String topic) { | ||
LabeledMetricNameUtils.MetricNameBuilder nameBuilder = | ||
LabeledMetricNameUtils.MetricNameBuilder.baseNameBuilder(RPC_LATENCY); | ||
nameBuilder.addLabel(RPC_METHOD, method.toString()); | ||
nameBuilder.addLabel(TOPIC_LABEL, topic); | ||
|
||
MetricName metricName = nameBuilder.build(METRICS_NAMESPACE); | ||
HistogramData.BucketType buckets = HistogramData.ExponentialBuckets.of(1, 17); | ||
|
||
return new DelegatingHistogram(metricName, buckets, false, true); | ||
} | ||
|
||
/** | ||
* Returns a container to store metrics for Kafka metrics in Unbounded Readed. If these metrics | ||
* are disabled, then we return a no-op container. | ||
*/ | ||
static KafkaMetrics kafkaMetrics() { | ||
if (supportKafkaMetrics) { | ||
return KafkaMetrics.KafkaMetricsImpl.create(); | ||
} else { | ||
return KafkaMetrics.NoOpKafkaMetrics.getInstance(); | ||
} | ||
} | ||
|
||
public static void setSupportKafkaMetrics(boolean supportKafkaMetrics) { | ||
KafkaSinkMetrics.supportKafkaMetrics = supportKafkaMetrics; | ||
} | ||
} |
Oops, something went wrong.