-
Notifications
You must be signed in to change notification settings - Fork 3.6k
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
[feat][broker] PIP-264: Add OpenTelemetry consumer metrics #22693
Merged
merlimat
merged 22 commits into
apache:master
from
dragosvictor:dmisca-pip-264-consumer-metrics
May 10, 2024
Merged
Changes from all commits
Commits
Show all changes
22 commits
Select commit
Hold shift + click to select a range
e5f259f
Add consumer stats draft
dragosvictor fd36f1d
Add Consumer.messageAckCounter
dragosvictor 33faec1
Add Consumer.msgRedeliverCounter
dragosvictor 034c667
Add draft consumer metrics implementation
dragosvictor 22d55f0
Integrate consumer stats into PulsarService
dragosvictor 7bcc311
Cosmetic fixes
dragosvictor 925d5be
Add attributes
dragosvictor 62ba34d
Add attribute pulsar.consumer.blocked
dragosvictor b912ac2
Add consumer metrics test
dragosvictor 5bf88d2
Add attribute pulsar.client.address
dragosvictor 895dafc
Add attribute pulsar.client.version
dragosvictor 393f891
Add attribute pulsar.consumer.connected_since
dragosvictor f62fd38
Add attribute pulsar.consumer.metadata
dragosvictor d2386f0
Fix broken BrokerServiceLookupTest.testMultipleBrokerLookup
dragosvictor 8d5f4a8
Test fixes
dragosvictor 67fc99b
Validate consumer metadata attribute in test
dragosvictor eec08f0
Merge remote-tracking branch 'origin/master' into dmisca-pip-264-cons…
dragosvictor 794ee76
Close metric resources
dragosvictor 5b0c9a1
Test cleanup
dragosvictor abe311b
Comment fix
dragosvictor bad3431
Test cleanup
dragosvictor 0181f82
Test fix
dragosvictor File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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
170 changes: 170 additions & 0 deletions
170
pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryConsumerStats.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,170 @@ | ||
/* | ||
* 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.pulsar.broker.stats; | ||
|
||
import io.opentelemetry.api.common.Attributes; | ||
import io.opentelemetry.api.metrics.BatchCallback; | ||
import io.opentelemetry.api.metrics.ObservableLongMeasurement; | ||
import java.util.Collection; | ||
import java.util.Optional; | ||
import org.apache.pulsar.broker.PulsarService; | ||
import org.apache.pulsar.broker.service.Consumer; | ||
import org.apache.pulsar.broker.service.Subscription; | ||
import org.apache.pulsar.broker.service.Topic; | ||
import org.apache.pulsar.common.naming.TopicName; | ||
import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; | ||
|
||
public class OpenTelemetryConsumerStats implements AutoCloseable { | ||
|
||
// Replaces pulsar_consumer_msg_rate_out | ||
public static final String MESSAGE_OUT_COUNTER = "pulsar.broker.consumer.message.outgoing.count"; | ||
private final ObservableLongMeasurement messageOutCounter; | ||
|
||
// Replaces pulsar_consumer_msg_throughput_out | ||
public static final String BYTES_OUT_COUNTER = "pulsar.broker.consumer.message.outgoing.size"; | ||
private final ObservableLongMeasurement bytesOutCounter; | ||
|
||
// Replaces pulsar_consumer_msg_ack_rate | ||
public static final String MESSAGE_ACK_COUNTER = "pulsar.broker.consumer.message.ack.count"; | ||
private final ObservableLongMeasurement messageAckCounter; | ||
|
||
// Replaces pulsar_consumer_msg_rate_redeliver | ||
public static final String MESSAGE_REDELIVER_COUNTER = "pulsar.broker.consumer.message.redeliver.count"; | ||
private final ObservableLongMeasurement messageRedeliverCounter; | ||
|
||
// Replaces pulsar_consumer_unacked_messages | ||
public static final String MESSAGE_UNACKNOWLEDGED_COUNTER = "pulsar.broker.consumer.message.unack.count"; | ||
private final ObservableLongMeasurement messageUnacknowledgedCounter; | ||
|
||
// Replaces pulsar_consumer_available_permits | ||
public static final String MESSAGE_PERMITS_COUNTER = "pulsar.broker.consumer.permit.count"; | ||
private final ObservableLongMeasurement messagePermitsCounter; | ||
|
||
private final BatchCallback batchCallback; | ||
|
||
public OpenTelemetryConsumerStats(PulsarService pulsar) { | ||
var meter = pulsar.getOpenTelemetry().getMeter(); | ||
|
||
messageOutCounter = meter | ||
.counterBuilder(MESSAGE_OUT_COUNTER) | ||
.setUnit("{message}") | ||
.setDescription("The total number of messages dispatched to this consumer.") | ||
.buildObserver(); | ||
|
||
bytesOutCounter = meter | ||
.counterBuilder(BYTES_OUT_COUNTER) | ||
.setUnit("By") | ||
.setDescription("The total number of messages bytes dispatched to this consumer.") | ||
.buildObserver(); | ||
|
||
messageAckCounter = meter | ||
.counterBuilder(MESSAGE_ACK_COUNTER) | ||
.setUnit("{ack}") | ||
.setDescription("The total number of message acknowledgments received from this consumer.") | ||
.buildObserver(); | ||
|
||
messageRedeliverCounter = meter | ||
.counterBuilder(MESSAGE_REDELIVER_COUNTER) | ||
.setUnit("{message}") | ||
.setDescription("The total number of messages that have been redelivered to this consumer.") | ||
.buildObserver(); | ||
|
||
messageUnacknowledgedCounter = meter | ||
.upDownCounterBuilder(MESSAGE_UNACKNOWLEDGED_COUNTER) | ||
.setUnit("{message}") | ||
.setDescription("The total number of messages unacknowledged by this consumer.") | ||
.buildObserver(); | ||
|
||
messagePermitsCounter = meter | ||
.upDownCounterBuilder(MESSAGE_PERMITS_COUNTER) | ||
.setUnit("{permit}") | ||
.setDescription("The number of permits currently available for this consumer.") | ||
.buildObserver(); | ||
|
||
batchCallback = meter.batchCallback(() -> pulsar.getBrokerService() | ||
.getTopics() | ||
.values() | ||
.stream() | ||
.map(topicFuture -> topicFuture.getNow(Optional.empty())) | ||
.filter(Optional::isPresent) | ||
.map(Optional::get) | ||
.map(Topic::getSubscriptions) | ||
.flatMap(s -> s.values().stream()) | ||
.map(Subscription::getConsumers) | ||
.flatMap(Collection::stream) | ||
.forEach(this::recordMetricsForConsumer), | ||
messageOutCounter, | ||
bytesOutCounter, | ||
messageAckCounter, | ||
messageRedeliverCounter, | ||
messageUnacknowledgedCounter, | ||
messagePermitsCounter); | ||
} | ||
|
||
@Override | ||
public void close() { | ||
batchCallback.close(); | ||
} | ||
|
||
private void recordMetricsForConsumer(Consumer consumer) { | ||
var subscription = consumer.getSubscription(); | ||
var topicName = TopicName.get(subscription.getTopic().getName()); | ||
|
||
var builder = Attributes.builder() | ||
.put(OpenTelemetryAttributes.PULSAR_CONSUMER_NAME, consumer.consumerName()) | ||
.put(OpenTelemetryAttributes.PULSAR_CONSUMER_ID, consumer.consumerId()) | ||
.put(OpenTelemetryAttributes.PULSAR_CONSUMER_CONNECTED_SINCE, | ||
consumer.getConnectedSince().getEpochSecond()) | ||
.put(OpenTelemetryAttributes.PULSAR_SUBSCRIPTION_NAME, subscription.getName()) | ||
.put(OpenTelemetryAttributes.PULSAR_SUBSCRIPTION_TYPE, consumer.subType().toString()) | ||
.put(OpenTelemetryAttributes.PULSAR_DOMAIN, topicName.getDomain().toString()) | ||
.put(OpenTelemetryAttributes.PULSAR_TENANT, topicName.getTenant()) | ||
.put(OpenTelemetryAttributes.PULSAR_NAMESPACE, topicName.getNamespace()) | ||
.put(OpenTelemetryAttributes.PULSAR_TOPIC, topicName.getPartitionedTopicName()); | ||
if (topicName.isPartitioned()) { | ||
builder.put(OpenTelemetryAttributes.PULSAR_PARTITION_INDEX, topicName.getPartitionIndex()); | ||
} | ||
var clientAddress = consumer.getClientAddressAndPort(); | ||
if (clientAddress != null) { | ||
builder.put(OpenTelemetryAttributes.PULSAR_CLIENT_ADDRESS, clientAddress); | ||
} | ||
var clientVersion = consumer.getClientVersion(); | ||
if (clientVersion != null) { | ||
builder.put(OpenTelemetryAttributes.PULSAR_CLIENT_VERSION, clientVersion); | ||
} | ||
var metadataList = consumer.getMetadata() | ||
.entrySet() | ||
.stream() | ||
.map(e -> String.format("%s:%s", e.getKey(), e.getValue())) | ||
.toList(); | ||
builder.put(OpenTelemetryAttributes.PULSAR_CONSUMER_METADATA, metadataList); | ||
var attributes = builder.build(); | ||
|
||
messageOutCounter.record(consumer.getMsgOutCounter(), attributes); | ||
bytesOutCounter.record(consumer.getBytesOutCounter(), attributes); | ||
messageAckCounter.record(consumer.getMessageAckCounter(), attributes); | ||
messageRedeliverCounter.record(consumer.getMessageRedeliverCounter(), attributes); | ||
messageUnacknowledgedCounter.record(consumer.getUnackedMessages(), | ||
Attributes.builder() | ||
.putAll(attributes) | ||
.put(OpenTelemetryAttributes.PULSAR_CONSUMER_BLOCKED, consumer.isBlocked()) | ||
.build()); | ||
messagePermitsCounter.record(consumer.getAvailablePermits(), attributes); | ||
} | ||
} |
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@merlimat @dragosvictor This and other attributes here are super high cardinality - why are we recording with them? This will "kill" any TSDB and be very costly. Even when compared with
pulsar_consumer_msg_rate_out
- it didn't have all those attributes.