Skip to content

Commit

Permalink
[feat][broker] PIP-264: Add OpenTelemetry producer metrics (apache#22882
Browse files Browse the repository at this point in the history
)
  • Loading branch information
dragosvictor authored Jun 15, 2024
1 parent f122817 commit f83dbe9
Show file tree
Hide file tree
Showing 10 changed files with 348 additions and 41 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -112,6 +112,7 @@
import org.apache.pulsar.broker.service.schema.SchemaStorageFactory;
import org.apache.pulsar.broker.stats.MetricsGenerator;
import org.apache.pulsar.broker.stats.OpenTelemetryConsumerStats;
import org.apache.pulsar.broker.stats.OpenTelemetryProducerStats;
import org.apache.pulsar.broker.stats.OpenTelemetryTopicStats;
import org.apache.pulsar.broker.stats.PulsarBrokerOpenTelemetry;
import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsServlet;
Expand Down Expand Up @@ -258,6 +259,7 @@ public class PulsarService implements AutoCloseable, ShutdownService {
private final PulsarBrokerOpenTelemetry openTelemetry;
private OpenTelemetryTopicStats openTelemetryTopicStats;
private OpenTelemetryConsumerStats openTelemetryConsumerStats;
private OpenTelemetryProducerStats openTelemetryProducerStats;

private TransactionMetadataStoreService transactionMetadataStoreService;
private TransactionBufferProvider transactionBufferProvider;
Expand Down Expand Up @@ -676,6 +678,10 @@ public CompletableFuture<Void> closeAsync() {
brokerClientSharedTimer.stop();
monotonicSnapshotClock.close();

if (openTelemetryProducerStats != null) {
openTelemetryProducerStats.close();
openTelemetryProducerStats = null;
}
if (openTelemetryConsumerStats != null) {
openTelemetryConsumerStats.close();
openTelemetryConsumerStats = null;
Expand Down Expand Up @@ -827,6 +833,7 @@ public void start() throws PulsarServerException {

openTelemetryTopicStats = new OpenTelemetryTopicStats(this);
openTelemetryConsumerStats = new OpenTelemetryConsumerStats(this);
openTelemetryProducerStats = new OpenTelemetryProducerStats(this);

localMetadataSynchronizer = StringUtils.isNotBlank(config.getMetadataSyncEventTopic())
? new PulsarMetadataEventSynchronizer(this, config.getMetadataSyncEventTopic())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,10 +23,12 @@
import static org.apache.pulsar.common.protocol.Commands.hasChecksum;
import static org.apache.pulsar.common.protocol.Commands.readChecksum;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.CaseFormat;
import com.google.common.base.MoreObjects;
import io.netty.buffer.ByteBuf;
import io.netty.util.Recycler;
import io.netty.util.Recycler.Handle;
import io.opentelemetry.api.common.Attributes;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
Expand All @@ -36,6 +38,7 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLongFieldUpdater;
import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
import org.apache.bookkeeper.mledger.Position;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.broker.intercept.BrokerInterceptor;
Expand All @@ -57,8 +60,8 @@
import org.apache.pulsar.common.policies.data.stats.PublisherStatsImpl;
import org.apache.pulsar.common.protocol.Commands;
import org.apache.pulsar.common.protocol.schema.SchemaVersion;
import org.apache.pulsar.common.stats.Rate;
import org.apache.pulsar.common.util.DateFormatter;
import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand All @@ -74,10 +77,6 @@ public class Producer {
private final long producerId;
private final String appId;
private final BrokerInterceptor brokerInterceptor;
private Rate msgIn;
private Rate chunkedMessageRate;
// it records msg-drop rate only for non-persistent topic
private final Rate msgDrop;

private volatile long pendingPublishAcks = 0;
private static final AtomicLongFieldUpdater<Producer> pendingPublishAcksUpdater = AtomicLongFieldUpdater
Expand All @@ -87,6 +86,10 @@ public class Producer {
private final CompletableFuture<Void> closeFuture;

private final PublisherStatsImpl stats;
private volatile Attributes attributes = null;
private static final AtomicReferenceFieldUpdater<Producer, Attributes> ATTRIBUTES_FIELD_UPDATER =
AtomicReferenceFieldUpdater.newUpdater(Producer.class, Attributes.class, "attributes");

private final boolean isRemote;
private final String remoteCluster;
private final boolean isNonPersistentTopic;
Expand Down Expand Up @@ -118,10 +121,7 @@ public Producer(Topic topic, TransportCnx cnx, long producerId, String producerN
this.epoch = epoch;
this.closeFuture = new CompletableFuture<>();
this.appId = appId;
this.msgIn = new Rate();
this.chunkedMessageRate = new Rate();
this.isNonPersistentTopic = topic instanceof NonPersistentTopic;
this.msgDrop = this.isNonPersistentTopic ? new Rate() : null;
this.isShadowTopic =
topic instanceof PersistentTopic && ((PersistentTopic) topic).getShadowSourceTopic().isPresent();

Expand Down Expand Up @@ -270,7 +270,7 @@ public boolean checkAndStartPublish(long producerId, long sequenceId, ByteBuf he
private void publishMessageToTopic(ByteBuf headersAndPayload, long sequenceId, long batchSize, boolean isChunked,
boolean isMarker, Position position) {
MessagePublishContext messagePublishContext =
MessagePublishContext.get(this, sequenceId, msgIn, headersAndPayload.readableBytes(),
MessagePublishContext.get(this, sequenceId, headersAndPayload.readableBytes(),
batchSize, isChunked, System.nanoTime(), isMarker, position);
if (brokerInterceptor != null) {
brokerInterceptor
Expand All @@ -282,7 +282,7 @@ private void publishMessageToTopic(ByteBuf headersAndPayload, long sequenceId, l
private void publishMessageToTopic(ByteBuf headersAndPayload, long lowestSequenceId, long highestSequenceId,
long batchSize, boolean isChunked, boolean isMarker, Position position) {
MessagePublishContext messagePublishContext = MessagePublishContext.get(this, lowestSequenceId,
highestSequenceId, msgIn, headersAndPayload.readableBytes(), batchSize,
highestSequenceId, headersAndPayload.readableBytes(), batchSize,
isChunked, System.nanoTime(), isMarker, position);
if (brokerInterceptor != null) {
brokerInterceptor
Expand Down Expand Up @@ -339,8 +339,8 @@ private void publishOperationCompleted() {
}

public void recordMessageDrop(int batchSize) {
if (this.isNonPersistentTopic) {
msgDrop.recordEvent(batchSize);
if (stats instanceof NonPersistentPublisherStatsImpl nonPersistentPublisherStats) {
nonPersistentPublisherStats.recordMsgDrop(batchSize);
}
}

Expand Down Expand Up @@ -374,7 +374,6 @@ private static final class MessagePublishContext implements PublishContext, Runn
private long sequenceId;
private long ledgerId;
private long entryId;
private Rate rateIn;
private int msgSize;
private long batchSize;
private boolean chunked;
Expand Down Expand Up @@ -536,13 +535,13 @@ public void run() {
}

// stats
rateIn.recordMultipleEvents(batchSize, msgSize);
producer.stats.recordMsgIn(batchSize, msgSize);
producer.topic.recordAddLatency(System.nanoTime() - startTimeNs, TimeUnit.NANOSECONDS);
producer.cnx.getCommandSender().sendSendReceiptResponse(producer.producerId, sequenceId, highestSequenceId,
ledgerId, entryId);
producer.cnx.completedSendOperation(producer.isNonPersistentTopic, msgSize);
if (this.chunked) {
producer.chunkedMessageRate.recordEvent();
producer.stats.recordChunkedMsgIn();
}
producer.publishOperationCompleted();
if (producer.brokerInterceptor != null) {
Expand All @@ -552,12 +551,11 @@ public void run() {
recycle();
}

static MessagePublishContext get(Producer producer, long sequenceId, Rate rateIn, int msgSize,
long batchSize, boolean chunked, long startTimeNs, boolean isMarker, Position position) {
static MessagePublishContext get(Producer producer, long sequenceId, int msgSize, long batchSize,
boolean chunked, long startTimeNs, boolean isMarker, Position position) {
MessagePublishContext callback = RECYCLER.get();
callback.producer = producer;
callback.sequenceId = sequenceId;
callback.rateIn = rateIn;
callback.msgSize = msgSize;
callback.batchSize = batchSize;
callback.chunked = chunked;
Expand All @@ -573,13 +571,12 @@ static MessagePublishContext get(Producer producer, long sequenceId, Rate rateIn
return callback;
}

static MessagePublishContext get(Producer producer, long lowestSequenceId, long highestSequenceId, Rate rateIn,
int msgSize, long batchSize, boolean chunked, long startTimeNs, boolean isMarker, Position position) {
static MessagePublishContext get(Producer producer, long lowestSequenceId, long highestSequenceId, int msgSize,
long batchSize, boolean chunked, long startTimeNs, boolean isMarker, Position position) {
MessagePublishContext callback = RECYCLER.get();
callback.producer = producer;
callback.sequenceId = lowestSequenceId;
callback.highestSequenceId = highestSequenceId;
callback.rateIn = rateIn;
callback.msgSize = msgSize;
callback.batchSize = batchSize;
callback.originalProducerName = null;
Expand Down Expand Up @@ -628,7 +625,6 @@ public void recycle() {
highestSequenceId = -1L;
originalSequenceId = -1L;
originalHighestSequenceId = -1L;
rateIn = null;
msgSize = 0;
ledgerId = -1L;
entryId = -1L;
Expand Down Expand Up @@ -733,25 +729,12 @@ public void topicMigrated(Optional<ClusterUrl> clusterUrl) {
}

public void updateRates() {
msgIn.calculateRate();
chunkedMessageRate.calculateRate();
stats.msgRateIn = msgIn.getRate();
stats.msgThroughputIn = msgIn.getValueRate();
stats.averageMsgSize = msgIn.getAverageValue();
stats.chunkedMessageRate = chunkedMessageRate.getRate();
if (chunkedMessageRate.getCount() > 0 && this.topic instanceof PersistentTopic) {
((PersistentTopic) this.topic).msgChunkPublished = true;
}
if (this.isNonPersistentTopic) {
msgDrop.calculateRate();
((NonPersistentPublisherStatsImpl) stats).msgDropRate = msgDrop.getValueRate();
stats.calculateRates();
if (stats.getMsgChunkIn().getCount() > 0 && topic instanceof PersistentTopic persistentTopic) {
persistentTopic.msgChunkPublished = true;
}
}

public void updateRates(int numOfMessages, long msgSizeInBytes) {
msgIn.recordMultipleEvents(numOfMessages, msgSizeInBytes);
}

public boolean isRemote() {
return isRemote;
}
Expand Down Expand Up @@ -817,7 +800,7 @@ public void publishTxnMessage(TxnID txnID, long producerId, long sequenceId, lon
return;
}
MessagePublishContext messagePublishContext =
MessagePublishContext.get(this, sequenceId, highSequenceId, msgIn,
MessagePublishContext.get(this, sequenceId, highSequenceId,
headersAndPayload.readableBytes(), batchSize, isChunked, System.nanoTime(), isMarker, null);
if (brokerInterceptor != null) {
brokerInterceptor
Expand Down Expand Up @@ -871,4 +854,29 @@ public void incrementThrottleCount() {
public void decrementThrottleCount() {
cnx.decrementThrottleCount();
}

public Attributes getOpenTelemetryAttributes() {
if (attributes != null) {
return attributes;
}
return ATTRIBUTES_FIELD_UPDATER.updateAndGet(this, old -> {
if (old != null) {
return old;
}
var topicName = TopicName.get(topic.getName());
var builder = Attributes.builder()
.put(OpenTelemetryAttributes.PULSAR_PRODUCER_NAME, producerName)
.put(OpenTelemetryAttributes.PULSAR_PRODUCER_ID, producerId)
.put(OpenTelemetryAttributes.PULSAR_PRODUCER_ACCESS_MODE,
CaseFormat.UPPER_CAMEL.to(CaseFormat.LOWER_UNDERSCORE, accessMode.name()))
.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());
}
return builder.build();
});
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,95 @@
/*
* 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.metrics.BatchCallback;
import io.opentelemetry.api.metrics.ObservableLongMeasurement;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.broker.service.Producer;
import org.apache.pulsar.common.policies.data.stats.NonPersistentPublisherStatsImpl;

public class OpenTelemetryProducerStats implements AutoCloseable {

// Replaces pulsar_producer_msg_rate_in
public static final String MESSAGE_IN_COUNTER = "pulsar.broker.producer.message.incoming.count";
private final ObservableLongMeasurement messageInCounter;

// Replaces pulsar_producer_msg_throughput_in
public static final String BYTES_IN_COUNTER = "pulsar.broker.producer.message.incoming.size";
private final ObservableLongMeasurement bytesInCounter;

public static final String MESSAGE_DROP_COUNTER = "pulsar.broker.producer.message.drop.count";
private final ObservableLongMeasurement messageDropCounter;

private final BatchCallback batchCallback;

public OpenTelemetryProducerStats(PulsarService pulsar) {
var meter = pulsar.getOpenTelemetry().getMeter();

messageInCounter = meter
.counterBuilder(MESSAGE_IN_COUNTER)
.setUnit("{message}")
.setDescription("The total number of messages received from this producer.")
.buildObserver();

bytesInCounter = meter
.counterBuilder(BYTES_IN_COUNTER)
.setUnit("By")
.setDescription("The total number of messages bytes received from this producer.")
.buildObserver();

messageDropCounter = meter
.counterBuilder(MESSAGE_DROP_COUNTER)
.setUnit("{message}")
.setDescription("The total number of messages dropped from this producer.")
.buildObserver();

batchCallback = meter.batchCallback(() -> pulsar.getBrokerService()
.getTopics()
.values()
.stream()
.filter(future -> future.isDone() && !future.isCompletedExceptionally())
.map(CompletableFuture::join)
.filter(Optional::isPresent)
.flatMap(topic -> topic.get().getProducers().values().stream())
.forEach(this::recordMetricsForProducer),
messageInCounter,
bytesInCounter,
messageDropCounter);
}

@Override
public void close() {
batchCallback.close();
}

private void recordMetricsForProducer(Producer producer) {
var attributes = producer.getOpenTelemetryAttributes();
var stats = producer.getStats();

messageInCounter.record(stats.getMsgInCounter(), attributes);
bytesInCounter.record(stats.getBytesInCounter(), attributes);

if (stats instanceof NonPersistentPublisherStatsImpl nonPersistentStats) {
messageDropCounter.record(nonPersistentStats.getMsgDropCount(), attributes);
}
}
}
Loading

0 comments on commit f83dbe9

Please sign in to comment.