From babad52f33007bd4a8c35b6e8a506ddf2a3673de Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Wed, 28 Feb 2024 16:57:29 -0800 Subject: [PATCH 01/22] WIP: PIP-342: OTel client metrics support --- ...ListenersWithInternalListenerNameTest.java | 6 +- pulsar-client-api/pom.xml | 6 ++ .../pulsar/client/api/ClientBuilder.java | 10 ++- .../pulsar/client/api/ConsumerStats.java | 6 +- .../pulsar/client/api/MetricsCardinality.java | 47 +++++++++++ .../pulsar/client/api/ProducerStats.java | 6 +- pulsar-client/pom.xml | 5 ++ .../impl/BatchMessageContainerImpl.java | 4 +- .../client/impl/BinaryProtoLookupService.java | 48 +++++++++++ .../pulsar/client/impl/ClientBuilderImpl.java | 14 ++++ .../apache/pulsar/client/impl/ClientCnx.java | 23 +++++- .../pulsar/client/impl/ConnectionPool.java | 28 +++++-- .../pulsar/client/impl/ConsumerBase.java | 5 +- .../pulsar/client/impl/ConsumerImpl.java | 58 ++++++++++++++ .../pulsar/client/impl/HttpLookupService.java | 72 ++++++++++++++++- .../pulsar/client/impl/ProducerImpl.java | 79 ++++++++++++++++--- .../pulsar/client/impl/PulsarClientImpl.java | 20 ++++- .../client/impl/UnAckedMessageTracker.java | 16 +++- .../impl/conf/ClientConfigurationData.java | 9 ++- .../pulsar/client/impl/metrics/Counter.java | 28 +++++++ .../impl/metrics/InstrumentProvider.java | 59 ++++++++++++++ .../client/impl/metrics/LatencyHistogram.java | 46 +++++++++++ .../pulsar/client/impl/metrics/Unit.java | 59 ++++++++++++++ .../client/impl/metrics/UpDownCounter.java | 35 ++++++++ .../AcknowledgementsGroupingTrackerTest.java | 2 +- .../conf/ClientConfigurationDataTest.java | 32 ++++++++ pulsar-testclient/pom.xml | 17 ++++ .../pulsar/testclient/PerfClientUtils.java | 7 +- 28 files changed, 711 insertions(+), 36 deletions(-) create mode 100644 pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MetricsCardinality.java create mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Counter.java create mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/InstrumentProvider.java create mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java create mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Unit.java create mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/UpDownCounter.java diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PulsarMultiListenersWithInternalListenerNameTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PulsarMultiListenersWithInternalListenerNameTest.java index 956b834e33435..e420a2949882f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PulsarMultiListenersWithInternalListenerNameTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PulsarMultiListenersWithInternalListenerNameTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.api; +import io.opentelemetry.api.OpenTelemetry; import java.io.IOException; import java.io.UncheckedIOException; import java.net.InetAddress; @@ -44,6 +45,7 @@ import org.apache.pulsar.client.impl.LookupService; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfo; @@ -137,7 +139,7 @@ private void doFindBrokerWithListenerName(boolean useHttp) throws Exception { conf.setMaxLookupRedirects(10); @Cleanup - LookupService lookupService = useHttp ? new HttpLookupService(conf, eventExecutors) : + LookupService lookupService = useHttp ? new HttpLookupService(new InstrumentProvider(new ClientConfigurationData()), conf, eventExecutors) : new BinaryProtoLookupService((PulsarClientImpl) this.pulsarClient, lookupUrl.toString(), "internal", false, this.executorService); TopicName topicName = TopicName.get("persistent://public/default/test"); @@ -172,7 +174,7 @@ public void testHttpLookupRedirect() throws Exception { conf.setMaxLookupRedirects(10); @Cleanup - HttpLookupService lookupService = new HttpLookupService(conf, eventExecutors); + HttpLookupService lookupService = new HttpLookupService(new InstrumentProvider(new ClientConfigurationData()), conf, eventExecutors); NamespaceService namespaceService = pulsar.getNamespaceService(); LookupResult lookupResult = new LookupResult(pulsar.getWebServiceAddress(), null, diff --git a/pulsar-client-api/pom.xml b/pulsar-client-api/pom.xml index d8b51713da832..35bdf73374b3e 100644 --- a/pulsar-client-api/pom.xml +++ b/pulsar-client-api/pom.xml @@ -46,6 +46,12 @@ protobuf-java provided + + + io.opentelemetry + opentelemetry-api + provided + diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java index b180f6ba7f906..e8ceb4e4f5ffa 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.api; +import io.opentelemetry.api.OpenTelemetry; import java.io.Serializable; import java.net.InetSocketAddress; import java.time.Clock; @@ -451,7 +452,7 @@ ClientBuilder authentication(String authPluginClassName, Map aut ClientBuilder memoryLimit(long memoryLimit, SizeUnit unit); /** - * Set the interval between each stat info (default: 60 seconds) Stats will be activated with positive + * Set the interval between each stat info (default: disabled) Stats will be activated with positive * statsInterval It should be set to at least 1 second. * * @param statsInterval @@ -459,7 +460,10 @@ ClientBuilder authentication(String authPluginClassName, Map aut * @param unit * time unit for {@code statsInterval} * @return the client builder instance + * + * @deprecated @see {@link #openTelemetry(OpenTelemetry)} */ + @Deprecated ClientBuilder statsInterval(long statsInterval, TimeUnit unit); /** @@ -554,6 +558,10 @@ ClientBuilder authentication(String authPluginClassName, Map aut */ ClientBuilder enableBusyWait(boolean enableBusyWait); + ClientBuilder openTelemetry(io.opentelemetry.api.OpenTelemetry openTelemetry); + + ClientBuilder openTelemetryMetricsCardinality(MetricsCardinality metricsCardinality); + /** * The clock used by the pulsar client. * diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerStats.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerStats.java index 7935e05d55b66..d36cb846c12c6 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerStats.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerStats.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.api; +import io.opentelemetry.api.OpenTelemetry; import java.io.Serializable; import java.util.Collections; import java.util.Map; @@ -29,9 +30,12 @@ * *

All the stats are relative to the last recording period. The interval of the stats refreshes is configured with * {@link ClientBuilder#statsInterval(long, java.util.concurrent.TimeUnit)} with a default of 1 minute. + * + * @deprecated use {@link ClientBuilder#openTelemetry(OpenTelemetry)} to enable stats */ @InterfaceAudience.Public -@InterfaceStability.Stable +@InterfaceStability.Evolving +@Deprecated public interface ConsumerStats extends Serializable { /** diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MetricsCardinality.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MetricsCardinality.java new file mode 100644 index 0000000000000..fd4275d3c0455 --- /dev/null +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MetricsCardinality.java @@ -0,0 +1,47 @@ +/* + * 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.client.api; + +public enum MetricsCardinality { + /** + * Do not add additional labels to metrics + */ + None, + + /** + * Label metrics by tenant + */ + Tenant, + + /** + * Label metrics by tenant and namespace + */ + Namespace, + + /** + * Label metrics by topic + */ + Topic, + + /** + * Label metrics by each partition + */ + Partition, +} diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ProducerStats.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ProducerStats.java index a26c20e740d37..9a9ade73669dd 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ProducerStats.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ProducerStats.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.api; +import io.opentelemetry.api.OpenTelemetry; import java.io.Serializable; import java.util.Collections; import java.util.Map; @@ -29,9 +30,12 @@ * *

All the stats are relative to the last recording period. The interval of the stats refreshes is configured with * {@link ClientBuilder#statsInterval(long, java.util.concurrent.TimeUnit)} with a default of 1 minute. + * + * @deprecated use {@link ClientBuilder#openTelemetry(OpenTelemetry)} to enable stats */ @InterfaceAudience.Public -@InterfaceStability.Stable +@InterfaceStability.Evolving +@Deprecated public interface ProducerStats extends Serializable { /** * @return the number of messages published in the last interval diff --git a/pulsar-client/pom.xml b/pulsar-client/pom.xml index 7424b12db5aa2..c775284b4d9ca 100644 --- a/pulsar-client/pom.xml +++ b/pulsar-client/pom.xml @@ -52,6 +52,11 @@ pkg + + io.opentelemetry + opentelemetry-api + + ${project.groupId} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageContainerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageContainerImpl.java index bf8c1f9de8201..e33719c14c48d 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageContainerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageContainerImpl.java @@ -263,7 +263,7 @@ public OpSendMsg createOpSendMsg() throws IOException { // Because when invoke `ProducerImpl.processOpSendMsg` on flush, // if `op.msg != null && isBatchMessagingEnabled()` checks true, it will call `batchMessageAndSend` to flush // messageContainers before publishing this one-batch message. - op = OpSendMsg.create(messages, cmd, messageMetadata.getSequenceId(), firstCallback, + op = OpSendMsg.create(producer, messages, cmd, messageMetadata.getSequenceId(), firstCallback, batchAllocatedSizeBytes); // NumMessagesInBatch and BatchSizeByte will not be serialized to the binary cmd. It's just useful for the @@ -314,7 +314,7 @@ public OpSendMsg createOpSendMsg() throws IOException { messageMetadata.getUncompressedSize(), encryptedPayload.readableBytes()); } - OpSendMsg op = OpSendMsg.create(messages, cmd, messageMetadata.getSequenceId(), + OpSendMsg op = OpSendMsg.create(producer, messages, cmd, messageMetadata.getSequenceId(), messageMetadata.getHighestSequenceId(), firstCallback, batchAllocatedSizeBytes); op.setNumMessagesInBatch(numMessagesInBatch); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java index bdf00844c1cd2..15692a3815c2f 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java @@ -20,6 +20,8 @@ import static java.lang.String.format; import io.netty.buffer.ByteBuf; +import io.opentelemetry.api.common.AttributeKey; +import io.opentelemetry.api.common.Attributes; import java.net.InetSocketAddress; import java.net.URI; import java.util.ArrayList; @@ -34,6 +36,7 @@ import org.apache.commons.lang3.mutable.MutableObject; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.SchemaSerializationException; +import org.apache.pulsar.client.impl.metrics.LatencyHistogram; import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace.Mode; import org.apache.pulsar.common.api.proto.CommandLookupTopicResponse; import org.apache.pulsar.common.api.proto.CommandLookupTopicResponse.LookupType; @@ -63,6 +66,11 @@ public class BinaryProtoLookupService implements LookupService { private final ConcurrentHashMap> partitionedMetadataInProgress = new ConcurrentHashMap<>(); + private final LatencyHistogram histoGetBroker; + private final LatencyHistogram histoGetTopicMetadata; + private final LatencyHistogram histoGetSchema; + private final LatencyHistogram histoListTopics; + public BinaryProtoLookupService(PulsarClientImpl client, String serviceUrl, boolean useTls, @@ -84,6 +92,21 @@ public BinaryProtoLookupService(PulsarClientImpl client, this.serviceNameResolver = new PulsarServiceNameResolver(); this.listenerName = listenerName; updateServiceUrl(serviceUrl); + + Attributes attrs = Attributes.of(AttributeKey.stringKey("transport-type"), "binary"); + + histoGetBroker = client.instrumentProvider().newLatencyHistogram("pulsar.client.lookup", + "Lookup operations", + attrs.toBuilder().put("type", "topic").build()); + histoGetTopicMetadata = client.instrumentProvider().newLatencyHistogram("pulsar.client.lookup", + "Lookup operations", + attrs.toBuilder().put("type", "metadata").build()); + histoGetSchema = client.instrumentProvider().newLatencyHistogram("pulsar.client.lookup", + "Lookup operations", + attrs.toBuilder().put("type", "schema").build()); + histoListTopics = client.instrumentProvider().newLatencyHistogram("pulsar.client.lookup", + "Lookup operations", + attrs.toBuilder().put("type", "list-topics").build()); } @Override @@ -99,12 +122,20 @@ public void updateServiceUrl(String serviceUrl) throws PulsarClientException { * @return broker-socket-address that serves given topic */ public CompletableFuture getBroker(TopicName topicName) { + long startTime = System.nanoTime(); final MutableObject newFutureCreated = new MutableObject<>(); try { return lookupInProgress.computeIfAbsent(topicName, tpName -> { CompletableFuture newFuture = findBroker(serviceNameResolver.resolveHost(), false, topicName, 0); newFutureCreated.setValue(newFuture); + + newFuture.thenRun(() -> { + histoGetBroker.recordSuccess(System.nanoTime() - startTime); + }).exceptionally(x -> { + histoGetBroker.recordFailure(System.nanoTime() - startTime); + return null; + }); return newFuture; }); } finally { @@ -121,12 +152,19 @@ public CompletableFuture getBroker(TopicName topicName) { * */ public CompletableFuture getPartitionedTopicMetadata(TopicName topicName) { + long startTime = System.nanoTime(); final MutableObject newFutureCreated = new MutableObject<>(); try { return partitionedMetadataInProgress.computeIfAbsent(topicName, tpName -> { CompletableFuture newFuture = getPartitionedTopicMetadata(serviceNameResolver.resolveHost(), topicName); newFutureCreated.setValue(newFuture); + newFuture.thenRun(() -> { + histoGetBroker.recordSuccess(System.nanoTime() - startTime); + }).exceptionally(x -> { + histoGetBroker.recordFailure(System.nanoTime() - startTime); + return null; + }); return newFuture; }); } finally { @@ -224,6 +262,7 @@ private CompletableFuture findBroker(InetSocketAddress socket private CompletableFuture getPartitionedTopicMetadata(InetSocketAddress socketAddress, TopicName topicName) { + long startTime = System.nanoTime(); CompletableFuture partitionFuture = new CompletableFuture<>(); client.getCnxPool().getConnection(socketAddress).thenAccept(clientCnx -> { @@ -231,11 +270,13 @@ private CompletableFuture getPartitionedTopicMetadata( ByteBuf request = Commands.newPartitionMetadataRequest(topicName.toString(), requestId); clientCnx.newLookup(request, requestId).whenComplete((r, t) -> { if (t != null) { + histoGetTopicMetadata.recordFailure(System.nanoTime() - startTime); log.warn("[{}] failed to get Partitioned metadata : {}", topicName, t.getMessage(), t); partitionFuture.completeExceptionally(t); } else { try { + histoGetTopicMetadata.recordSuccess(System.nanoTime() - startTime); partitionFuture.complete(new PartitionedTopicMetadata(r.partitions)); } catch (Exception e) { partitionFuture.completeExceptionally(new PulsarClientException.LookupException( @@ -263,6 +304,7 @@ public CompletableFuture> getSchema(TopicName topicName) { @Override public CompletableFuture> getSchema(TopicName topicName, byte[] version) { + long startTime = System.nanoTime(); CompletableFuture> schemaFuture = new CompletableFuture<>(); if (version != null && version.length == 0) { schemaFuture.completeExceptionally(new SchemaSerializationException("Empty schema version")); @@ -275,10 +317,12 @@ public CompletableFuture> getSchema(TopicName topicName, by Optional.ofNullable(BytesSchemaVersion.of(version))); clientCnx.sendGetSchema(request, requestId).whenComplete((r, t) -> { if (t != null) { + histoGetSchema.recordFailure(System.nanoTime() - startTime); log.warn("[{}] failed to get schema : {}", topicName, t.getMessage(), t); schemaFuture.completeExceptionally(t); } else { + histoGetSchema.recordSuccess(System.nanoTime() - startTime); schemaFuture.complete(r); } client.getCnxPool().releaseConnection(clientCnx); @@ -326,6 +370,8 @@ private void getTopicsUnderNamespace(InetSocketAddress socketAddress, Mode mode, String topicsPattern, String topicsHash) { + long startTime = System.nanoTime(); + client.getCnxPool().getConnection(socketAddress).thenAccept(clientCnx -> { long requestId = client.newRequestId(); ByteBuf request = Commands.newGetTopicsOfNamespaceRequest( @@ -333,8 +379,10 @@ private void getTopicsUnderNamespace(InetSocketAddress socketAddress, clientCnx.newGetTopicsOfNamespace(request, requestId).whenComplete((r, t) -> { if (t != null) { + histoListTopics.recordFailure(System.nanoTime() - startTime); getTopicsResultFuture.completeExceptionally(t); } else { + histoListTopics.recordSuccess(System.nanoTime() - startTime); if (log.isDebugEnabled()) { log.debug("[namespace: {}] Success get topics list in request: {}", namespace, requestId); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java index 48b3b2a6bda62..5029562ef1d0c 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java @@ -19,6 +19,7 @@ package org.apache.pulsar.client.impl; import static com.google.common.base.Preconditions.checkArgument; +import io.opentelemetry.api.OpenTelemetry; import java.net.InetSocketAddress; import java.time.Clock; import java.util.List; @@ -29,6 +30,7 @@ import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.ClientBuilder; +import org.apache.pulsar.client.api.MetricsCardinality; import org.apache.pulsar.client.api.ProxyProtocol; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; @@ -121,6 +123,18 @@ public ClientBuilder authentication(Authentication authentication) { return this; } + @Override + public ClientBuilder openTelemetry(OpenTelemetry openTelemetry) { + conf.setOpenTelemetry(openTelemetry); + return this; + } + + @Override + public ClientBuilder openTelemetryMetricsCardinality(MetricsCardinality metricsCardinality) { + conf.setOpenTelemetryMetricsCardinality(metricsCardinality); + return this; + } + @Override public ClientBuilder authentication(String authPluginClassName, String authParamsString) throws UnsupportedAuthenticationException { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java index b3444ae393ef0..30fe13497feba 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java @@ -32,6 +32,7 @@ import io.netty.handler.codec.LengthFieldBasedFrameDecoder; import io.netty.handler.ssl.SslHandshakeCompletionEvent; import io.netty.util.concurrent.Promise; +import io.opentelemetry.api.common.Attributes; import java.net.InetSocketAddress; import java.net.SocketAddress; import java.net.URI; @@ -60,6 +61,9 @@ import org.apache.pulsar.client.api.PulsarClientException.TimeoutException; import org.apache.pulsar.client.impl.BinaryProtoLookupService.LookupDataResult; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.metrics.Counter; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; +import org.apache.pulsar.client.impl.metrics.Unit; import org.apache.pulsar.client.impl.schema.SchemaInfoUtil; import org.apache.pulsar.client.impl.transaction.TransactionBufferHandler; import org.apache.pulsar.client.util.TimedCompletableFuture; @@ -202,6 +206,9 @@ protected enum State { None, SentConnectFrame, Ready, Failed, Connecting } + private final Counter connectionsOpenedCounter; + private final Counter connectionsClosedCounter; + private static class RequestTime { private final long creationTimeNanos; final long requestId; @@ -239,10 +246,15 @@ String getDescription() { public ClientCnx(ClientConfigurationData conf, EventLoopGroup eventLoopGroup) { - this(conf, eventLoopGroup, Commands.getCurrentProtocolVersion()); + this(new InstrumentProvider(conf), conf, eventLoopGroup, Commands.getCurrentProtocolVersion()); + } + + public ClientCnx(InstrumentProvider instrumentProvider, + ClientConfigurationData conf, EventLoopGroup eventLoopGroup) { + this(instrumentProvider, conf, eventLoopGroup, Commands.getCurrentProtocolVersion()); } - public ClientCnx(ClientConfigurationData conf, EventLoopGroup eventLoopGroup, int protocolVersion) { + public ClientCnx(InstrumentProvider instrumentProvider, ClientConfigurationData conf, EventLoopGroup eventLoopGroup, int protocolVersion) { super(conf.getKeepAliveIntervalSeconds(), TimeUnit.SECONDS); checkArgument(conf.getMaxLookupRequest() > conf.getConcurrentLookupRequest()); this.pendingLookupRequestSemaphore = new Semaphore(conf.getConcurrentLookupRequest(), false); @@ -258,11 +270,17 @@ public ClientCnx(ClientConfigurationData conf, EventLoopGroup eventLoopGroup, in this.idleState = new ClientCnxIdleState(this); this.clientVersion = "Pulsar-Java-v" + PulsarVersion.getVersion() + (conf.getDescription() == null ? "" : ("-" + conf.getDescription())); + this.connectionsOpenedCounter = instrumentProvider.newCounter("pulsar.client.connections.opened", Unit.Connections, + "Counter of connections opened", Attributes.empty()); + this.connectionsClosedCounter = instrumentProvider.newCounter("pulsar.client.connections.closed", Unit.Connections, + "Counter of connections closed", Attributes.empty()); + } @Override public void channelActive(ChannelHandlerContext ctx) throws Exception { super.channelActive(ctx); + connectionsOpenedCounter.increment(); this.localAddress = ctx.channel().localAddress(); this.remoteAddress = ctx.channel().remoteAddress(); @@ -305,6 +323,7 @@ protected ByteBuf newConnectCommand() throws Exception { @Override public void channelInactive(ChannelHandlerContext ctx) throws Exception { super.channelInactive(ctx); + connectionsClosedCounter.increment(); lastDisconnectedTimestamp = System.currentTimeMillis(); log.info("{} Disconnected", ctx.channel()); if (!connectionFuture.isDone()) { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java index 9750911b37c21..b5ae64cb693b4 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java @@ -32,6 +32,7 @@ import io.netty.resolver.dns.SequentialDnsServerAddressStreamProvider; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.ScheduledFuture; +import io.opentelemetry.api.common.Attributes; import java.net.InetSocketAddress; import java.net.URI; import java.net.URISyntaxException; @@ -53,6 +54,9 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.PulsarClientException.InvalidServiceURL; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.metrics.Counter; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; +import org.apache.pulsar.client.impl.metrics.Unit; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.netty.DnsResolverUtil; @@ -87,16 +91,22 @@ public class ConnectionPool implements AutoCloseable { /** Async release useless connections task. **/ private ScheduledFuture asyncReleaseUselessConnectionsTask; - public ConnectionPool(ClientConfigurationData conf, EventLoopGroup eventLoopGroup) throws PulsarClientException { - this(conf, eventLoopGroup, () -> new ClientCnx(conf, eventLoopGroup)); + private final Counter connectionsTcpFailureCounter; + private final Counter connectionsHandshakeFailureCounter; + + public ConnectionPool(InstrumentProvider instrumentProvider, + ClientConfigurationData conf, EventLoopGroup eventLoopGroup) throws PulsarClientException { + this(instrumentProvider, conf, eventLoopGroup, () -> new ClientCnx(instrumentProvider, conf, eventLoopGroup)); } - public ConnectionPool(ClientConfigurationData conf, EventLoopGroup eventLoopGroup, + public ConnectionPool(InstrumentProvider instrumentProvider, + ClientConfigurationData conf, EventLoopGroup eventLoopGroup, Supplier clientCnxSupplier) throws PulsarClientException { - this(conf, eventLoopGroup, clientCnxSupplier, Optional.empty()); + this(instrumentProvider, conf, eventLoopGroup, clientCnxSupplier, Optional.empty()); } - public ConnectionPool(ClientConfigurationData conf, EventLoopGroup eventLoopGroup, + public ConnectionPool(InstrumentProvider instrumentProvider, + ClientConfigurationData conf, EventLoopGroup eventLoopGroup, Supplier clientCnxSupplier, Optional> addressResolver) throws PulsarClientException { @@ -146,6 +156,12 @@ public ConnectionPool(ClientConfigurationData conf, EventLoopGroup eventLoopGrou } }, idleDetectionIntervalSeconds, idleDetectionIntervalSeconds, TimeUnit.SECONDS); } + + connectionsTcpFailureCounter = instrumentProvider.newCounter("pulsar.client.connections.failed", Unit.None, + "Counter of failed connections", Attributes.builder().put("type", "tcp-failed").build()); + connectionsHandshakeFailureCounter = instrumentProvider.newCounter("pulsar.client.connections.failed", + Unit.None, "Counter of failed connections", + Attributes.builder().put("type", "handshake").build()); } private static AddressResolver createAddressResolver(ClientConfigurationData conf, @@ -294,6 +310,7 @@ private CompletableFuture createConnection(InetSocketAddress logicalA } cnxFuture.complete(cnx); }).exceptionally(exception -> { + connectionsHandshakeFailureCounter.increment(); log.warn("[{}] Connection handshake failed: {}", cnx.channel(), exception.getMessage()); cnxFuture.completeExceptionally(exception); // this cleanupConnection may happen before that the @@ -305,6 +322,7 @@ private CompletableFuture createConnection(InetSocketAddress logicalA return null; }); }).exceptionally(exception -> { + connectionsTcpFailureCounter.increment(); eventLoopGroup.execute(() -> { log.warn("Failed to open connection to {} : {}", physicalAddress, exception.getMessage()); cleanupConnection(logicalAddress, connectionKey, cnxFuture); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java index d45c125c590de..4474df778c892 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java @@ -1228,7 +1228,10 @@ public int getTotalIncomingMessages() { protected void clearIncomingMessages() { // release messages if they are pooled messages - incomingMessages.forEach(Message::release); + incomingMessages.forEach(msg -> { + + msg.release(); + }); incomingMessages.clear(); resetIncomingMessageSize(); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 5619837757363..13c5a37ae1113 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -34,6 +34,7 @@ import io.netty.util.ReferenceCountUtil; import io.netty.util.Timeout; import io.netty.util.concurrent.FastThreadLocal; +import io.opentelemetry.api.common.Attributes; import java.io.IOException; import java.net.URI; import java.nio.ByteBuffer; @@ -91,6 +92,10 @@ import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; import org.apache.pulsar.client.impl.crypto.MessageCryptoBc; +import org.apache.pulsar.client.impl.metrics.Counter; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; +import org.apache.pulsar.client.impl.metrics.Unit; +import org.apache.pulsar.client.impl.metrics.UpDownCounter; import org.apache.pulsar.client.impl.schema.AutoConsumeSchema; import org.apache.pulsar.client.impl.transaction.TransactionImpl; import org.apache.pulsar.client.util.ExecutorProvider; @@ -214,6 +219,17 @@ public class ConsumerImpl extends ConsumerBase implements ConnectionHandle private final boolean createTopicIfDoesNotExist; private final boolean poolMessages; + private final Counter messagesReceivedCounter; + private final Counter bytesReceivedCounter; + private final UpDownCounter messagesPrefetchedGauge; + private final UpDownCounter bytesPrefetchedGauge; + private final Counter consumersOpenedCounter; + private final Counter consumersClosedCounter; + private final Counter consumerAcksCounter; + private final Counter consumerNacksCounter; + + private final Counter consumerDlqMessagesCounter; + private final AtomicReference clientCnxUsedForConsumerRegistration = new AtomicReference<>(); private final List previousExceptions = new CopyOnWriteArrayList(); static ConsumerImpl newConsumerImpl(PulsarClientImpl client, @@ -386,7 +402,30 @@ protected ConsumerImpl(PulsarClientImpl client, String topic, ConsumerConfigurat topicNameWithoutPartition = topicName.getPartitionedTopicName(); + InstrumentProvider ip = client.instrumentProvider(); + Attributes attrs = ip.getAttributes(topic); + consumersOpenedCounter = ip.newCounter("pulsar.client.session.opened", Unit.Sessions, + "Counter of sessions opened", attrs.toBuilder().put("type", "consumer").build()); + consumersClosedCounter = ip.newCounter("pulsar.client.session.closed", Unit.Sessions, + "Counter of sessions closed", attrs.toBuilder().put("type", "consumer").build()); + messagesReceivedCounter = ip.newCounter("pulsar.client.received", Unit.Messages, + "Number of messages received", attrs); + bytesReceivedCounter = ip.newCounter("pulsar.client.received", Unit.Bytes, + "Bytes received", attrs); + messagesPrefetchedGauge = ip.newUpDownCounter("pulsar.client.consumer.preteched.messages", Unit.Messages, + "Number of messages currently sitting in the consumer pre-fetch queue", attrs); + bytesPrefetchedGauge = ip.newUpDownCounter("pulsar.client.consumer.preteched", Unit.Bytes, + "Total number of bytes currently sitting in the consumer pre-fetch queue", attrs); + + consumerAcksCounter = ip.newCounter("pulsar.client.consumer.ack", Unit.Messages, + "Number of ack operations", attrs); + consumerNacksCounter = ip.newCounter("pulsar.client.consumer.nack", Unit.Messages, + "Number of negative ack operations", attrs); + consumerDlqMessagesCounter = ip.newCounter("pulsar.client.consumer.dlq", Unit.Messages, + "Number of messages sent to DLQ", attrs); grabCnx(); + + consumersOpenedCounter.increment(); } public ConnectionHandler getConnectionHandler() { @@ -549,6 +588,8 @@ protected CompletableFuture> internalBatchReceiveAsync() { protected CompletableFuture doAcknowledge(MessageId messageId, AckType ackType, Map properties, TransactionImpl txn) { + consumerAcksCounter.increment(); + if (getState() != State.Ready && getState() != State.Connecting) { stats.incrementNumAcksFailed(); PulsarClientException exception = new PulsarClientException("Consumer not ready. State: " + getState()); @@ -570,6 +611,8 @@ protected CompletableFuture doAcknowledge(MessageId messageId, AckType ack @Override protected CompletableFuture doAcknowledge(List messageIdList, AckType ackType, Map properties, TransactionImpl txn) { + consumerAcksCounter.increment(); + if (getState() != State.Ready && getState() != State.Connecting) { stats.incrementNumAcksFailed(); PulsarClientException exception = new PulsarClientException("Consumer not ready. State: " + getState()); @@ -665,6 +708,8 @@ protected CompletableFuture doReconsumeLater(Message message, AckType a .value(retryMessage.getData()) .properties(propertiesMap); typedMessageBuilderNew.sendAsync().thenAccept(msgId -> { + consumerDlqMessagesCounter.increment(); + doAcknowledge(finalMessageId, ackType, Collections.emptyMap(), null).thenAccept(v -> { result.complete(null); }).exceptionally(ex -> { @@ -757,6 +802,7 @@ private MessageImpl getMessageImpl(Message message) { @Override public void negativeAcknowledge(MessageId messageId) { + consumerNacksCounter.increment(); negativeAcksTracker.add(messageId); // Ensure the message is not redelivered for ack-timeout, since we did receive an "ack" @@ -765,6 +811,7 @@ public void negativeAcknowledge(MessageId messageId) { @Override public void negativeAcknowledge(Message message) { + consumerNacksCounter.increment(); negativeAcksTracker.add(message); // Ensure the message is not redelivered for ack-timeout, since we did receive an "ack" @@ -1036,6 +1083,8 @@ public CompletableFuture closeAsync() { return closeFuture; } + consumersClosedCounter.increment(); + if (!isConnected()) { log.info("[{}] [{}] Closed Consumer (not connected)", topic, subscription); setState(State.Closed); @@ -1228,6 +1277,9 @@ protected MessageImpl newMessage(final MessageIdImpl messageId, } private void executeNotifyCallback(final MessageImpl message) { + messagesPrefetchedGauge.increment(); + bytesPrefetchedGauge.add(message.size()); + // Enqueue the message so that it can be retrieved when application calls receive() // if the conf.getReceiverQueueSize() is 0 then discard message if no one is waiting for it. // if asyncReceive is waiting then notify callback without adding to incomingMessages queue @@ -1720,6 +1772,12 @@ protected synchronized void messageProcessed(Message msg) { ClientCnx msgCnx = ((MessageImpl) msg).getCnx(); lastDequeuedMessageId = msg.getMessageId(); + messagesPrefetchedGauge.decrement(); + messagesReceivedCounter.increment(); + + bytesPrefetchedGauge.subtract(msg.size()); + bytesReceivedCounter.add(msg.size()); + if (msgCnx != currentCnx) { // The processed message did belong to the old queue that was cleared after reconnection. } else { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java index 02d0d10626fa6..73f2127570d84 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java @@ -19,6 +19,8 @@ package org.apache.pulsar.client.impl; import io.netty.channel.EventLoopGroup; +import io.opentelemetry.api.common.AttributeKey; +import io.opentelemetry.api.common.Attributes; import java.net.InetSocketAddress; import java.net.URI; import java.nio.ByteBuffer; @@ -34,6 +36,8 @@ import org.apache.pulsar.client.api.PulsarClientException.NotFoundException; import org.apache.pulsar.client.api.SchemaSerializationException; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; +import org.apache.pulsar.client.impl.metrics.LatencyHistogram; import org.apache.pulsar.client.impl.schema.SchemaInfoUtil; import org.apache.pulsar.client.impl.schema.SchemaUtils; import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace.Mode; @@ -60,11 +64,31 @@ public class HttpLookupService implements LookupService { private static final String BasePathV1 = "lookup/v2/destination/"; private static final String BasePathV2 = "lookup/v2/topic/"; - public HttpLookupService(ClientConfigurationData conf, EventLoopGroup eventLoopGroup) + private final LatencyHistogram histoGetBroker; + private final LatencyHistogram histoGetTopicMetadata; + private final LatencyHistogram histoGetSchema; + private final LatencyHistogram histoListTopics; + + public HttpLookupService(InstrumentProvider instrumentProvider, ClientConfigurationData conf, EventLoopGroup eventLoopGroup) throws PulsarClientException { this.httpClient = new HttpClient(conf, eventLoopGroup); this.useTls = conf.isUseTls(); this.listenerName = conf.getListenerName(); + + Attributes attrs = Attributes.of(AttributeKey.stringKey("transport-type"), "binary"); + + histoGetBroker = instrumentProvider.newLatencyHistogram("pulsar.client.lookup", + "Lookup operations", + attrs.toBuilder().put("type", "topic").build()); + histoGetTopicMetadata = instrumentProvider.newLatencyHistogram("pulsar.client.lookup", + "Lookup operations", + attrs.toBuilder().put("type", "metadata").build()); + histoGetSchema = instrumentProvider.newLatencyHistogram("pulsar.client.lookup", + "Lookup operations", + attrs.toBuilder().put("type", "schema").build()); + histoListTopics = instrumentProvider.newLatencyHistogram("pulsar.client.lookup", + "Lookup operations", + attrs.toBuilder().put("type", "list-topics").build()); } @Override @@ -84,8 +108,18 @@ public CompletableFuture getBroker(TopicName topicName) { String basePath = topicName.isV2() ? BasePathV2 : BasePathV1; String path = basePath + topicName.getLookupName(); path = StringUtils.isBlank(listenerName) ? path : path + "?listenerName=" + Codec.encode(listenerName); - return httpClient.get(path, LookupData.class) - .thenCompose(lookupData -> { + + long startTime = System.nanoTime(); + CompletableFuture httpFuture = httpClient.get(path, LookupData.class); + + httpFuture.thenRun(() -> { + histoGetBroker.recordSuccess(System.nanoTime() - startTime); + }).exceptionally(x -> { + histoGetBroker.recordFailure(System.nanoTime() - startTime); + return null; + }); + + return httpFuture.thenCompose(lookupData -> { // Convert LookupData into as SocketAddress, handling exceptions URI uri = null; try { @@ -112,9 +146,21 @@ public CompletableFuture getBroker(TopicName topicName) { @Override public CompletableFuture getPartitionedTopicMetadata(TopicName topicName) { + long startTime = System.nanoTime(); + String format = topicName.isV2() ? "admin/v2/%s/partitions" : "admin/%s/partitions"; - return httpClient.get(String.format(format, topicName.getLookupName()) + "?checkAllowAutoCreation=true", + CompletableFuture httpFuture = httpClient.get( + String.format(format, topicName.getLookupName()) + "?checkAllowAutoCreation=true", PartitionedTopicMetadata.class); + + httpFuture.thenRun(() -> { + histoGetTopicMetadata.recordSuccess(System.nanoTime() - startTime); + }).exceptionally(x -> { + histoGetTopicMetadata.recordFailure(System.nanoTime() - startTime); + return null; + }); + + return httpFuture; } @Override @@ -130,6 +176,8 @@ public InetSocketAddress resolveHost() { @Override public CompletableFuture getTopicsUnderNamespace(NamespaceName namespace, Mode mode, String topicsPattern, String topicsHash) { + long startTime = System.nanoTime(); + CompletableFuture future = new CompletableFuture<>(); String format = namespace.isV2() @@ -152,6 +200,14 @@ public CompletableFuture getTopicsUnderNamespace(NamespaceName future.completeExceptionally(cause); return null; }); + + future.thenRun(() -> { + histoListTopics.recordSuccess(System.nanoTime() - startTime); + }).exceptionally(x -> { + histoListTopics.recordFailure(System.nanoTime() - startTime); + return null; + }); + return future; } @@ -162,6 +218,7 @@ public CompletableFuture> getSchema(TopicName topicName) { @Override public CompletableFuture> getSchema(TopicName topicName, byte[] version) { + long startTime = System.nanoTime(); CompletableFuture> future = new CompletableFuture<>(); String schemaName = topicName.getSchemaName(); @@ -201,6 +258,13 @@ public CompletableFuture> getSchema(TopicName topicName, by } return null; }); + + future.thenRun(() -> { + histoGetSchema.recordSuccess(System.nanoTime() - startTime); + }).exceptionally(x -> { + histoGetSchema.recordFailure(System.nanoTime() - startTime); + return null; + }); return future; } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java index 4908d10f330b3..4405c2f5c9381 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java @@ -40,6 +40,7 @@ import io.netty.util.Timeout; import io.netty.util.TimerTask; import io.netty.util.concurrent.ScheduledFuture; +import io.opentelemetry.api.common.Attributes; import java.io.IOException; import java.net.URI; import java.nio.ByteBuffer; @@ -76,6 +77,11 @@ import org.apache.pulsar.client.api.transaction.Transaction; import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; import org.apache.pulsar.client.impl.crypto.MessageCryptoBc; +import org.apache.pulsar.client.impl.metrics.Counter; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; +import org.apache.pulsar.client.impl.metrics.LatencyHistogram; +import org.apache.pulsar.client.impl.metrics.Unit; +import org.apache.pulsar.client.impl.metrics.UpDownCounter; import org.apache.pulsar.client.impl.schema.JSONSchema; import org.apache.pulsar.client.impl.transaction.TransactionImpl; import org.apache.pulsar.client.util.MathUtils; @@ -171,6 +177,15 @@ public class ProducerImpl extends ProducerBase implements TimerTask, Conne private boolean errorState; + private final LatencyHistogram latencyHistogram; + private final LatencyHistogram rpcLatencyHistogram; + private final Counter publishedBytesCounter; + private final UpDownCounter pendingMessagesCounter; + private final UpDownCounter pendingBytesCounter; + + private final Counter producersOpenedCounter; + private final Counter producersClosedCounter; + public ProducerImpl(PulsarClientImpl client, String topic, ProducerConfigurationData conf, CompletableFuture> producerCreatedFuture, int partitionIndex, Schema schema, ProducerInterceptors interceptors, Optional overrideProducerName) { @@ -268,6 +283,23 @@ public ProducerImpl(PulsarClientImpl client, String topic, ProducerConfiguration metadata = Collections.unmodifiableMap(new HashMap<>(conf.getProperties())); } + InstrumentProvider ip = client.instrumentProvider(); + Attributes attrs = ip.getAttributes(topic); + latencyHistogram = ip.newLatencyHistogram("pulsar.client.producer.latency", + "Publish latency experienced by the application, includes client batching time", attrs); + rpcLatencyHistogram = ip.newLatencyHistogram("pulsar.client.producer.rpc.latency", + "Publish RPC latency experienced internally by the client when sending data to receiving an ack", attrs); + publishedBytesCounter = ip.newCounter("pulsar.client.producer.published", + Unit.Bytes, "Bytes published", attrs); + pendingMessagesCounter = ip.newUpDownCounter("pulsar.client.producer.pending.messages.count", Unit.Messages, + "Pending messages for this producer", attrs); + pendingBytesCounter = ip.newUpDownCounter("pulsar.client.producer.pending.count", Unit.Bytes, + "Pending bytes for this producer", attrs); + producersOpenedCounter = ip.newCounter("pulsar.client.session.opened", Unit.Sessions, + "Counter of sessions opened", attrs.toBuilder().put("type", "producer").build()); + producersClosedCounter = ip.newCounter("pulsar.client.session.closed", Unit.Sessions, + "Counter of sessions closed", attrs.toBuilder().put("type", "producer").build()); + this.connectionHandler = new ConnectionHandler(this, new BackoffBuilder() .setInitialTime(client.getConfiguration().getInitialBackoffIntervalNanos(), TimeUnit.NANOSECONDS) @@ -277,6 +309,7 @@ public ProducerImpl(PulsarClientImpl client, String topic, ProducerConfiguration this); grabCnx(); + producersOpenedCounter.increment(); } protected void semaphoreRelease(final int releaseCountRequest) { @@ -334,6 +367,11 @@ CompletableFuture internalSendAsync(Message message) { if (interceptors != null) { interceptorMessage.getProperties(); } + + int msgSize = interceptorMessage.getDataBuffer().readableBytes(); + pendingMessagesCounter.increment(); + pendingBytesCounter.add(msgSize); + sendAsync(interceptorMessage, new SendCallback() { SendCallback nextCallback = null; MessageImpl nextMsg = null; @@ -356,15 +394,22 @@ public MessageImpl getNextMessage() { @Override public void sendComplete(Exception e) { + long latencyNanos = System.nanoTime() - createdAt; + pendingMessagesCounter.decrement(); + pendingBytesCounter.subtract(msgSize); + try { if (e != null) { + latencyHistogram.recordFailure(latencyNanos); stats.incrementSendFailed(); onSendAcknowledgement(interceptorMessage, null, e); future.completeExceptionally(e); } else { + latencyHistogram.recordSuccess(latencyNanos); + publishedBytesCounter.add(msgSize); onSendAcknowledgement(interceptorMessage, interceptorMessage.getMessageId(), null); future.complete(interceptorMessage.getMessageId()); - stats.incrementNumAcksReceived(System.nanoTime() - createdAt); + stats.incrementNumAcksReceived(latencyNanos); } } finally { interceptorMessage.getDataBuffer().release(); @@ -694,9 +739,9 @@ private void serializeAndSendMessage(MessageImpl msg, if (msg.getSchemaState() == MessageImpl.SchemaState.Ready) { ByteBufPair cmd = sendMessage(producerId, sequenceId, numMessages, messageId, msgMetadata, encryptedPayload); - op = OpSendMsg.create(msg, cmd, sequenceId, callback); + op = OpSendMsg.create(this, msg, cmd, sequenceId, callback); } else { - op = OpSendMsg.create(msg, null, sequenceId, callback); + op = OpSendMsg.create(this, msg, null, sequenceId, callback); final MessageMetadata finalMsgMetadata = msgMetadata; op.rePopulate = () -> { if (msgMetadata.hasChunkId()) { @@ -1067,6 +1112,7 @@ public CompletableFuture closeAsync() { return CompletableFuture.completedFuture(null); } + producersClosedCounter.increment(); closeProducerTasks(); ClientCnx cnx = cnx(); @@ -1399,6 +1445,7 @@ public ReferenceCounted touch(Object hint) { } protected static final class OpSendMsg { + ProducerImpl producer; MessageImpl msg; List> msgs; ByteBufPair cmd; @@ -1418,6 +1465,7 @@ protected static final class OpSendMsg { int chunkId = -1; void initialize() { + producer = null; msg = null; msgs = null; cmd = null; @@ -1437,9 +1485,10 @@ void initialize() { chunkedMessageCtx = null; } - static OpSendMsg create(MessageImpl msg, ByteBufPair cmd, long sequenceId, SendCallback callback) { + static OpSendMsg create(ProducerImpl producer, MessageImpl msg, ByteBufPair cmd, long sequenceId, SendCallback callback) { OpSendMsg op = RECYCLER.get(); op.initialize(); + op.producer = producer; op.msg = msg; op.cmd = cmd; op.callback = callback; @@ -1449,10 +1498,11 @@ static OpSendMsg create(MessageImpl msg, ByteBufPair cmd, long sequenceId, Se return op; } - static OpSendMsg create(List> msgs, ByteBufPair cmd, long sequenceId, SendCallback callback, + static OpSendMsg create(ProducerImpl producer, List> msgs, ByteBufPair cmd, long sequenceId, SendCallback callback, int batchAllocatedSize) { OpSendMsg op = RECYCLER.get(); op.initialize(); + op.producer = producer; op.msgs = msgs; op.cmd = cmd; op.callback = callback; @@ -1466,10 +1516,11 @@ static OpSendMsg create(List> msgs, ByteBufPair cmd, long sequenc return op; } - static OpSendMsg create(List> msgs, ByteBufPair cmd, long lowestSequenceId, + static OpSendMsg create(ProducerImpl producer, List> msgs, ByteBufPair cmd, long lowestSequenceId, long highestSequenceId, SendCallback callback, int batchAllocatedSize) { OpSendMsg op = RECYCLER.get(); op.initialize(); + op.producer = producer; op.msgs = msgs; op.cmd = cmd; op.callback = callback; @@ -1494,30 +1545,38 @@ void updateSentTimestamp() { void sendComplete(final Exception e) { SendCallback callback = this.callback; + + long now = System.nanoTime(); if (null != callback) { Exception finalEx = e; if (finalEx instanceof TimeoutException) { TimeoutException te = (TimeoutException) e; long sequenceId = te.getSequenceId(); - long ns = System.nanoTime(); + //firstSentAt and lastSentAt maybe -1, it means that the message didn't flush to channel. String errMsg = String.format( "%s : createdAt %s seconds ago, firstSentAt %s seconds ago, lastSentAt %s seconds ago, " + "retryCount %s", te.getMessage(), - RelativeTimeUtil.nsToSeconds(ns - this.createdAt), + RelativeTimeUtil.nsToSeconds(now - this.createdAt), RelativeTimeUtil.nsToSeconds(this.firstSentAt <= 0 ? this.firstSentAt - : ns - this.firstSentAt), + : now - this.firstSentAt), RelativeTimeUtil.nsToSeconds(this.lastSentAt <= 0 ? this.lastSentAt - : ns - this.lastSentAt), + : now - this.lastSentAt), retryCount ); finalEx = new TimeoutException(errMsg, sequenceId); } + if (e == null) { + producer.rpcLatencyHistogram.recordSuccess(now - this.lastSentAt); + } else { + producer.rpcLatencyHistogram.recordFailure(now - this.lastSentAt); + } + callback.sendComplete(finalEx); } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index 179996f4ea9f1..0b75098f9baec 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -26,6 +26,7 @@ import io.netty.channel.EventLoopGroup; import io.netty.util.HashedWheelTimer; import io.netty.util.Timer; +import io.opentelemetry.api.metrics.Meter; import java.io.IOException; import java.net.InetSocketAddress; import java.time.Clock; @@ -70,6 +71,8 @@ import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; import org.apache.pulsar.client.impl.conf.ReaderConfigurationData; +import org.apache.pulsar.client.impl.metrics.Counter; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.client.impl.schema.AutoConsumeSchema; import org.apache.pulsar.client.impl.schema.AutoProduceBytesSchema; import org.apache.pulsar.client.impl.schema.generic.GenericAvroSchema; @@ -149,6 +152,8 @@ public SchemaInfoProvider load(String topicName) { private final Clock clientClock; + private final InstrumentProvider instrumentProvider; + @Getter private TransactionCoordinatorClientImpl tcClient; @@ -176,6 +181,9 @@ private PulsarClientImpl(ClientConfigurationData conf, EventLoopGroup eventLoopG Timer timer, ExecutorProvider externalExecutorProvider, ExecutorProvider internalExecutorProvider, ScheduledExecutorProvider scheduledExecutorProvider) throws PulsarClientException { + + this.instrumentProvider = new InstrumentProvider(conf); + EventLoopGroup eventLoopGroupReference = null; ConnectionPool connectionPoolReference = null; try { @@ -196,7 +204,8 @@ private PulsarClientImpl(ClientConfigurationData conf, EventLoopGroup eventLoopG clientClock = conf.getClock(); conf.getAuthentication().start(); connectionPoolReference = - connectionPool != null ? connectionPool : new ConnectionPool(conf, this.eventLoopGroup); + connectionPool != null ? connectionPool : + new ConnectionPool(instrumentProvider, conf, this.eventLoopGroup); this.cnxPool = connectionPoolReference; this.externalExecutorProvider = externalExecutorProvider != null ? externalExecutorProvider : new ExecutorProvider(conf.getNumListenerThreads(), "pulsar-external-listener"); @@ -205,7 +214,7 @@ private PulsarClientImpl(ClientConfigurationData conf, EventLoopGroup eventLoopG this.scheduledExecutorProvider = scheduledExecutorProvider != null ? scheduledExecutorProvider : new ScheduledExecutorProvider(conf.getNumIoThreads(), "pulsar-client-scheduled"); if (conf.getServiceUrl().startsWith("http")) { - lookup = new HttpLookupService(conf, this.eventLoopGroup); + lookup = new HttpLookupService(instrumentProvider, conf, this.eventLoopGroup); } else { lookup = new BinaryProtoLookupService(this, conf.getServiceUrl(), conf.getListenerName(), conf.isUseTls(), this.scheduledExecutorProvider.getExecutor()); @@ -1053,7 +1062,7 @@ public void reloadLookUp() throws PulsarClientException { public LookupService createLookup(String url) throws PulsarClientException { if (url.startsWith("http")) { - return new HttpLookupService(conf, eventLoopGroup); + return new HttpLookupService(instrumentProvider, conf, eventLoopGroup); } else { return new BinaryProtoLookupService(this, url, conf.getListenerName(), conf.isUseTls(), externalExecutorProvider.getExecutor()); @@ -1231,6 +1240,11 @@ public ScheduledExecutorProvider getScheduledExecutorProvider() { return scheduledExecutorProvider; } + InstrumentProvider instrumentProvider() { + return instrumentProvider; + } + + // // Transaction related API // diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java index 20ec9c3d99af4..f5e841db68cd5 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java @@ -22,6 +22,7 @@ import io.netty.util.Timeout; import io.netty.util.TimerTask; import io.netty.util.concurrent.FastThreadLocal; +import io.opentelemetry.api.common.Attributes; import java.io.Closeable; import java.util.ArrayDeque; import java.util.Collections; @@ -35,6 +36,9 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; +import org.apache.pulsar.client.impl.metrics.Counter; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; +import org.apache.pulsar.client.impl.metrics.Unit; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -52,6 +56,8 @@ public class UnAckedMessageTracker implements Closeable { protected final long ackTimeoutMillis; protected final long tickDurationInMs; + private final Counter consumerAckTimeoutsCounter; + private static class UnAckedMessageTrackerDisabled extends UnAckedMessageTracker { @Override public void clear() { @@ -89,13 +95,14 @@ public void close() { protected Timeout timeout; - public UnAckedMessageTracker() { + private UnAckedMessageTracker() { readLock = null; writeLock = null; timePartitions = null; messageIdPartitionMap = null; this.ackTimeoutMillis = 0; this.tickDurationInMs = 0; + this.consumerAckTimeoutsCounter = null; } protected static final FastThreadLocal> TL_MESSAGE_IDS_SET = @@ -114,6 +121,12 @@ public UnAckedMessageTracker(PulsarClientImpl client, ConsumerBase consumerBa ReentrantReadWriteLock readWriteLock = new ReentrantReadWriteLock(); this.readLock = readWriteLock.readLock(); this.writeLock = readWriteLock.writeLock(); + + InstrumentProvider ip = client.instrumentProvider(); + Attributes attrs = ip.getAttributes(consumerBase.getTopic()); + consumerAckTimeoutsCounter = ip.newCounter("pulsar.client.consumer.ack.timeout", Unit.Messages, + "Number of ack timeouts events", attrs); + if (conf.getAckTimeoutRedeliveryBackoff() == null) { this.messageIdPartitionMap = new HashMap<>(); this.timePartitions = new ArrayDeque<>(); @@ -136,6 +149,7 @@ public void run(Timeout t) throws Exception { try { HashSet headPartition = timePartitions.removeFirst(); if (!headPartition.isEmpty()) { + consumerAckTimeoutsCounter.add(headPartition.size()); log.info("[{}] {} messages will be re-delivered", consumerBase, headPartition.size()); headPartition.forEach(messageId -> { if (messageId instanceof ChunkMessageIdImpl) { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java index 1dc1c2a8689c6..18bbf34e1a69d 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java @@ -20,6 +20,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import io.opentelemetry.api.OpenTelemetry; import io.swagger.annotations.ApiModelProperty; import java.io.Serializable; import java.net.InetSocketAddress; @@ -37,6 +38,7 @@ import lombok.Data; import lombok.NoArgsConstructor; import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.MetricsCardinality; import org.apache.pulsar.client.api.ProxyProtocol; import org.apache.pulsar.client.api.ServiceUrlProvider; import org.apache.pulsar.client.impl.auth.AuthenticationDisabled; @@ -107,7 +109,7 @@ public class ClientConfigurationData implements Serializable, Cloneable { name = "statsIntervalSeconds", value = "Interval to print client stats (in seconds)." ) - private long statsIntervalSeconds = 60; + private long statsIntervalSeconds = 0; @ApiModelProperty( name = "numIoThreads", @@ -395,6 +397,11 @@ public class ClientConfigurationData implements Serializable, Cloneable { ) private String description; + + private transient OpenTelemetry openTelemetry; + + private MetricsCardinality openTelemetryMetricsCardinality = MetricsCardinality.Topic; + /** * Gets the authentication settings for the client. * diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Counter.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Counter.java new file mode 100644 index 0000000000000..f573d5557dfe1 --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Counter.java @@ -0,0 +1,28 @@ +package org.apache.pulsar.client.impl.metrics; + +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.metrics.LongCounter; +import io.opentelemetry.api.metrics.Meter; + +public class Counter { + + private final LongCounter counter; + private final Attributes attributes; + + Counter(Meter meter, String name, Unit unit, String description, Attributes attributes) { + counter = meter.counterBuilder(name) + .setDescription(description) + .setUnit(unit.toString()) + .build(); + this.attributes = attributes; + } + + public void increment() { + add(1); + } + + public void add(int delta) { + counter.add(delta, attributes); + } + +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/InstrumentProvider.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/InstrumentProvider.java new file mode 100644 index 0000000000000..eb923d270c94c --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/InstrumentProvider.java @@ -0,0 +1,59 @@ +package org.apache.pulsar.client.impl.metrics; + +import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.common.AttributesBuilder; +import io.opentelemetry.api.metrics.Meter; +import org.apache.pulsar.client.api.MetricsCardinality; +import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.common.naming.TopicName; + +public class InstrumentProvider { + + private final Meter meter; + private final MetricsCardinality metricsCardinality; + + public InstrumentProvider(ClientConfigurationData conf) { + this.meter = conf.getOpenTelemetry().getMeter("org.apache.pulsar.client"); + this.metricsCardinality = conf.getOpenTelemetryMetricsCardinality(); + } + + public Attributes getAttributes(String topic) { + if (metricsCardinality == MetricsCardinality.None) { + return Attributes.empty(); + } + + AttributesBuilder ab = Attributes.builder(); + TopicName tn = TopicName.get(topic); + + switch (metricsCardinality) { + case Partition: + if (tn.isPartitioned()) { + ab.put("partition", tn.getPartitionIndex()); + } + // fallthrough + case Topic: + ab.put("topic", tn.getPartitionedTopicName()); + // fallthrough + case Namespace: + ab.put("namespace", tn.getNamespace()); + // fallthrough + case Tenant: + ab.put("tenant", tn.getTenant()); + } + + return ab.build(); + } + + public Counter newCounter(String name, Unit unit, String description, Attributes attributes) { + return new Counter(meter, name, unit, description, attributes); + } + + public UpDownCounter newUpDownCounter(String name, Unit unit, String description, Attributes attributes) { + return new UpDownCounter(meter, name, unit, description, attributes); + } + + public LatencyHistogram newLatencyHistogram(String name, String description, Attributes attributes) { + return new LatencyHistogram(meter, name, description, attributes); + } +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java new file mode 100644 index 0000000000000..c23acac277898 --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java @@ -0,0 +1,46 @@ +package org.apache.pulsar.client.impl.metrics; + +import com.google.common.collect.Lists; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.metrics.DoubleHistogram; +import io.opentelemetry.api.metrics.Meter; +import java.util.List; +import java.util.concurrent.TimeUnit; + +public class LatencyHistogram { + + private static final List latencyHistogramBuckets = + Lists.newArrayList(.0005, .001, .0025, .005, .01, .025, .05, .1, .25, .5, 1.0, 2.5, 5.0, 10.0, 30.0, 60.0); + + private static final double NANOS = TimeUnit.SECONDS.toNanos(1); + + + private final Attributes successAttributes; + + private final Attributes failedAttributes; + private final DoubleHistogram histogram; + + LatencyHistogram(Meter meter, String name, String description, Attributes attributes) { + histogram = meter.histogramBuilder(name) + .setDescription(description) + .setUnit(Unit.Seconds.toString()) + .setExplicitBucketBoundariesAdvice(latencyHistogramBuckets) + .build(); + + successAttributes = attributes.toBuilder() + .put("success", true) + .build(); + failedAttributes = attributes.toBuilder() + .put("success", false) + .build(); + } + + + public void recordSuccess(long latencyNanos) { + histogram.record(latencyNanos / NANOS, successAttributes); + } + + public void recordFailure(long latencyNanos) { + histogram.record(latencyNanos / NANOS, failedAttributes); + } +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Unit.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Unit.java new file mode 100644 index 0000000000000..5204cc2f03eae --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Unit.java @@ -0,0 +1,59 @@ +/* + * 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.client.impl.metrics; + +public enum Unit { + Bytes, + + Messages, + + Seconds, + + Connections, + + Sessions, + + None, + + ; + + public String toString() { + switch (this) { + case Bytes: + return "By"; + + case Messages: + return "{message}"; + + case Seconds: + return "s"; + + case Connections: + return "{connection}"; + + case Sessions: + return "{session}"; + + case None: + default: + return "1"; + } + } +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/UpDownCounter.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/UpDownCounter.java new file mode 100644 index 0000000000000..6a5abd86e96ae --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/UpDownCounter.java @@ -0,0 +1,35 @@ +package org.apache.pulsar.client.impl.metrics; + +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.metrics.LongUpDownCounter; +import io.opentelemetry.api.metrics.Meter; + +public class UpDownCounter { + + private final LongUpDownCounter counter; + private final Attributes attributes; + + UpDownCounter(Meter meter, String name, Unit unit, String description, Attributes attributes) { + counter = meter.upDownCounterBuilder(name) + .setDescription(description) + .setUnit(unit.toString()) + .build(); + this.attributes = attributes; + } + + public void increment() { + add(1); + } + + public void decrement() { + add(-1); + } + + public void add(int delta) { + counter.add(delta, attributes); + } + + public void subtract(int diff) { + add(-diff); + } +} diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AcknowledgementsGroupingTrackerTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AcknowledgementsGroupingTrackerTest.java index 1d1a6f85bfd41..b57a3e3d7439e 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AcknowledgementsGroupingTrackerTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AcknowledgementsGroupingTrackerTest.java @@ -70,7 +70,7 @@ public void setup() throws NoSuchFieldException, IllegalAccessException { doReturn(client).when(consumer).getClient(); doReturn(cnx).when(consumer).getClientCnx(); doReturn(new ConsumerStatsRecorderImpl()).when(consumer).getStats(); - doReturn(new UnAckedMessageTracker().UNACKED_MESSAGE_TRACKER_DISABLED) + doReturn(UnAckedMessageTracker.UNACKED_MESSAGE_TRACKER_DISABLED) .when(consumer).getUnAckedMessageTracker(); ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); when(cnx.ctx()).thenReturn(ctx); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/conf/ClientConfigurationDataTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/conf/ClientConfigurationDataTest.java index 5856395566a67..27f521ef1ff73 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/conf/ClientConfigurationDataTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/conf/ClientConfigurationDataTest.java @@ -22,7 +22,14 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.SerializationFeature; +import io.opentelemetry.api.OpenTelemetry; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import lombok.Cleanup; import org.apache.pulsar.client.impl.auth.AuthenticationToken; +import org.testng.Assert; import org.testng.annotations.Test; /** @@ -36,10 +43,35 @@ public void testDoNotPrintSensitiveInfo() throws JsonProcessingException { clientConfigurationData.setTlsTrustStorePassword("xxxx"); clientConfigurationData.setSocks5ProxyPassword("yyyy"); clientConfigurationData.setAuthentication(new AuthenticationToken("zzzz")); + clientConfigurationData.setOpenTelemetry(OpenTelemetry.noop()); ObjectMapper objectMapper = new ObjectMapper(); objectMapper.configure(SerializationFeature.FAIL_ON_EMPTY_BEANS, false); String serializedConf = objectMapper.writeValueAsString(clientConfigurationData); assertThat(serializedConf).doesNotContain("xxxx", "yyyy", "zzzz"); } + @Test + public void testSerializable() throws Exception { + ClientConfigurationData conf = new ClientConfigurationData(); + conf.setConnectionsPerBroker(3); + conf.setTlsTrustStorePassword("xxxx"); + conf.setOpenTelemetry(OpenTelemetry.noop()); + + @Cleanup + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + @Cleanup + ObjectOutputStream oos = new ObjectOutputStream(bos); + oos.writeObject(conf); + byte[] serialized = bos.toByteArray(); + + // Deserialize + @Cleanup + ByteArrayInputStream bis = new ByteArrayInputStream(serialized); + @Cleanup + ObjectInputStream ois = new ObjectInputStream(bis); + Object object = ois.readObject(); + + Assert.assertEquals(object.getClass(), ClientConfigurationData.class); + Assert.assertEquals(object, conf); + } } diff --git a/pulsar-testclient/pom.xml b/pulsar-testclient/pom.xml index db2b84356e686..64e2136aa8761 100644 --- a/pulsar-testclient/pom.xml +++ b/pulsar-testclient/pom.xml @@ -112,6 +112,23 @@ jackson-databind + + io.opentelemetry + opentelemetry-exporter-prometheus + + + io.opentelemetry + opentelemetry-exporter-otlp + + + io.opentelemetry + opentelemetry-sdk + + + io.opentelemetry + opentelemetry-sdk-extension-autoconfigure + + org.awaitility awaitility diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerfClientUtils.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerfClientUtils.java index 3b44023ef503e..36396465d419a 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerfClientUtils.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerfClientUtils.java @@ -19,6 +19,7 @@ package org.apache.pulsar.testclient; import static org.apache.commons.lang3.StringUtils.isNotBlank; +import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdk; import java.lang.management.ManagementFactory; import java.util.Objects; import java.util.concurrent.TimeUnit; @@ -28,6 +29,7 @@ import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminBuilder; import org.apache.pulsar.client.api.ClientBuilder; +import org.apache.pulsar.client.api.MetricsCardinality; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.SizeUnit; @@ -76,7 +78,10 @@ public static ClientBuilder createClientBuilderFromArguments(PerformanceBaseArgu .listenerThreads(arguments.listenerThreads) .tlsTrustCertsFilePath(arguments.tlsTrustCertsFilePath) .maxLookupRequests(arguments.maxLookupRequest) - .proxyServiceUrl(arguments.proxyServiceURL, arguments.proxyProtocol); + .proxyServiceUrl(arguments.proxyServiceURL, arguments.proxyProtocol) + .openTelemetry(AutoConfiguredOpenTelemetrySdk.builder() + .build().getOpenTelemetrySdk()) + .openTelemetryMetricsCardinality(MetricsCardinality.Topic); if (isNotBlank(arguments.authPluginClassName)) { clientBuilder.authentication(arguments.authPluginClassName, arguments.authParams); From 8e184be917670cdfe8e48802f71c6ed3dec3afb0 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Mon, 4 Mar 2024 13:51:16 -0800 Subject: [PATCH 02/22] Addressed comments --- .../client/impl/BinaryProtoLookupService.java | 22 +++++++++---------- .../apache/pulsar/client/impl/ClientCnx.java | 4 ++-- .../pulsar/client/impl/ConnectionPool.java | 9 ++++---- .../pulsar/client/impl/ConsumerBase.java | 5 +---- .../pulsar/client/impl/ConsumerImpl.java | 20 ++++++++--------- .../pulsar/client/impl/HttpLookupService.java | 18 +++++++-------- .../pulsar/client/impl/ProducerImpl.java | 10 ++++----- .../client/impl/UnAckedMessageTracker.java | 2 +- .../impl/conf/ClientConfigurationData.java | 1 - .../impl/metrics/InstrumentProvider.java | 22 ++++++++++++++----- 10 files changed, 61 insertions(+), 52 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java index 15692a3815c2f..40d4a2bfaa005 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java @@ -93,20 +93,20 @@ public BinaryProtoLookupService(PulsarClientImpl client, this.listenerName = listenerName; updateServiceUrl(serviceUrl); - Attributes attrs = Attributes.of(AttributeKey.stringKey("transport-type"), "binary"); + Attributes attrs = Attributes.of(AttributeKey.stringKey("pulsar.lookup.transport-type"), "binary"); histoGetBroker = client.instrumentProvider().newLatencyHistogram("pulsar.client.lookup", - "Lookup operations", - attrs.toBuilder().put("type", "topic").build()); + "Duration of lookup operations", + attrs.toBuilder().put("pulsar.lookup.type", "topic").build()); histoGetTopicMetadata = client.instrumentProvider().newLatencyHistogram("pulsar.client.lookup", - "Lookup operations", - attrs.toBuilder().put("type", "metadata").build()); + "Duration of lookup operations", + attrs.toBuilder().put("pulsar.lookup.type", "metadata").build()); histoGetSchema = client.instrumentProvider().newLatencyHistogram("pulsar.client.lookup", - "Lookup operations", - attrs.toBuilder().put("type", "schema").build()); + "Duration of lookup operations", + attrs.toBuilder().put("pulsar.lookup.type", "schema").build()); histoListTopics = client.instrumentProvider().newLatencyHistogram("pulsar.client.lookup", - "Lookup operations", - attrs.toBuilder().put("type", "list-topics").build()); + "Duration of lookup operations", + attrs.toBuilder().put("pulsar.lookup.type", "list-topics").build()); } @Override @@ -160,9 +160,9 @@ public CompletableFuture getPartitionedTopicMetadata(T getPartitionedTopicMetadata(serviceNameResolver.resolveHost(), topicName); newFutureCreated.setValue(newFuture); newFuture.thenRun(() -> { - histoGetBroker.recordSuccess(System.nanoTime() - startTime); + histoGetTopicMetadata.recordSuccess(System.nanoTime() - startTime); }).exceptionally(x -> { - histoGetBroker.recordFailure(System.nanoTime() - startTime); + histoGetTopicMetadata.recordFailure(System.nanoTime() - startTime); return null; }); return newFuture; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java index 30fe13497feba..f7b42b1fcfa49 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java @@ -271,9 +271,9 @@ public ClientCnx(InstrumentProvider instrumentProvider, ClientConfigurationData this.clientVersion = "Pulsar-Java-v" + PulsarVersion.getVersion() + (conf.getDescription() == null ? "" : ("-" + conf.getDescription())); this.connectionsOpenedCounter = instrumentProvider.newCounter("pulsar.client.connections.opened", Unit.Connections, - "Counter of connections opened", Attributes.empty()); + "The number of connections opened", Attributes.empty()); this.connectionsClosedCounter = instrumentProvider.newCounter("pulsar.client.connections.closed", Unit.Connections, - "Counter of connections closed", Attributes.empty()); + "The number of connections closed", Attributes.empty()); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java index b5ae64cb693b4..93fb547827356 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java @@ -157,11 +157,12 @@ public ConnectionPool(InstrumentProvider instrumentProvider, }, idleDetectionIntervalSeconds, idleDetectionIntervalSeconds, TimeUnit.SECONDS); } - connectionsTcpFailureCounter = instrumentProvider.newCounter("pulsar.client.connections.failed", Unit.None, - "Counter of failed connections", Attributes.builder().put("type", "tcp-failed").build()); + connectionsTcpFailureCounter = + instrumentProvider.newCounter("pulsar.client.connections.failed", Unit.Connections, + "The number of failed connection attempts", Attributes.builder().put("pulsar.failure.type", "tcp-failed").build()); connectionsHandshakeFailureCounter = instrumentProvider.newCounter("pulsar.client.connections.failed", - Unit.None, "Counter of failed connections", - Attributes.builder().put("type", "handshake").build()); + Unit.Connections, "The number of failed connection attempts", + Attributes.builder().put("pulsar.failure.type", "handshake").build()); } private static AddressResolver createAddressResolver(ClientConfigurationData conf, diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java index 4474df778c892..d45c125c590de 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java @@ -1228,10 +1228,7 @@ public int getTotalIncomingMessages() { protected void clearIncomingMessages() { // release messages if they are pooled messages - incomingMessages.forEach(msg -> { - - msg.release(); - }); + incomingMessages.forEach(Message::release); incomingMessages.clear(); resetIncomingMessageSize(); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 13c5a37ae1113..f8212a8f1f9ad 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -408,19 +408,19 @@ protected ConsumerImpl(PulsarClientImpl client, String topic, ConsumerConfigurat "Counter of sessions opened", attrs.toBuilder().put("type", "consumer").build()); consumersClosedCounter = ip.newCounter("pulsar.client.session.closed", Unit.Sessions, "Counter of sessions closed", attrs.toBuilder().put("type", "consumer").build()); - messagesReceivedCounter = ip.newCounter("pulsar.client.received", Unit.Messages, - "Number of messages received", attrs); - bytesReceivedCounter = ip.newCounter("pulsar.client.received", Unit.Bytes, - "Bytes received", attrs); - messagesPrefetchedGauge = ip.newUpDownCounter("pulsar.client.consumer.preteched.messages", Unit.Messages, + messagesReceivedCounter = ip.newCounter("pulsar.client.received.count", Unit.Messages, + "The number of messages explicitly received by the consumer application", attrs); + bytesReceivedCounter = ip.newCounter("pulsar.client.received.size", Unit.Bytes, + "The number of bytes explicitly received by the consumer application", attrs); + messagesPrefetchedGauge = ip.newUpDownCounter("pulsar.client.consumer.prefetched.count", Unit.Messages, "Number of messages currently sitting in the consumer pre-fetch queue", attrs); - bytesPrefetchedGauge = ip.newUpDownCounter("pulsar.client.consumer.preteched", Unit.Bytes, + bytesPrefetchedGauge = ip.newUpDownCounter("pulsar.client.consumer.prefetched.size", Unit.Bytes, "Total number of bytes currently sitting in the consumer pre-fetch queue", attrs); - consumerAcksCounter = ip.newCounter("pulsar.client.consumer.ack", Unit.Messages, - "Number of ack operations", attrs); - consumerNacksCounter = ip.newCounter("pulsar.client.consumer.nack", Unit.Messages, - "Number of negative ack operations", attrs); + consumerAcksCounter = ip.newCounter("pulsar.client.consumer.message.ack", Unit.Messages, + "The number of acknowledged messages", attrs); + consumerNacksCounter = ip.newCounter("pulsar.client.consumer.message.nack", Unit.Messages, + "The number of negatively acknowledged messages", attrs); consumerDlqMessagesCounter = ip.newCounter("pulsar.client.consumer.dlq", Unit.Messages, "Number of messages sent to DLQ", attrs); grabCnx(); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java index 73f2127570d84..5f453d0ef32ac 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java @@ -75,20 +75,20 @@ public HttpLookupService(InstrumentProvider instrumentProvider, ClientConfigurat this.useTls = conf.isUseTls(); this.listenerName = conf.getListenerName(); - Attributes attrs = Attributes.of(AttributeKey.stringKey("transport-type"), "binary"); + Attributes attrs = Attributes.of(AttributeKey.stringKey("pulsar.lookup.transport-type"), "binary"); histoGetBroker = instrumentProvider.newLatencyHistogram("pulsar.client.lookup", - "Lookup operations", - attrs.toBuilder().put("type", "topic").build()); + "Duration of lookup operations", + attrs.toBuilder().put("pulsar.lookup.type", "topic").build()); histoGetTopicMetadata = instrumentProvider.newLatencyHistogram("pulsar.client.lookup", - "Lookup operations", - attrs.toBuilder().put("type", "metadata").build()); + "Duration of lookup operations", + attrs.toBuilder().put("pulsar.lookup.type", "metadata").build()); histoGetSchema = instrumentProvider.newLatencyHistogram("pulsar.client.lookup", - "Lookup operations", - attrs.toBuilder().put("type", "schema").build()); + "Duration of lookup operations", + attrs.toBuilder().put("pulsar.lookup.type", "schema").build()); histoListTopics = instrumentProvider.newLatencyHistogram("pulsar.client.lookup", - "Lookup operations", - attrs.toBuilder().put("type", "list-topics").build()); + "Duration of lookup operations", + attrs.toBuilder().put("pulsar.lookup.type", "list-topics").build()); } @Override diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java index 4405c2f5c9381..4618a56144b25 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java @@ -285,15 +285,15 @@ public ProducerImpl(PulsarClientImpl client, String topic, ProducerConfiguration InstrumentProvider ip = client.instrumentProvider(); Attributes attrs = ip.getAttributes(topic); - latencyHistogram = ip.newLatencyHistogram("pulsar.client.producer.latency", + latencyHistogram = ip.newLatencyHistogram("pulsar.client.producer.message.send.duration", "Publish latency experienced by the application, includes client batching time", attrs); - rpcLatencyHistogram = ip.newLatencyHistogram("pulsar.client.producer.rpc.latency", + rpcLatencyHistogram = ip.newLatencyHistogram("pulsar.client.producer.rpc.send.duration", "Publish RPC latency experienced internally by the client when sending data to receiving an ack", attrs); - publishedBytesCounter = ip.newCounter("pulsar.client.producer.published", + publishedBytesCounter = ip.newCounter("pulsar.client.producer.message.send.size", Unit.Bytes, "Bytes published", attrs); - pendingMessagesCounter = ip.newUpDownCounter("pulsar.client.producer.pending.messages.count", Unit.Messages, + pendingMessagesCounter = ip.newUpDownCounter("pulsar.client.producer.message.pending.count", Unit.Messages, "Pending messages for this producer", attrs); - pendingBytesCounter = ip.newUpDownCounter("pulsar.client.producer.pending.count", Unit.Bytes, + pendingBytesCounter = ip.newUpDownCounter("pulsar.client.producer.message.pending.size", Unit.Bytes, "Pending bytes for this producer", attrs); producersOpenedCounter = ip.newCounter("pulsar.client.session.opened", Unit.Sessions, "Counter of sessions opened", attrs.toBuilder().put("type", "producer").build()); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java index f5e841db68cd5..04781a34da7f5 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java @@ -124,7 +124,7 @@ public UnAckedMessageTracker(PulsarClientImpl client, ConsumerBase consumerBa InstrumentProvider ip = client.instrumentProvider(); Attributes attrs = ip.getAttributes(consumerBase.getTopic()); - consumerAckTimeoutsCounter = ip.newCounter("pulsar.client.consumer.ack.timeout", Unit.Messages, + consumerAckTimeoutsCounter = ip.newCounter("pulsar.client.consumer.message.ack.timeout", Unit.Messages, "Number of ack timeouts events", attrs); if (conf.getAckTimeoutRedeliveryBackoff() == null) { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java index 18bbf34e1a69d..a4aa29aa0f804 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java @@ -397,7 +397,6 @@ public class ClientConfigurationData implements Serializable, Cloneable { ) private String description; - private transient OpenTelemetry openTelemetry; private MetricsCardinality openTelemetryMetricsCardinality = MetricsCardinality.Topic; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/InstrumentProvider.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/InstrumentProvider.java index eb923d270c94c..b3f0566a1932f 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/InstrumentProvider.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/InstrumentProvider.java @@ -1,9 +1,11 @@ package org.apache.pulsar.client.impl.metrics; +import io.opentelemetry.api.GlobalOpenTelemetry; import io.opentelemetry.api.OpenTelemetry; import io.opentelemetry.api.common.Attributes; import io.opentelemetry.api.common.AttributesBuilder; import io.opentelemetry.api.metrics.Meter; +import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.client.api.MetricsCardinality; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.common.naming.TopicName; @@ -14,7 +16,17 @@ public class InstrumentProvider { private final MetricsCardinality metricsCardinality; public InstrumentProvider(ClientConfigurationData conf) { - this.meter = conf.getOpenTelemetry().getMeter("org.apache.pulsar.client"); + OpenTelemetry otel = conf.getOpenTelemetry(); + if (otel == null) { + // By default, metrics are disabled, unless the OTel java agent is configured. + // This allows to enable metrics without any code change. + otel = GlobalOpenTelemetry.get(); + } + + this.meter = otel.getMeterProvider() + .meterBuilder("org.apache.pulsar.client") + .setInstrumentationVersion(PulsarVersion.getVersion()) + .build(); this.metricsCardinality = conf.getOpenTelemetryMetricsCardinality(); } @@ -29,17 +41,17 @@ public Attributes getAttributes(String topic) { switch (metricsCardinality) { case Partition: if (tn.isPartitioned()) { - ab.put("partition", tn.getPartitionIndex()); + ab.put("pulsar.partition", tn.getPartitionIndex()); } // fallthrough case Topic: - ab.put("topic", tn.getPartitionedTopicName()); + ab.put("pulsar.topic", tn.getPartitionedTopicName()); // fallthrough case Namespace: - ab.put("namespace", tn.getNamespace()); + ab.put("pulsar.namespace", tn.getNamespace()); // fallthrough case Tenant: - ab.put("tenant", tn.getTenant()); + ab.put("pulsar.tenant", tn.getTenant()); } return ab.build(); From 4d0120b05e4c5dda834376bb248cfd7e6e434b28 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Mon, 4 Mar 2024 16:07:21 -0800 Subject: [PATCH 03/22] Removed MetricsCardinality from API --- .../pulsar/client/api/ClientBuilder.java | 2 - .../pulsar/client/api/MetricsCardinality.java | 47 -- pulsar-client/pom.xml | 5 + .../client/impl/BinaryProtoLookupService.java | 8 +- .../pulsar/client/impl/ClientBuilderImpl.java | 7 - .../apache/pulsar/client/impl/ClientCnx.java | 4 +- .../pulsar/client/impl/ConnectionPool.java | 5 +- .../pulsar/client/impl/ConsumerImpl.java | 23 +- .../pulsar/client/impl/HttpLookupService.java | 8 +- .../pulsar/client/impl/ProducerImpl.java | 439 +++++++++--------- .../client/impl/UnAckedMessageTracker.java | 3 +- .../impl/conf/ClientConfigurationData.java | 3 - .../pulsar/client/impl/metrics/Counter.java | 37 +- .../impl/metrics/InstrumentProvider.java | 64 ++- .../client/impl/metrics/LatencyHistogram.java | 37 +- .../client/impl/metrics/MetricsUtil.java | 59 +++ .../client/impl/metrics/UpDownCounter.java | 20 +- .../pulsar/testclient/PerfClientUtils.java | 4 +- 18 files changed, 424 insertions(+), 351 deletions(-) delete mode 100644 pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MetricsCardinality.java create mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/MetricsUtil.java diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java index e8ceb4e4f5ffa..83bc6adf501d9 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java @@ -560,8 +560,6 @@ ClientBuilder authentication(String authPluginClassName, Map aut ClientBuilder openTelemetry(io.opentelemetry.api.OpenTelemetry openTelemetry); - ClientBuilder openTelemetryMetricsCardinality(MetricsCardinality metricsCardinality); - /** * The clock used by the pulsar client. * diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MetricsCardinality.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MetricsCardinality.java deleted file mode 100644 index fd4275d3c0455..0000000000000 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MetricsCardinality.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * 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.client.api; - -public enum MetricsCardinality { - /** - * Do not add additional labels to metrics - */ - None, - - /** - * Label metrics by tenant - */ - Tenant, - - /** - * Label metrics by tenant and namespace - */ - Namespace, - - /** - * Label metrics by topic - */ - Topic, - - /** - * Label metrics by each partition - */ - Partition, -} diff --git a/pulsar-client/pom.xml b/pulsar-client/pom.xml index c775284b4d9ca..3917e2996e180 100644 --- a/pulsar-client/pom.xml +++ b/pulsar-client/pom.xml @@ -57,6 +57,11 @@ opentelemetry-api + + io.opentelemetry + opentelemetry-extension-incubator + + ${project.groupId} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java index 40d4a2bfaa005..6b379b1625076 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java @@ -96,16 +96,16 @@ public BinaryProtoLookupService(PulsarClientImpl client, Attributes attrs = Attributes.of(AttributeKey.stringKey("pulsar.lookup.transport-type"), "binary"); histoGetBroker = client.instrumentProvider().newLatencyHistogram("pulsar.client.lookup", - "Duration of lookup operations", + "Duration of lookup operations", null, attrs.toBuilder().put("pulsar.lookup.type", "topic").build()); histoGetTopicMetadata = client.instrumentProvider().newLatencyHistogram("pulsar.client.lookup", - "Duration of lookup operations", + "Duration of lookup operations", null, attrs.toBuilder().put("pulsar.lookup.type", "metadata").build()); histoGetSchema = client.instrumentProvider().newLatencyHistogram("pulsar.client.lookup", - "Duration of lookup operations", + "Duration of lookup operations", null, attrs.toBuilder().put("pulsar.lookup.type", "schema").build()); histoListTopics = client.instrumentProvider().newLatencyHistogram("pulsar.client.lookup", - "Duration of lookup operations", + "Duration of lookup operations", null, attrs.toBuilder().put("pulsar.lookup.type", "list-topics").build()); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java index 5029562ef1d0c..a17f5bc81b096 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java @@ -30,7 +30,6 @@ import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.ClientBuilder; -import org.apache.pulsar.client.api.MetricsCardinality; import org.apache.pulsar.client.api.ProxyProtocol; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; @@ -129,12 +128,6 @@ public ClientBuilder openTelemetry(OpenTelemetry openTelemetry) { return this; } - @Override - public ClientBuilder openTelemetryMetricsCardinality(MetricsCardinality metricsCardinality) { - conf.setOpenTelemetryMetricsCardinality(metricsCardinality); - return this; - } - @Override public ClientBuilder authentication(String authPluginClassName, String authParamsString) throws UnsupportedAuthenticationException { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java index f7b42b1fcfa49..f394ffa190e88 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java @@ -271,9 +271,9 @@ public ClientCnx(InstrumentProvider instrumentProvider, ClientConfigurationData this.clientVersion = "Pulsar-Java-v" + PulsarVersion.getVersion() + (conf.getDescription() == null ? "" : ("-" + conf.getDescription())); this.connectionsOpenedCounter = instrumentProvider.newCounter("pulsar.client.connections.opened", Unit.Connections, - "The number of connections opened", Attributes.empty()); + "The number of connections opened", null, Attributes.empty()); this.connectionsClosedCounter = instrumentProvider.newCounter("pulsar.client.connections.closed", Unit.Connections, - "The number of connections closed", Attributes.empty()); + "The number of connections closed", null, Attributes.empty()); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java index 93fb547827356..9bd7d0cd036c2 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java @@ -159,9 +159,10 @@ public ConnectionPool(InstrumentProvider instrumentProvider, connectionsTcpFailureCounter = instrumentProvider.newCounter("pulsar.client.connections.failed", Unit.Connections, - "The number of failed connection attempts", Attributes.builder().put("pulsar.failure.type", "tcp-failed").build()); + "The number of failed connection attempts", null, + Attributes.builder().put("pulsar.failure.type", "tcp-failed").build()); connectionsHandshakeFailureCounter = instrumentProvider.newCounter("pulsar.client.connections.failed", - Unit.Connections, "The number of failed connection attempts", + Unit.Connections, "The number of failed connection attempts", null, Attributes.builder().put("pulsar.failure.type", "handshake").build()); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index f8212a8f1f9ad..33623038d271f 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -403,26 +403,25 @@ protected ConsumerImpl(PulsarClientImpl client, String topic, ConsumerConfigurat topicNameWithoutPartition = topicName.getPartitionedTopicName(); InstrumentProvider ip = client.instrumentProvider(); - Attributes attrs = ip.getAttributes(topic); - consumersOpenedCounter = ip.newCounter("pulsar.client.session.opened", Unit.Sessions, - "Counter of sessions opened", attrs.toBuilder().put("type", "consumer").build()); - consumersClosedCounter = ip.newCounter("pulsar.client.session.closed", Unit.Sessions, - "Counter of sessions closed", attrs.toBuilder().put("type", "consumer").build()); + consumersOpenedCounter = ip.newCounter("pulsar.client.consumer.opened", Unit.Sessions, + "Counter of sessions opened", topic, Attributes.empty()); + consumersClosedCounter = ip.newCounter("pulsar.client.consumer.closed", Unit.Sessions, + "Counter of sessions closed", topic, Attributes.empty()); messagesReceivedCounter = ip.newCounter("pulsar.client.received.count", Unit.Messages, - "The number of messages explicitly received by the consumer application", attrs); + "The number of messages explicitly received by the consumer application", topic, Attributes.empty()); bytesReceivedCounter = ip.newCounter("pulsar.client.received.size", Unit.Bytes, - "The number of bytes explicitly received by the consumer application", attrs); + "The number of bytes explicitly received by the consumer application", topic, Attributes.empty()); messagesPrefetchedGauge = ip.newUpDownCounter("pulsar.client.consumer.prefetched.count", Unit.Messages, - "Number of messages currently sitting in the consumer pre-fetch queue", attrs); + "Number of messages currently sitting in the consumer pre-fetch queue", topic, Attributes.empty()); bytesPrefetchedGauge = ip.newUpDownCounter("pulsar.client.consumer.prefetched.size", Unit.Bytes, - "Total number of bytes currently sitting in the consumer pre-fetch queue", attrs); + "Total number of bytes currently sitting in the consumer pre-fetch queue", topic, Attributes.empty()); consumerAcksCounter = ip.newCounter("pulsar.client.consumer.message.ack", Unit.Messages, - "The number of acknowledged messages", attrs); + "The number of acknowledged messages", topic, Attributes.empty()); consumerNacksCounter = ip.newCounter("pulsar.client.consumer.message.nack", Unit.Messages, - "The number of negatively acknowledged messages", attrs); + "The number of negatively acknowledged messages", topic, Attributes.empty()); consumerDlqMessagesCounter = ip.newCounter("pulsar.client.consumer.dlq", Unit.Messages, - "Number of messages sent to DLQ", attrs); + "Number of messages sent to DLQ", topic, Attributes.empty()); grabCnx(); consumersOpenedCounter.increment(); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java index 5f453d0ef32ac..c20461f7e8c8d 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java @@ -79,16 +79,16 @@ public HttpLookupService(InstrumentProvider instrumentProvider, ClientConfigurat histoGetBroker = instrumentProvider.newLatencyHistogram("pulsar.client.lookup", "Duration of lookup operations", - attrs.toBuilder().put("pulsar.lookup.type", "topic").build()); + null, attrs.toBuilder().put("pulsar.lookup.type", "topic").build()); histoGetTopicMetadata = instrumentProvider.newLatencyHistogram("pulsar.client.lookup", "Duration of lookup operations", - attrs.toBuilder().put("pulsar.lookup.type", "metadata").build()); + null, attrs.toBuilder().put("pulsar.lookup.type", "metadata").build()); histoGetSchema = instrumentProvider.newLatencyHistogram("pulsar.client.lookup", "Duration of lookup operations", - attrs.toBuilder().put("pulsar.lookup.type", "schema").build()); + null, attrs.toBuilder().put("pulsar.lookup.type", "schema").build()); histoListTopics = instrumentProvider.newLatencyHistogram("pulsar.client.lookup", "Duration of lookup operations", - attrs.toBuilder().put("pulsar.lookup.type", "list-topics").build()); + null, attrs.toBuilder().put("pulsar.lookup.type", "list-topics").build()); } @Override diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java index 4618a56144b25..17da214626d81 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java @@ -40,6 +40,7 @@ import io.netty.util.Timeout; import io.netty.util.TimerTask; import io.netty.util.concurrent.ScheduledFuture; +import io.opentelemetry.api.common.AttributeKey; import io.opentelemetry.api.common.Attributes; import java.io.IOException; import java.net.URI; @@ -284,29 +285,31 @@ public ProducerImpl(PulsarClientImpl client, String topic, ProducerConfiguration } InstrumentProvider ip = client.instrumentProvider(); - Attributes attrs = ip.getAttributes(topic); latencyHistogram = ip.newLatencyHistogram("pulsar.client.producer.message.send.duration", - "Publish latency experienced by the application, includes client batching time", attrs); + "Publish latency experienced by the application, includes client batching time", topic, + Attributes.empty()); rpcLatencyHistogram = ip.newLatencyHistogram("pulsar.client.producer.rpc.send.duration", - "Publish RPC latency experienced internally by the client when sending data to receiving an ack", attrs); + "Publish RPC latency experienced internally by the client when sending data to receiving an ack", topic, + Attributes.empty()); publishedBytesCounter = ip.newCounter("pulsar.client.producer.message.send.size", - Unit.Bytes, "Bytes published", attrs); + Unit.Bytes, "Bytes published", topic, Attributes.empty()); pendingMessagesCounter = ip.newUpDownCounter("pulsar.client.producer.message.pending.count", Unit.Messages, - "Pending messages for this producer", attrs); + "Pending messages for this producer", topic, Attributes.empty()); pendingBytesCounter = ip.newUpDownCounter("pulsar.client.producer.message.pending.size", Unit.Bytes, - "Pending bytes for this producer", attrs); - producersOpenedCounter = ip.newCounter("pulsar.client.session.opened", Unit.Sessions, - "Counter of sessions opened", attrs.toBuilder().put("type", "producer").build()); - producersClosedCounter = ip.newCounter("pulsar.client.session.closed", Unit.Sessions, - "Counter of sessions closed", attrs.toBuilder().put("type", "producer").build()); + "Pending bytes for this producer", topic, Attributes.empty()); + producersOpenedCounter = ip.newCounter("pulsar.client.producer.opened", Unit.Sessions, + "Counter of producer sessions opened", topic, Attributes.empty()); + producersClosedCounter = ip.newCounter("pulsar.client.producer.closed", Unit.Sessions, + "Counter of producer sessions closed", topic, Attributes.empty()); this.connectionHandler = new ConnectionHandler(this, - new BackoffBuilder() - .setInitialTime(client.getConfiguration().getInitialBackoffIntervalNanos(), TimeUnit.NANOSECONDS) - .setMax(client.getConfiguration().getMaxBackoffIntervalNanos(), TimeUnit.NANOSECONDS) - .setMandatoryStop(Math.max(100, conf.getSendTimeoutMs() - 100), TimeUnit.MILLISECONDS) - .create(), - this); + new BackoffBuilder() + .setInitialTime(client.getConfiguration().getInitialBackoffIntervalNanos(), + TimeUnit.NANOSECONDS) + .setMax(client.getConfiguration().getMaxBackoffIntervalNanos(), TimeUnit.NANOSECONDS) + .setMandatoryStop(Math.max(100, conf.getSendTimeoutMs() - 100), TimeUnit.MILLISECONDS) + .create(), + this); grabCnx(); producersOpenedCounter.increment(); @@ -455,15 +458,16 @@ CompletableFuture internalSendWithTxnAsync(Message message, Transa } else { CompletableFuture completableFuture = new CompletableFuture<>(); if (!((TransactionImpl) txn).checkIfOpen(completableFuture)) { - return completableFuture; + return completableFuture; } return ((TransactionImpl) txn).registerProducedTopic(topic) - .thenCompose(ignored -> internalSendAsync(message)); + .thenCompose(ignored -> internalSendAsync(message)); } } /** * Compress the payload if compression is configured. + * * @param payload * @return a new payload */ @@ -507,7 +511,8 @@ public void sendAsync(Message message, SendCallback callback) { new PulsarClientException.InvalidMessageException( format("The producer %s of the topic %s sends a %s message with %d bytes that exceeds" + " %d bytes", - producerName, topic, compressedStr, compressedSize, ClientCnx.getMaxMessageSize())); + producerName, topic, compressedStr, compressedSize, + ClientCnx.getMaxMessageSize())); completeCallbackAndReleaseSemaphore(uncompressedSize, callback, invalidMessageException); return; } @@ -515,9 +520,10 @@ public void sendAsync(Message message, SendCallback callback) { if (!msg.isReplicated() && msgMetadata.hasProducerName()) { PulsarClientException.InvalidMessageException invalidMessageException = - new PulsarClientException.InvalidMessageException( - format("The producer %s of the topic %s can not reuse the same message", producerName, topic), - msg.getSequenceId()); + new PulsarClientException.InvalidMessageException( + format("The producer %s of the topic %s can not reuse the same message", producerName, + topic), + msg.getSequenceId()); completeCallbackAndReleaseSemaphore(uncompressedSize, callback, invalidMessageException); compressedPayload.release(); return; @@ -687,8 +693,8 @@ private void serializeAndSendMessage(MessageImpl msg, msgMetadata.setUuid(uuid); } msgMetadata.setChunkId(chunkId) - .setNumChunksFromMsg(totalChunks) - .setTotalChunkMsgSize(compressedPayloadSize); + .setNumChunksFromMsg(totalChunks) + .setTotalChunkMsgSize(compressedPayloadSize); } if (canAddToBatch(msg) && totalChunks <= 1) { @@ -822,8 +828,8 @@ private void tryRegisterSchema(ClientCnx cnx, MessageImpl msg, SendCallback call } SchemaInfo schemaInfo = msg.hasReplicateFrom() ? msg.getSchemaInfoForReplicator() : msg.getSchemaInfo(); schemaInfo = Optional.ofNullable(schemaInfo) - .filter(si -> si.getType().getValue() > 0) - .orElse(Schema.BYTES.getSchemaInfo()); + .filter(si -> si.getType().getValue() > 0) + .orElse(Schema.BYTES.getSchemaInfo()); getOrCreateSchemaAsync(cnx, schemaInfo).handle((v, ex) -> { if (ex != null) { Throwable t = FutureUtil.unwrapCompletionException(ex); @@ -858,10 +864,10 @@ private void tryRegisterSchema(ClientCnx cnx, MessageImpl msg, SendCallback call private CompletableFuture getOrCreateSchemaAsync(ClientCnx cnx, SchemaInfo schemaInfo) { if (!Commands.peerSupportsGetOrCreateSchema(cnx.getRemoteEndpointProtocolVersion())) { return FutureUtil.failedFuture( - new PulsarClientException.NotSupportedException( - format("The command `GetOrCreateSchema` is not supported for the protocol version %d. " - + "The producer is %s, topic is %s", - cnx.getRemoteEndpointProtocolVersion(), producerName, topic))); + new PulsarClientException.NotSupportedException( + format("The command `GetOrCreateSchema` is not supported for the protocol version %d. " + + "The producer is %s, topic is %s", + cnx.getRemoteEndpointProtocolVersion(), producerName, topic))); } long requestId = client.newRequestId(); ByteBuf request = Commands.newGetOrCreateSchema(requestId, topic, schemaInfo); @@ -933,7 +939,7 @@ private boolean canAddToBatch(MessageImpl msg) { private boolean canAddToCurrentBatch(MessageImpl msg) { return batchMessageContainer.haveEnoughSpace(msg) - && (!isMultiSchemaEnabled(false) || batchMessageContainer.hasSameSchema(msg)) + && (!isMultiSchemaEnabled(false) || batchMessageContainer.hasSameSchema(msg)) && batchMessageContainer.hasSameTxn(msg); } @@ -962,30 +968,31 @@ private void doBatchSendAndAdd(MessageImpl msg, SendCallback callback, ByteBu private boolean isValidProducerState(SendCallback callback, long sequenceId) { switch (getState()) { - case Ready: - // OK - case Connecting: - // We are OK to queue the messages on the client, it will be sent to the broker once we get the connection - case RegisteringSchema: - // registering schema - return true; - case Closing: - case Closed: - callback.sendComplete( - new PulsarClientException.AlreadyClosedException("Producer already closed", sequenceId)); - return false; - case ProducerFenced: - callback.sendComplete(new PulsarClientException.ProducerFencedException("Producer was fenced")); - return false; - case Terminated: - callback.sendComplete( - new PulsarClientException.TopicTerminatedException("Topic was terminated", sequenceId)); - return false; - case Failed: - case Uninitialized: - default: - callback.sendComplete(new PulsarClientException.NotConnectedException(sequenceId)); - return false; + case Ready: + // OK + case Connecting: + // We are OK to queue the messages on the client, it will be sent to the broker once we get the + // connection + case RegisteringSchema: + // registering schema + return true; + case Closing: + case Closed: + callback.sendComplete( + new PulsarClientException.AlreadyClosedException("Producer already closed", sequenceId)); + return false; + case ProducerFenced: + callback.sendComplete(new PulsarClientException.ProducerFencedException("Producer was fenced")); + return false; + case Terminated: + callback.sendComplete( + new PulsarClientException.TopicTerminatedException("Topic was terminated", sequenceId)); + return false; + case Failed: + case Uninitialized: + default: + callback.sendComplete(new PulsarClientException.NotConnectedException(sequenceId)); + return false; } } @@ -1085,9 +1092,11 @@ private static final class LastSendFutureWrapper { private LastSendFutureWrapper(CompletableFuture lastSendFuture) { this.lastSendFuture = lastSendFuture; } + static LastSendFutureWrapper create(CompletableFuture lastSendFuture) { return new LastSendFutureWrapper(lastSendFuture); } + public CompletableFuture handleOnce() { return lastSendFuture.handle((ignore, t) -> { if (t != null && THROW_ONCE_UPDATER.compareAndSet(this, FALSE, TRUE)) { @@ -1319,9 +1328,10 @@ protected synchronized void recoverChecksumError(ClientCnx cnx, long sequenceId) releaseSemaphoreForSendOp(op); try { op.sendComplete( - new PulsarClientException.ChecksumException( - format("The checksum of the message which is produced by producer %s to the topic " - + "%s is corrupted", producerName, topic))); + new PulsarClientException.ChecksumException( + format("The checksum of the message which is produced by producer %s to the " + + "topic " + + "%s is corrupted", producerName, topic))); } catch (Throwable t) { log.warn("[{}] [{}] Got exception while completing the callback for msg {}:", topic, producerName, sequenceId, t); @@ -1369,7 +1379,7 @@ protected synchronized void recoverNotAllowedError(long sequenceId, String error * * @param op * @return returns true only if message is not modified and computed-checksum is same as previous checksum else - * return false that means that message is corrupted. Returns true if checksum is not present. + * return false that means that message is corrupted. Returns true if checksum is not present. */ protected boolean verifyLocalBufferIsNotCorrupted(OpSendMsg op) { ByteBufPair msg = op.cmd; @@ -1485,7 +1495,8 @@ void initialize() { chunkedMessageCtx = null; } - static OpSendMsg create(ProducerImpl producer, MessageImpl msg, ByteBufPair cmd, long sequenceId, SendCallback callback) { + static OpSendMsg create(ProducerImpl producer, MessageImpl msg, ByteBufPair cmd, long sequenceId, + SendCallback callback) { OpSendMsg op = RECYCLER.get(); op.initialize(); op.producer = producer; @@ -1498,7 +1509,8 @@ static OpSendMsg create(ProducerImpl producer, MessageImpl msg, ByteBufPai return op; } - static OpSendMsg create(ProducerImpl producer, List> msgs, ByteBufPair cmd, long sequenceId, SendCallback callback, + static OpSendMsg create(ProducerImpl producer, List> msgs, ByteBufPair cmd, long sequenceId, + SendCallback callback, int batchAllocatedSize) { OpSendMsg op = RECYCLER.get(); op.initialize(); @@ -1516,8 +1528,9 @@ static OpSendMsg create(ProducerImpl producer, List> msgs, Byt return op; } - static OpSendMsg create(ProducerImpl producer, List> msgs, ByteBufPair cmd, long lowestSequenceId, - long highestSequenceId, SendCallback callback, int batchAllocatedSize) { + static OpSendMsg create(ProducerImpl producer, List> msgs, ByteBufPair cmd, + long lowestSequenceId, + long highestSequenceId, SendCallback callback, int batchAllocatedSize) { OpSendMsg op = RECYCLER.get(); op.initialize(); op.producer = producer; @@ -1555,17 +1568,17 @@ void sendComplete(final Exception e) { //firstSentAt and lastSentAt maybe -1, it means that the message didn't flush to channel. String errMsg = String.format( - "%s : createdAt %s seconds ago, firstSentAt %s seconds ago, lastSentAt %s seconds ago, " - + "retryCount %s", - te.getMessage(), - RelativeTimeUtil.nsToSeconds(now - this.createdAt), - RelativeTimeUtil.nsToSeconds(this.firstSentAt <= 0 - ? this.firstSentAt - : now - this.firstSentAt), - RelativeTimeUtil.nsToSeconds(this.lastSentAt <= 0 - ? this.lastSentAt - : now - this.lastSentAt), - retryCount + "%s : createdAt %s seconds ago, firstSentAt %s seconds ago, lastSentAt %s seconds ago, " + + "retryCount %s", + te.getMessage(), + RelativeTimeUtil.nsToSeconds(now - this.createdAt), + RelativeTimeUtil.nsToSeconds(this.firstSentAt <= 0 + ? this.firstSentAt + : now - this.firstSentAt), + RelativeTimeUtil.nsToSeconds(this.lastSentAt <= 0 + ? this.lastSentAt + : now - this.lastSentAt), + retryCount ); finalEx = new TimeoutException(errMsg, sequenceId); @@ -1742,7 +1755,7 @@ public CompletableFuture connectionOpened(final ClientCnx cnx) { long requestId = client.newRequestId(); PRODUCER_DEADLINE_UPDATER - .compareAndSet(this, 0, System.currentTimeMillis() + client.getConfiguration().getOperationTimeoutMs()); + .compareAndSet(this, 0, System.currentTimeMillis() + client.getConfiguration().getOperationTimeoutMs()); SchemaInfo schemaInfo = null; if (schema != null) { @@ -1753,7 +1766,7 @@ public CompletableFuture connectionOpened(final ClientCnx cnx) { // but now we have standardized on every schema to generate an Avro based schema if (Commands.peerSupportJsonSchemaAvroFormat(cnx.getRemoteEndpointProtocolVersion())) { schemaInfo = schema.getSchemaInfo(); - } else if (schema instanceof JSONSchema){ + } else if (schema instanceof JSONSchema) { JSONSchema jsonSchema = (JSONSchema) schema; schemaInfo = jsonSchema.getBackwardsCompatibleJsonSchemaInfo(); } else { @@ -1776,146 +1789,146 @@ public CompletableFuture connectionOpened(final ClientCnx cnx) { conf.getAccessMode(), topicEpoch, client.conf.isEnableTransaction(), conf.getInitialSubscriptionName()), requestId).thenAccept(response -> { - String producerName = response.getProducerName(); - long lastSequenceId = response.getLastSequenceId(); - schemaVersion = Optional.ofNullable(response.getSchemaVersion()); - schemaVersion.ifPresent(v -> schemaCache.put(SchemaHash.of(schema), v)); - - // We are now reconnected to broker and clear to send messages. Re-send all pending messages and - // set the cnx pointer so that new messages will be sent immediately - synchronized (ProducerImpl.this) { - State state = getState(); - if (state == State.Closing || state == State.Closed) { - // Producer was closed while reconnecting, close the connection to make sure the broker - // drops the producer on its side - cnx.removeProducer(producerId); - cnx.channel().close(); - future.complete(null); - return; - } - resetBackoff(); + String producerName = response.getProducerName(); + long lastSequenceId = response.getLastSequenceId(); + schemaVersion = Optional.ofNullable(response.getSchemaVersion()); + schemaVersion.ifPresent(v -> schemaCache.put(SchemaHash.of(schema), v)); - log.info("[{}] [{}] Created producer on cnx {}", topic, producerName, cnx.ctx().channel()); - connectionId = cnx.ctx().channel().toString(); - connectedSince = DateFormatter.now(); - if (conf.getAccessMode() != ProducerAccessMode.Shared && !topicEpoch.isPresent()) { - log.info("[{}] [{}] Producer epoch is {}", topic, producerName, response.getTopicEpoch()); - } - topicEpoch = response.getTopicEpoch(); + // We are now reconnected to broker and clear to send messages. Re-send all pending messages and + // set the cnx pointer so that new messages will be sent immediately + synchronized (ProducerImpl.this) { + State state = getState(); + if (state == State.Closing || state == State.Closed) { + // Producer was closed while reconnecting, close the connection to make sure the broker + // drops the producer on its side + cnx.removeProducer(producerId); + cnx.channel().close(); + future.complete(null); + return; + } + resetBackoff(); - if (this.producerName == null) { - this.producerName = producerName; - } + log.info("[{}] [{}] Created producer on cnx {}", topic, producerName, cnx.ctx().channel()); + connectionId = cnx.ctx().channel().toString(); + connectedSince = DateFormatter.now(); + if (conf.getAccessMode() != ProducerAccessMode.Shared && !topicEpoch.isPresent()) { + log.info("[{}] [{}] Producer epoch is {}", topic, producerName, response.getTopicEpoch()); + } + topicEpoch = response.getTopicEpoch(); - if (this.msgIdGenerator == 0 && conf.getInitialSequenceId() == null) { - // Only update sequence id generator if it wasn't already modified. That means we only want - // to update the id generator the first time the producer gets established, and ignore the - // sequence id sent by broker in subsequent producer reconnects - this.lastSequenceIdPublished = lastSequenceId; - this.msgIdGenerator = lastSequenceId + 1; - } + if (this.producerName == null) { + this.producerName = producerName; + } - resendMessages(cnx, epoch); - } - future.complete(null); - }).exceptionally((e) -> { - Throwable cause = e.getCause(); - cnx.removeProducer(producerId); - State state = getState(); - if (state == State.Closing || state == State.Closed) { - // Producer was closed while reconnecting, close the connection to make sure the broker - // drops the producer on its side - cnx.channel().close(); - future.complete(null); - return null; - } + if (this.msgIdGenerator == 0 && conf.getInitialSequenceId() == null) { + // Only update sequence id generator if it wasn't already modified. That means we only want + // to update the id generator the first time the producer gets established, and ignore the + // sequence id sent by broker in subsequent producer reconnects + this.lastSequenceIdPublished = lastSequenceId; + this.msgIdGenerator = lastSequenceId + 1; + } - if (cause instanceof TimeoutException) { - // Creating the producer has timed out. We need to ensure the broker closes the producer - // in case it was indeed created, otherwise it might prevent new create producer operation, - // since we are not necessarily closing the connection. - long closeRequestId = client.newRequestId(); - ByteBuf cmd = Commands.newCloseProducer(producerId, closeRequestId); - cnx.sendRequestWithId(cmd, closeRequestId); - } + resendMessages(cnx, epoch); + } + future.complete(null); + }).exceptionally((e) -> { + Throwable cause = e.getCause(); + cnx.removeProducer(producerId); + State state = getState(); + if (state == State.Closing || state == State.Closed) { + // Producer was closed while reconnecting, close the connection to make sure the broker + // drops the producer on its side + cnx.channel().close(); + future.complete(null); + return null; + } - if (cause instanceof PulsarClientException.ProducerFencedException) { - if (log.isDebugEnabled()) { - log.debug("[{}] [{}] Failed to create producer: {}", - topic, producerName, cause.getMessage()); - } - } else { - log.error("[{}] [{}] Failed to create producer: {}", topic, producerName, cause.getMessage()); + if (cause instanceof TimeoutException) { + // Creating the producer has timed out. We need to ensure the broker closes the producer + // in case it was indeed created, otherwise it might prevent new create producer operation, + // since we are not necessarily closing the connection. + long closeRequestId = client.newRequestId(); + ByteBuf cmd = Commands.newCloseProducer(producerId, closeRequestId); + cnx.sendRequestWithId(cmd, closeRequestId); + } + + if (cause instanceof PulsarClientException.ProducerFencedException) { + if (log.isDebugEnabled()) { + log.debug("[{}] [{}] Failed to create producer: {}", + topic, producerName, cause.getMessage()); + } + } else { + log.error("[{}] [{}] Failed to create producer: {}", topic, producerName, cause.getMessage()); + } + // Close the producer since topic does not exist. + if (cause instanceof PulsarClientException.TopicDoesNotExistException) { + closeAsync().whenComplete((v, ex) -> { + if (ex != null) { + log.error("Failed to close producer on TopicDoesNotExistException.", ex); } - // Close the producer since topic does not exist. - if (cause instanceof PulsarClientException.TopicDoesNotExistException) { - closeAsync().whenComplete((v, ex) -> { - if (ex != null) { - log.error("Failed to close producer on TopicDoesNotExistException.", ex); - } - producerCreatedFuture.completeExceptionally(cause); - }); - future.complete(null); - return null; + producerCreatedFuture.completeExceptionally(cause); + }); + future.complete(null); + return null; + } + if (cause instanceof PulsarClientException.ProducerBlockedQuotaExceededException) { + synchronized (this) { + log.warn("[{}] [{}] Topic backlog quota exceeded. Throwing Exception on producer.", topic, + producerName); + + if (log.isDebugEnabled()) { + log.debug("[{}] [{}] Pending messages: {}", topic, producerName, + pendingMessages.messagesCount()); } - if (cause instanceof PulsarClientException.ProducerBlockedQuotaExceededException) { - synchronized (this) { - log.warn("[{}] [{}] Topic backlog quota exceeded. Throwing Exception on producer.", topic, - producerName); - - if (log.isDebugEnabled()) { - log.debug("[{}] [{}] Pending messages: {}", topic, producerName, - pendingMessages.messagesCount()); - } - - PulsarClientException bqe = new PulsarClientException.ProducerBlockedQuotaExceededException( - format("The backlog quota of the topic %s that the producer %s produces to is exceeded", + + PulsarClientException bqe = new PulsarClientException.ProducerBlockedQuotaExceededException( + format("The backlog quota of the topic %s that the producer %s produces to is exceeded", topic, producerName)); - failPendingMessages(cnx(), bqe); - } - } else if (cause instanceof PulsarClientException.ProducerBlockedQuotaExceededError) { - log.warn("[{}] [{}] Producer is blocked on creation because backlog exceeded on topic.", - producerName, topic); - } + failPendingMessages(cnx(), bqe); + } + } else if (cause instanceof PulsarClientException.ProducerBlockedQuotaExceededError) { + log.warn("[{}] [{}] Producer is blocked on creation because backlog exceeded on topic.", + producerName, topic); + } - if (cause instanceof PulsarClientException.TopicTerminatedException) { - setState(State.Terminated); - synchronized (this) { - failPendingMessages(cnx(), (PulsarClientException) cause); - } - producerCreatedFuture.completeExceptionally(cause); - closeProducerTasks(); - client.cleanupProducer(this); - } else if (cause instanceof PulsarClientException.ProducerFencedException) { - setState(State.ProducerFenced); - synchronized (this) { - failPendingMessages(cnx(), (PulsarClientException) cause); - } - producerCreatedFuture.completeExceptionally(cause); - closeProducerTasks(); - client.cleanupProducer(this); - } else if (producerCreatedFuture.isDone() || // - (cause instanceof PulsarClientException && PulsarClientException.isRetriableError(cause) - && System.currentTimeMillis() < PRODUCER_DEADLINE_UPDATER.get(ProducerImpl.this))) { - // Either we had already created the producer once (producerCreatedFuture.isDone()) or we are - // still within the initial timeout budget and we are dealing with a retriable error - future.completeExceptionally(cause); - } else { - setState(State.Failed); - producerCreatedFuture.completeExceptionally(cause); - closeProducerTasks(); - client.cleanupProducer(this); - Timeout timeout = sendTimeout; - if (timeout != null) { - timeout.cancel(); - sendTimeout = null; - } - } - if (!future.isDone()) { - future.complete(null); - } - return null; - }); + if (cause instanceof PulsarClientException.TopicTerminatedException) { + setState(State.Terminated); + synchronized (this) { + failPendingMessages(cnx(), (PulsarClientException) cause); + } + producerCreatedFuture.completeExceptionally(cause); + closeProducerTasks(); + client.cleanupProducer(this); + } else if (cause instanceof PulsarClientException.ProducerFencedException) { + setState(State.ProducerFenced); + synchronized (this) { + failPendingMessages(cnx(), (PulsarClientException) cause); + } + producerCreatedFuture.completeExceptionally(cause); + closeProducerTasks(); + client.cleanupProducer(this); + } else if (producerCreatedFuture.isDone() || // + (cause instanceof PulsarClientException && PulsarClientException.isRetriableError(cause) + && System.currentTimeMillis() < PRODUCER_DEADLINE_UPDATER.get(ProducerImpl.this))) { + // Either we had already created the producer once (producerCreatedFuture.isDone()) or we are + // still within the initial timeout budget and we are dealing with a retriable error + future.completeExceptionally(cause); + } else { + setState(State.Failed); + producerCreatedFuture.completeExceptionally(cause); + closeProducerTasks(); + client.cleanupProducer(this); + Timeout timeout = sendTimeout; + if (timeout != null) { + timeout.cancel(); + sendTimeout = null; + } + } + if (!future.isDone()) { + future.complete(null); + } + return null; + }); return future; } @@ -2021,7 +2034,7 @@ private void stripChecksum(OpSendMsg op) { headerFrame.setInt(0, newTotalFrameSizeLength); // rewrite new [total-size] ByteBuf metadata = headerFrame.slice(checksumMark, headerFrameSize - checksumMark); // sliced only - // metadata + // metadata headerFrame.writerIndex(headerSize); // set headerFrame write-index to overwrite metadata over checksum metadata.readBytes(headerFrame, metadata.readableBytes()); headerFrame.capacity(headerFrameSize - checksumSize); // reduce capacity by removed checksum bytes @@ -2157,7 +2170,6 @@ private void failPendingMessages(ClientCnx cnx, PulsarClientException ex) { /** * fail any pending batch messages that were enqueued, however batch was not closed out. - * */ private void failPendingBatchMessages(PulsarClientException ex) { if (batchMessageContainer.isEmpty()) { @@ -2177,7 +2189,7 @@ public CompletableFuture flushAsync() { if (isBatchMessagingEnabled()) { batchMessageAndSend(false); } - CompletableFuture lastSendFuture = this.lastSendFuture; + CompletableFuture lastSendFuture = this.lastSendFuture; if (!(lastSendFuture == this.lastSendFutureWrapper.lastSendFuture)) { this.lastSendFutureWrapper = LastSendFutureWrapper.create(lastSendFuture); } @@ -2296,7 +2308,7 @@ protected void processOpSendMsg(OpSendMsg op) { } else { if (log.isDebugEnabled()) { log.debug("[{}] [{}] Connection is not ready -- sequenceId {}", topic, producerName, - op.sequenceId); + op.sequenceId); } } } catch (Throwable t) { @@ -2312,7 +2324,8 @@ private void recoverProcessOpSendMsgFrom(ClientCnx cnx, MessageImpl from, long e // In this case, the cnx passed to this method is no longer the active connection. This method will get // called again once the new connection registers the producer with the broker. log.info("[{}][{}] Producer epoch mismatch or the current connection is null. Skip re-sending the " - + " {} pending messages since they will deliver using another connection.", topic, producerName, + + " {} pending messages since they will deliver using another connection.", topic, + producerName, pendingMessages.messagesCount()); return; } @@ -2353,7 +2366,7 @@ private void recoverProcessOpSendMsgFrom(ClientCnx cnx, MessageImpl from, long e op.cmd.retain(); if (log.isDebugEnabled()) { log.debug("[{}] [{}] Re-Sending message in cnx {}, sequenceId {}", topic, producerName, - cnx.channel(), op.sequenceId); + cnx.channel(), op.sequenceId); } cnx.ctx().write(op.cmd, cnx.ctx().voidPromise()); op.updateSentTimestamp(); @@ -2377,7 +2390,7 @@ private void recoverProcessOpSendMsgFrom(ClientCnx cnx, MessageImpl from, long e } /** - * Check if final message size for non-batch and non-chunked messages is larger than max message size. + * Check if final message size for non-batch and non-chunked messages is larger than max message size. */ private boolean isMessageSizeExceeded(OpSendMsg op) { if (op.msg != null && !conf.isChunkingEnabled()) { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java index 04781a34da7f5..ebed22ad78bcb 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java @@ -123,9 +123,8 @@ public UnAckedMessageTracker(PulsarClientImpl client, ConsumerBase consumerBa this.writeLock = readWriteLock.writeLock(); InstrumentProvider ip = client.instrumentProvider(); - Attributes attrs = ip.getAttributes(consumerBase.getTopic()); consumerAckTimeoutsCounter = ip.newCounter("pulsar.client.consumer.message.ack.timeout", Unit.Messages, - "Number of ack timeouts events", attrs); + "Number of ack timeouts events", consumerBase.getTopic(), Attributes.empty()); if (conf.getAckTimeoutRedeliveryBackoff() == null) { this.messageIdPartitionMap = new HashMap<>(); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java index a4aa29aa0f804..c50f1b542c6c7 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java @@ -38,7 +38,6 @@ import lombok.Data; import lombok.NoArgsConstructor; import org.apache.pulsar.client.api.Authentication; -import org.apache.pulsar.client.api.MetricsCardinality; import org.apache.pulsar.client.api.ProxyProtocol; import org.apache.pulsar.client.api.ServiceUrlProvider; import org.apache.pulsar.client.impl.auth.AuthenticationDisabled; @@ -399,8 +398,6 @@ public class ClientConfigurationData implements Serializable, Cloneable { private transient OpenTelemetry openTelemetry; - private MetricsCardinality openTelemetryMetricsCardinality = MetricsCardinality.Topic; - /** * Gets the authentication settings for the client. * diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Counter.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Counter.java index f573d5557dfe1..ab4c3016df303 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Counter.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Counter.java @@ -1,19 +1,48 @@ +/* + * 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.client.impl.metrics; +import static org.apache.pulsar.client.impl.metrics.MetricsUtil.*; import io.opentelemetry.api.common.Attributes; import io.opentelemetry.api.metrics.LongCounter; +import io.opentelemetry.api.metrics.LongCounterBuilder; import io.opentelemetry.api.metrics.Meter; +import io.opentelemetry.extension.incubator.metrics.ExtendedLongCounterBuilder; public class Counter { private final LongCounter counter; private final Attributes attributes; - Counter(Meter meter, String name, Unit unit, String description, Attributes attributes) { - counter = meter.counterBuilder(name) + Counter(Meter meter, String name, Unit unit, String description, String topic, Attributes attributes) { + LongCounterBuilder builder = meter.counterBuilder(name) .setDescription(description) - .setUnit(unit.toString()) - .build(); + .setUnit(unit.toString()); + + if (topic != null) { + ExtendedLongCounterBuilder eb = (ExtendedLongCounterBuilder) builder; + eb.setAttributesAdvice(getDefaultAggregationLabels(attributes)); + + attributes = getTopicAttributes(topic, attributes); + } + + this.counter = builder.build(); this.attributes = attributes; } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/InstrumentProvider.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/InstrumentProvider.java index b3f0566a1932f..1c2604b3f0e46 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/InstrumentProvider.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/InstrumentProvider.java @@ -1,19 +1,35 @@ +/* + * 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.client.impl.metrics; import io.opentelemetry.api.GlobalOpenTelemetry; import io.opentelemetry.api.OpenTelemetry; import io.opentelemetry.api.common.Attributes; -import io.opentelemetry.api.common.AttributesBuilder; import io.opentelemetry.api.metrics.Meter; +import java.util.Optional; import org.apache.pulsar.PulsarVersion; -import org.apache.pulsar.client.api.MetricsCardinality; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; -import org.apache.pulsar.common.naming.TopicName; public class InstrumentProvider { private final Meter meter; - private final MetricsCardinality metricsCardinality; public InstrumentProvider(ClientConfigurationData conf) { OpenTelemetry otel = conf.getOpenTelemetry(); @@ -27,45 +43,17 @@ public InstrumentProvider(ClientConfigurationData conf) { .meterBuilder("org.apache.pulsar.client") .setInstrumentationVersion(PulsarVersion.getVersion()) .build(); - this.metricsCardinality = conf.getOpenTelemetryMetricsCardinality(); - } - - public Attributes getAttributes(String topic) { - if (metricsCardinality == MetricsCardinality.None) { - return Attributes.empty(); - } - - AttributesBuilder ab = Attributes.builder(); - TopicName tn = TopicName.get(topic); - - switch (metricsCardinality) { - case Partition: - if (tn.isPartitioned()) { - ab.put("pulsar.partition", tn.getPartitionIndex()); - } - // fallthrough - case Topic: - ab.put("pulsar.topic", tn.getPartitionedTopicName()); - // fallthrough - case Namespace: - ab.put("pulsar.namespace", tn.getNamespace()); - // fallthrough - case Tenant: - ab.put("pulsar.tenant", tn.getTenant()); - } - - return ab.build(); } - public Counter newCounter(String name, Unit unit, String description, Attributes attributes) { - return new Counter(meter, name, unit, description, attributes); + public Counter newCounter(String name, Unit unit, String description, String topic, Attributes attributes) { + return new Counter(meter, name, unit, description, topic, attributes); } - public UpDownCounter newUpDownCounter(String name, Unit unit, String description, Attributes attributes) { - return new UpDownCounter(meter, name, unit, description, attributes); + public UpDownCounter newUpDownCounter(String name, Unit unit, String description, String topic, Attributes attributes) { + return new UpDownCounter(meter, name, unit, description, topic, attributes); } - public LatencyHistogram newLatencyHistogram(String name, String description, Attributes attributes) { - return new LatencyHistogram(meter, name, description, attributes); + public LatencyHistogram newLatencyHistogram(String name, String description, String topic, Attributes attributes) { + return new LatencyHistogram(meter, name, description, topic, attributes); } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java index c23acac277898..f0ec84ff44a6f 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java @@ -1,9 +1,32 @@ +/* + * 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.client.impl.metrics; +import static org.apache.pulsar.client.impl.metrics.MetricsUtil.getDefaultAggregationLabels; +import static org.apache.pulsar.client.impl.metrics.MetricsUtil.getTopicAttributes; import com.google.common.collect.Lists; import io.opentelemetry.api.common.Attributes; import io.opentelemetry.api.metrics.DoubleHistogram; +import io.opentelemetry.api.metrics.DoubleHistogramBuilder; import io.opentelemetry.api.metrics.Meter; +import io.opentelemetry.extension.incubator.metrics.ExtendedDoubleHistogramBuilder; import java.util.List; import java.util.concurrent.TimeUnit; @@ -20,12 +43,17 @@ public class LatencyHistogram { private final Attributes failedAttributes; private final DoubleHistogram histogram; - LatencyHistogram(Meter meter, String name, String description, Attributes attributes) { - histogram = meter.histogramBuilder(name) + LatencyHistogram(Meter meter, String name, String description, String topic, Attributes attributes) { + DoubleHistogramBuilder builder = meter.histogramBuilder(name) .setDescription(description) .setUnit(Unit.Seconds.toString()) - .setExplicitBucketBoundariesAdvice(latencyHistogramBuckets) - .build(); + .setExplicitBucketBoundariesAdvice(latencyHistogramBuckets); + + if (topic != null) { + ExtendedDoubleHistogramBuilder eb = (ExtendedDoubleHistogramBuilder) builder; + eb.setAttributesAdvice(getDefaultAggregationLabels(attributes.toBuilder().put("success", true).build())); + attributes = getTopicAttributes(topic, attributes); + } successAttributes = attributes.toBuilder() .put("success", true) @@ -33,6 +61,7 @@ public class LatencyHistogram { failedAttributes = attributes.toBuilder() .put("success", false) .build(); + this.histogram = builder.build(); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/MetricsUtil.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/MetricsUtil.java new file mode 100644 index 0000000000000..1bc1847612230 --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/MetricsUtil.java @@ -0,0 +1,59 @@ +/* + * 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.client.impl.metrics; + +import com.google.common.collect.Lists; +import io.opentelemetry.api.common.AttributeKey; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.common.AttributesBuilder; +import java.util.ArrayList; +import java.util.List; +import lombok.experimental.UtilityClass; +import org.apache.pulsar.common.naming.TopicName; + +@UtilityClass +public class MetricsUtil { + + // By default, advice to use namespace level aggregation only + private static final List> DEFAULT_AGGREGATION_LABELS = Lists.newArrayList( + AttributeKey.stringKey("pulsar.tenant"), + AttributeKey.stringKey("pulsar.namespace") + ); + + static List> getDefaultAggregationLabels(Attributes attrs) { + List> res = new ArrayList<>(); + res.addAll(DEFAULT_AGGREGATION_LABELS); + res.addAll(attrs.asMap().keySet()); + return res; + } + + static Attributes getTopicAttributes(String topic, Attributes baseAttributes) { + TopicName tn = TopicName.get(topic); + + AttributesBuilder ab = Attributes.builder(); + if (tn.isPartitioned()) { + ab.put("pulsar.partition", tn.getPartitionIndex()); + } + ab.put("pulsar.topic", tn.getPartitionedTopicName()); + ab.put("pulsar.namespace", tn.getNamespace()); + ab.put("pulsar.tenant", tn.getTenant()); + return ab.build(); + } +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/UpDownCounter.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/UpDownCounter.java index 6a5abd86e96ae..50a3a654b9258 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/UpDownCounter.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/UpDownCounter.java @@ -1,19 +1,31 @@ package org.apache.pulsar.client.impl.metrics; +import static org.apache.pulsar.client.impl.metrics.MetricsUtil.getDefaultAggregationLabels; +import static org.apache.pulsar.client.impl.metrics.MetricsUtil.getTopicAttributes; import io.opentelemetry.api.common.Attributes; import io.opentelemetry.api.metrics.LongUpDownCounter; +import io.opentelemetry.api.metrics.LongUpDownCounterBuilder; import io.opentelemetry.api.metrics.Meter; +import io.opentelemetry.extension.incubator.metrics.ExtendedLongUpDownCounterBuilder; public class UpDownCounter { private final LongUpDownCounter counter; private final Attributes attributes; - UpDownCounter(Meter meter, String name, Unit unit, String description, Attributes attributes) { - counter = meter.upDownCounterBuilder(name) + UpDownCounter(Meter meter, String name, Unit unit, String description, String topic, Attributes attributes) { + LongUpDownCounterBuilder builder = meter.upDownCounterBuilder(name) .setDescription(description) - .setUnit(unit.toString()) - .build(); + .setUnit(unit.toString()); + + if (topic != null) { + ExtendedLongUpDownCounterBuilder eb = (ExtendedLongUpDownCounterBuilder) builder; + eb.setAttributesAdvice(getDefaultAggregationLabels(attributes)); + + attributes = getTopicAttributes(topic, attributes); + } + + this.counter = builder.build(); this.attributes = attributes; } diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerfClientUtils.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerfClientUtils.java index 36396465d419a..b6b3d805edc75 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerfClientUtils.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerfClientUtils.java @@ -29,7 +29,6 @@ import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminBuilder; import org.apache.pulsar.client.api.ClientBuilder; -import org.apache.pulsar.client.api.MetricsCardinality; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.SizeUnit; @@ -80,8 +79,7 @@ public static ClientBuilder createClientBuilderFromArguments(PerformanceBaseArgu .maxLookupRequests(arguments.maxLookupRequest) .proxyServiceUrl(arguments.proxyServiceURL, arguments.proxyProtocol) .openTelemetry(AutoConfiguredOpenTelemetrySdk.builder() - .build().getOpenTelemetrySdk()) - .openTelemetryMetricsCardinality(MetricsCardinality.Topic); + .build().getOpenTelemetrySdk()); if (isNotBlank(arguments.authPluginClassName)) { clientBuilder.authentication(arguments.authPluginClassName, arguments.authParams); From 252196e328e5c56da30987859185d44cfefda2a6 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Mon, 4 Mar 2024 22:20:46 -0800 Subject: [PATCH 04/22] Fixed tests code --- .../impl/BatchMessageContainerImpl.java | 6 ++--- .../pulsar/client/impl/ProducerImpl.java | 27 +++++++++---------- .../client/impl/metrics/LatencyHistogram.java | 10 ++++++- .../client/impl/OpSendMsgQueueTest.java | 3 ++- .../pulsar/client/impl/ProducerImplTest.java | 2 ++ .../client/impl/PulsarClientImplTest.java | 5 ++-- 6 files changed, 32 insertions(+), 21 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageContainerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageContainerImpl.java index e33719c14c48d..88a15c6f11b04 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageContainerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageContainerImpl.java @@ -263,8 +263,8 @@ public OpSendMsg createOpSendMsg() throws IOException { // Because when invoke `ProducerImpl.processOpSendMsg` on flush, // if `op.msg != null && isBatchMessagingEnabled()` checks true, it will call `batchMessageAndSend` to flush // messageContainers before publishing this one-batch message. - op = OpSendMsg.create(producer, messages, cmd, messageMetadata.getSequenceId(), firstCallback, - batchAllocatedSizeBytes); + op = OpSendMsg.create(producer.rpcLatencyHistogram, messages, cmd, messageMetadata.getSequenceId(), + firstCallback, batchAllocatedSizeBytes); // NumMessagesInBatch and BatchSizeByte will not be serialized to the binary cmd. It's just useful for the // ProducerStats @@ -314,7 +314,7 @@ public OpSendMsg createOpSendMsg() throws IOException { messageMetadata.getUncompressedSize(), encryptedPayload.readableBytes()); } - OpSendMsg op = OpSendMsg.create(producer, messages, cmd, messageMetadata.getSequenceId(), + OpSendMsg op = OpSendMsg.create(producer.rpcLatencyHistogram, messages, cmd, messageMetadata.getSequenceId(), messageMetadata.getHighestSequenceId(), firstCallback, batchAllocatedSizeBytes); op.setNumMessagesInBatch(numMessagesInBatch); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java index 17da214626d81..feda0b4e56917 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java @@ -40,7 +40,6 @@ import io.netty.util.Timeout; import io.netty.util.TimerTask; import io.netty.util.concurrent.ScheduledFuture; -import io.opentelemetry.api.common.AttributeKey; import io.opentelemetry.api.common.Attributes; import java.io.IOException; import java.net.URI; @@ -179,7 +178,7 @@ public class ProducerImpl extends ProducerBase implements TimerTask, Conne private boolean errorState; private final LatencyHistogram latencyHistogram; - private final LatencyHistogram rpcLatencyHistogram; + final LatencyHistogram rpcLatencyHistogram; private final Counter publishedBytesCounter; private final UpDownCounter pendingMessagesCounter; private final UpDownCounter pendingBytesCounter; @@ -745,9 +744,9 @@ private void serializeAndSendMessage(MessageImpl msg, if (msg.getSchemaState() == MessageImpl.SchemaState.Ready) { ByteBufPair cmd = sendMessage(producerId, sequenceId, numMessages, messageId, msgMetadata, encryptedPayload); - op = OpSendMsg.create(this, msg, cmd, sequenceId, callback); + op = OpSendMsg.create(rpcLatencyHistogram, msg, cmd, sequenceId, callback); } else { - op = OpSendMsg.create(this, msg, null, sequenceId, callback); + op = OpSendMsg.create(rpcLatencyHistogram, msg, null, sequenceId, callback); final MessageMetadata finalMsgMetadata = msgMetadata; op.rePopulate = () -> { if (msgMetadata.hasChunkId()) { @@ -1455,7 +1454,7 @@ public ReferenceCounted touch(Object hint) { } protected static final class OpSendMsg { - ProducerImpl producer; + LatencyHistogram rpcLatencyHistogram; MessageImpl msg; List> msgs; ByteBufPair cmd; @@ -1475,7 +1474,7 @@ protected static final class OpSendMsg { int chunkId = -1; void initialize() { - producer = null; + rpcLatencyHistogram = null; msg = null; msgs = null; cmd = null; @@ -1495,11 +1494,11 @@ void initialize() { chunkedMessageCtx = null; } - static OpSendMsg create(ProducerImpl producer, MessageImpl msg, ByteBufPair cmd, long sequenceId, + static OpSendMsg create(LatencyHistogram rpcLatencyHistogram, MessageImpl msg, ByteBufPair cmd, long sequenceId, SendCallback callback) { OpSendMsg op = RECYCLER.get(); op.initialize(); - op.producer = producer; + op.rpcLatencyHistogram = rpcLatencyHistogram; op.msg = msg; op.cmd = cmd; op.callback = callback; @@ -1509,12 +1508,12 @@ static OpSendMsg create(ProducerImpl producer, MessageImpl msg, ByteBufPai return op; } - static OpSendMsg create(ProducerImpl producer, List> msgs, ByteBufPair cmd, long sequenceId, + static OpSendMsg create(LatencyHistogram rpcLatencyHistogram, List> msgs, ByteBufPair cmd, long sequenceId, SendCallback callback, int batchAllocatedSize) { OpSendMsg op = RECYCLER.get(); op.initialize(); - op.producer = producer; + op.rpcLatencyHistogram = rpcLatencyHistogram; op.msgs = msgs; op.cmd = cmd; op.callback = callback; @@ -1528,12 +1527,12 @@ static OpSendMsg create(ProducerImpl producer, List> msgs, Byt return op; } - static OpSendMsg create(ProducerImpl producer, List> msgs, ByteBufPair cmd, + static OpSendMsg create(LatencyHistogram rpcLatencyHistogram, List> msgs, ByteBufPair cmd, long lowestSequenceId, long highestSequenceId, SendCallback callback, int batchAllocatedSize) { OpSendMsg op = RECYCLER.get(); op.initialize(); - op.producer = producer; + op.rpcLatencyHistogram = rpcLatencyHistogram; op.msgs = msgs; op.cmd = cmd; op.callback = callback; @@ -1585,9 +1584,9 @@ void sendComplete(final Exception e) { } if (e == null) { - producer.rpcLatencyHistogram.recordSuccess(now - this.lastSentAt); + rpcLatencyHistogram.recordSuccess(now - this.lastSentAt); } else { - producer.rpcLatencyHistogram.recordFailure(now - this.lastSentAt); + rpcLatencyHistogram.recordFailure(now - this.lastSentAt); } callback.sendComplete(finalEx); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java index f0ec84ff44a6f..688709cacc16f 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java @@ -32,12 +32,20 @@ public class LatencyHistogram { + // Used for tests + public final static LatencyHistogram NOOP = new LatencyHistogram(null, null, null, null, null) { + public void recordSuccess(long latencyNanos) { + } + + public void recordFailure(long latencyNanos) { + } + }; + private static final List latencyHistogramBuckets = Lists.newArrayList(.0005, .001, .0025, .005, .01, .025, .05, .1, .25, .5, 1.0, 2.5, 5.0, 10.0, 30.0, 60.0); private static final double NANOS = TimeUnit.SECONDS.toNanos(1); - private final Attributes successAttributes; private final Attributes failedAttributes; diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/OpSendMsgQueueTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/OpSendMsgQueueTest.java index 2db23782640eb..efcc06bede3e1 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/OpSendMsgQueueTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/OpSendMsgQueueTest.java @@ -23,6 +23,7 @@ import static org.testng.Assert.assertEquals; import com.google.common.collect.Lists; import java.util.Arrays; +import org.apache.pulsar.client.impl.metrics.LatencyHistogram; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; @@ -39,7 +40,7 @@ public void createMockMessage() { } private ProducerImpl.OpSendMsg createDummyOpSendMsg() { - return ProducerImpl.OpSendMsg.create(message, null, 0L, null); + return ProducerImpl.OpSendMsg.create(LatencyHistogram.NOOP, message, null, 0L, null); } @Test diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerImplTest.java index 6fcedc3f94de7..f9df63759394a 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerImplTest.java @@ -27,6 +27,7 @@ import static org.testng.Assert.assertTrue; import java.nio.ByteBuffer; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.metrics.LatencyHistogram; import org.apache.pulsar.common.api.proto.MessageMetadata; import org.mockito.Mockito; import org.testng.annotations.Test; @@ -42,6 +43,7 @@ public void testChunkedMessageCtxDeallocate() { for (int i = 0; i < totalChunks; i++) { ProducerImpl.OpSendMsg opSendMsg = ProducerImpl.OpSendMsg.create( + LatencyHistogram.NOOP, MessageImpl.create(new MessageMetadata(), ByteBuffer.allocate(0), Schema.STRING, null), null, 0, null); opSendMsg.chunkedMessageCtx = ctx; diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java index c96443c1e2f9f..3d1da9696f3c2 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java @@ -53,6 +53,7 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.client.util.ExecutorProvider; import org.apache.pulsar.client.util.ScheduledExecutorProvider; import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; @@ -180,7 +181,7 @@ public void testInitializeWithTimer() throws PulsarClientException { ClientConfigurationData conf = new ClientConfigurationData(); @Cleanup("shutdownGracefully") EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(1, false, new DefaultThreadFactory("test")); - ConnectionPool pool = Mockito.spy(new ConnectionPool(conf, eventLoop)); + ConnectionPool pool = Mockito.spy(new ConnectionPool(new InstrumentProvider(conf), conf, eventLoop)); conf.setServiceUrl("pulsar://localhost:6650"); HashedWheelTimer timer = new HashedWheelTimer(); @@ -205,7 +206,7 @@ public void testResourceCleanup() throws Exception { ClientConfigurationData conf = new ClientConfigurationData(); conf.setServiceUrl(""); initializeEventLoopGroup(conf); - try (ConnectionPool connectionPool = new ConnectionPool(conf, eventLoopGroup)) { + try (ConnectionPool connectionPool = new ConnectionPool(new InstrumentProvider(conf), conf, eventLoopGroup)) { assertThrows(() -> new PulsarClientImpl(conf, eventLoopGroup, connectionPool)); } finally { // Externally passed eventLoopGroup should not be shutdown. From 445231c36ea8e298ba7388243c35a4dab0343ecd Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 5 Mar 2024 10:16:07 -0800 Subject: [PATCH 05/22] Ensure single-initialization of InstrumentProvider --- .../broker/service/BrokerServiceTest.java | 4 +++- .../service/BrokerServiceThrottlingTest.java | 3 ++- .../service/EnableProxyProtocolTest.java | 5 +++-- .../api/InjectedClientCnxClientBuilder.java | 3 ++- ...ListenersWithInternalListenerNameTest.java | 4 ++-- .../client/impl/ConnectionPoolTest.java | 3 ++- .../client/impl/MessageChecksumTest.java | 3 ++- .../client/impl/MessageChunkingTest.java | 3 ++- .../impl/PatternTopicsConsumerImplTest.java | 3 ++- .../pulsar/client/impl/PulsarTestClient.java | 5 +++-- .../client/impl/BinaryProtoLookupService.java | 8 ++++---- .../apache/pulsar/client/impl/ClientCnx.java | 5 ----- .../pulsar/client/impl/HttpLookupService.java | 8 ++++---- .../pulsar/client/impl/PulsarClientImpl.java | 2 +- .../impl/metrics/InstrumentProvider.java | 6 +++--- .../AcknowledgementsGroupingTrackerTest.java | 3 ++- .../ClientCnxRequestTimeoutQueueTest.java | 3 ++- .../pulsar/client/impl/ClientCnxTest.java | 19 ++++++++++--------- .../client/impl/PulsarClientImplTest.java | 4 ++-- 19 files changed, 51 insertions(+), 43 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java index b6a73274f440b..8ebba5c9aeabd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java @@ -99,6 +99,7 @@ import org.apache.pulsar.client.impl.PulsarServiceNameResolver; import org.apache.pulsar.client.impl.auth.AuthenticationTls; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.api.proto.CommandLookupTopicResponse; import org.apache.pulsar.common.api.proto.CommandPartitionedTopicMetadataResponse; import org.apache.pulsar.common.naming.NamespaceBundle; @@ -997,7 +998,8 @@ public void testLookupThrottlingForClientByClient() throws Exception { // Using an AtomicReference in order to reset a new CountDownLatch AtomicReference latchRef = new AtomicReference<>(); latchRef.set(new CountDownLatch(1)); - try (ConnectionPool pool = new ConnectionPool(conf, eventLoop, () -> new ClientCnx(conf, eventLoop) { + try (ConnectionPool pool = new ConnectionPool(InstrumentProvider.NOOP, conf, eventLoop, + () -> new ClientCnx(InstrumentProvider.NOOP, conf, eventLoop) { @Override protected void handleLookupResponse(CommandLookupTopicResponse lookupResult) { try { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceThrottlingTest.java index b2cfe63e2e5b4..a6b3ad5f6ebcf 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceThrottlingTest.java @@ -45,6 +45,7 @@ import org.apache.pulsar.client.impl.ConnectionPool; import org.apache.pulsar.client.impl.PulsarServiceNameResolver; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.util.netty.EventLoopUtil; import org.testng.annotations.AfterMethod; @@ -159,7 +160,7 @@ public void testLookupThrottlingForClientByBroker() throws Exception { EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(20, false, new DefaultThreadFactory("test-pool", Thread.currentThread().isDaemon())); ExecutorService executor = Executors.newFixedThreadPool(10); - try (ConnectionPool pool = new ConnectionPool(conf, eventLoop)) { + try (ConnectionPool pool = new ConnectionPool(InstrumentProvider.NOOP, conf, eventLoop)) { final int totalConsumers = 20; List> futures = new ArrayList<>(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/EnableProxyProtocolTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/EnableProxyProtocolTest.java index a596e1ed32d6b..725b895fe6e14 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/EnableProxyProtocolTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/EnableProxyProtocolTest.java @@ -31,6 +31,7 @@ import org.apache.pulsar.client.impl.ClientBuilderImpl; import org.apache.pulsar.client.impl.ClientCnx; import org.apache.pulsar.client.impl.PulsarClientImpl; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.policies.data.SubscriptionStats; import org.apache.pulsar.common.policies.data.TopicStats; import org.awaitility.Awaitility; @@ -99,7 +100,7 @@ public void testProxyProtocol() throws Exception { ClientBuilderImpl clientBuilder = (ClientBuilderImpl) PulsarClient.builder().serviceUrl(lookupUrl.toString()); @Cleanup PulsarClientImpl protocolClient = InjectedClientCnxClientBuilder.create(clientBuilder, - (conf, eventLoopGroup) -> new ClientCnx(conf, eventLoopGroup) { + (conf, eventLoopGroup) -> new ClientCnx(InstrumentProvider.NOOP, conf, eventLoopGroup) { public void channelActive(ChannelHandlerContext ctx) throws Exception { byte[] bs = "PROXY TCP4 198.51.100.22 203.0.113.7 35646 80\r\n".getBytes(); ctx.writeAndFlush(Unpooled.copiedBuffer(bs)); @@ -124,7 +125,7 @@ public void testPubSubWhenSlowNetwork() throws Exception { ClientBuilderImpl clientBuilder = (ClientBuilderImpl) PulsarClient.builder().serviceUrl(lookupUrl.toString()); @Cleanup PulsarClientImpl protocolClient = InjectedClientCnxClientBuilder.create(clientBuilder, - (conf, eventLoopGroup) -> new ClientCnx(conf, eventLoopGroup) { + (conf, eventLoopGroup) -> new ClientCnx(InstrumentProvider.NOOP, conf, eventLoopGroup) { public void channelActive(ChannelHandlerContext ctx) throws Exception { Thread task = new Thread(() -> { try { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InjectedClientCnxClientBuilder.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InjectedClientCnxClientBuilder.java index 2a7908242707b..13447e089eab8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InjectedClientCnxClientBuilder.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InjectedClientCnxClientBuilder.java @@ -28,6 +28,7 @@ import org.apache.pulsar.client.impl.ConnectionPool; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.client.util.ExecutorProvider; import org.apache.pulsar.common.util.netty.EventLoopUtil; @@ -42,7 +43,7 @@ public static PulsarClientImpl create(final ClientBuilderImpl clientBuilder, EventLoopUtil.newEventLoopGroup(conf.getNumIoThreads(), conf.isEnableBusyWait(), threadFactory); // Inject into ClientCnx. - ConnectionPool pool = new ConnectionPool(conf, eventLoopGroup, + ConnectionPool pool = new ConnectionPool(InstrumentProvider.NOOP, conf, eventLoopGroup, () -> clientCnxFactory.generate(conf, eventLoopGroup)); return new InjectedClientCnxPulsarClientImpl(conf, eventLoopGroup, pool); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PulsarMultiListenersWithInternalListenerNameTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PulsarMultiListenersWithInternalListenerNameTest.java index e420a2949882f..ced7fccd6b7ed 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PulsarMultiListenersWithInternalListenerNameTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PulsarMultiListenersWithInternalListenerNameTest.java @@ -139,7 +139,7 @@ private void doFindBrokerWithListenerName(boolean useHttp) throws Exception { conf.setMaxLookupRedirects(10); @Cleanup - LookupService lookupService = useHttp ? new HttpLookupService(new InstrumentProvider(new ClientConfigurationData()), conf, eventExecutors) : + LookupService lookupService = useHttp ? new HttpLookupService(InstrumentProvider.NOOP, conf, eventExecutors) : new BinaryProtoLookupService((PulsarClientImpl) this.pulsarClient, lookupUrl.toString(), "internal", false, this.executorService); TopicName topicName = TopicName.get("persistent://public/default/test"); @@ -174,7 +174,7 @@ public void testHttpLookupRedirect() throws Exception { conf.setMaxLookupRedirects(10); @Cleanup - HttpLookupService lookupService = new HttpLookupService(new InstrumentProvider(new ClientConfigurationData()), conf, eventExecutors); + HttpLookupService lookupService = new HttpLookupService(InstrumentProvider.NOOP, conf, eventExecutors); NamespaceService namespaceService = pulsar.getNamespaceService(); LookupResult lookupResult = new LookupResult(pulsar.getWebServiceAddress(), null, diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionPoolTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionPoolTest.java index 79ffada4a90c8..553aad294dd66 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionPoolTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionPoolTest.java @@ -35,6 +35,7 @@ import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.api.proto.CommandCloseProducer; import org.apache.pulsar.common.util.netty.EventLoopUtil; import org.awaitility.Awaitility; @@ -234,7 +235,7 @@ protected void doResolveAll(SocketAddress socketAddress, Promise promise) throws }; ConnectionPool pool = spyWithClassAndConstructorArgs(ConnectionPool.class, conf, eventLoop, - (Supplier) () -> new ClientCnx(conf, eventLoop), Optional.of(resolver)); + (Supplier) () -> new ClientCnx(InstrumentProvider.NOOP, conf, eventLoop), Optional.of(resolver)); ClientCnx cnx = pool.getConnection( diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChecksumTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChecksumTest.java index 515b34db8509d..0b25e3409563a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChecksumTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChecksumTest.java @@ -37,6 +37,7 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.TypedMessageBuilder; import org.apache.pulsar.client.impl.ProducerImpl.OpSendMsg; +import org.apache.pulsar.client.impl.metrics.LatencyHistogram; import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.api.proto.ProtocolVersion; import org.apache.pulsar.common.protocol.ByteBufPair; @@ -233,7 +234,7 @@ public void testTamperingMessageIsDetected() throws Exception { // WHEN // protocol message is created with checksum ByteBufPair cmd = Commands.newSend(1, 1, 1, ChecksumType.Crc32c, msgMetadata, payload); - OpSendMsg op = OpSendMsg.create((MessageImpl) msgBuilder.getMessage(), cmd, 1, null); + OpSendMsg op = OpSendMsg.create(LatencyHistogram.NOOP, (MessageImpl) msgBuilder.getMessage(), cmd, 1, null); // THEN // the checksum validation passes diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChunkingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChunkingTest.java index 6686edd2b67d2..da359a6aeb9c5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChunkingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChunkingTest.java @@ -56,6 +56,7 @@ import org.apache.pulsar.client.api.SizeUnit; import org.apache.pulsar.client.impl.MessageImpl.SchemaState; import org.apache.pulsar.client.impl.ProducerImpl.OpSendMsg; +import org.apache.pulsar.client.impl.metrics.LatencyHistogram; import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.policies.data.PublisherStats; import org.apache.pulsar.common.protocol.ByteBufPair; @@ -499,7 +500,7 @@ public void testExpireIncompleteChunkMessage() throws Exception{ ByteBufPair cmd = Commands.newSend(producerId, 1, 1, ChecksumType.Crc32c, msgMetadata, payload); MessageImpl msgImpl = ((MessageImpl) msg.getMessage()); msgImpl.setSchemaState(SchemaState.Ready); - OpSendMsg op = OpSendMsg.create(msgImpl, cmd, 1, null); + OpSendMsg op = OpSendMsg.create(LatencyHistogram.NOOP, msgImpl, cmd, 1, null); producer.processOpSendMsg(op); retryStrategically((test) -> { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java index 7707abafde8de..94d78e418ab87 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java @@ -49,6 +49,7 @@ import org.apache.pulsar.client.api.RegexSubscriptionMode; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.api.proto.BaseCommand; import org.apache.pulsar.common.api.proto.CommandWatchTopicListSuccess; import org.apache.pulsar.common.naming.NamespaceName; @@ -811,7 +812,7 @@ public void testPreciseRegexpSubscribeDisabledTopicWatcher(boolean partitioned) private PulsarClient createDelayWatchTopicsClient() throws Exception { ClientBuilderImpl clientBuilder = (ClientBuilderImpl) PulsarClient.builder().serviceUrl(lookupUrl.toString()); return InjectedClientCnxClientBuilder.create(clientBuilder, - (conf, eventLoopGroup) -> new ClientCnx(conf, eventLoopGroup) { + (conf, eventLoopGroup) -> new ClientCnx(InstrumentProvider.NOOP, conf, eventLoopGroup) { public CompletableFuture newWatchTopicList( BaseCommand command, long requestId) { // Inject 2 seconds delay when sending command New Watch Topics. diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PulsarTestClient.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PulsarTestClient.java index ab273913fde29..8a79eb502439f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PulsarTestClient.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PulsarTestClient.java @@ -35,6 +35,7 @@ import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.util.netty.EventLoopUtil; import org.awaitility.Awaitility; @@ -79,7 +80,7 @@ public static PulsarTestClient create(ClientBuilder clientBuilder) throws Pulsar new DefaultThreadFactory("pulsar-test-client-io", Thread.currentThread().isDaemon())); AtomicReference> clientCnxSupplierReference = new AtomicReference<>(); - ConnectionPool connectionPool = new ConnectionPool(clientConfigurationData, eventLoopGroup, + ConnectionPool connectionPool = new ConnectionPool(InstrumentProvider.NOOP, clientConfigurationData, eventLoopGroup, () -> clientCnxSupplierReference.get().get()); return new PulsarTestClient(clientConfigurationData, eventLoopGroup, connectionPool, @@ -101,7 +102,7 @@ private PulsarTestClient(ClientConfigurationData conf, EventLoopGroup eventLoopG * @return new ClientCnx instance */ protected ClientCnx createClientCnx() { - return new ClientCnx(conf, eventLoopGroup) { + return new ClientCnx(InstrumentProvider.NOOP, conf, eventLoopGroup) { @Override public int getRemoteEndpointProtocolVersion() { return overrideRemoteEndpointProtocolVersion != 0 diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java index 6b379b1625076..3d41265ce72f5 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java @@ -95,16 +95,16 @@ public BinaryProtoLookupService(PulsarClientImpl client, Attributes attrs = Attributes.of(AttributeKey.stringKey("pulsar.lookup.transport-type"), "binary"); - histoGetBroker = client.instrumentProvider().newLatencyHistogram("pulsar.client.lookup", + histoGetBroker = client.instrumentProvider().newLatencyHistogram("pulsar.client.lookup.duration", "Duration of lookup operations", null, attrs.toBuilder().put("pulsar.lookup.type", "topic").build()); - histoGetTopicMetadata = client.instrumentProvider().newLatencyHistogram("pulsar.client.lookup", + histoGetTopicMetadata = client.instrumentProvider().newLatencyHistogram("pulsar.client.lookup.duration", "Duration of lookup operations", null, attrs.toBuilder().put("pulsar.lookup.type", "metadata").build()); - histoGetSchema = client.instrumentProvider().newLatencyHistogram("pulsar.client.lookup", + histoGetSchema = client.instrumentProvider().newLatencyHistogram("pulsar.client.lookup.duration", "Duration of lookup operations", null, attrs.toBuilder().put("pulsar.lookup.type", "schema").build()); - histoListTopics = client.instrumentProvider().newLatencyHistogram("pulsar.client.lookup", + histoListTopics = client.instrumentProvider().newLatencyHistogram("pulsar.client.lookup.duration", "Duration of lookup operations", null, attrs.toBuilder().put("pulsar.lookup.type", "list-topics").build()); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java index f394ffa190e88..0cdc388b5d6bc 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java @@ -244,11 +244,6 @@ String getDescription() { } } - - public ClientCnx(ClientConfigurationData conf, EventLoopGroup eventLoopGroup) { - this(new InstrumentProvider(conf), conf, eventLoopGroup, Commands.getCurrentProtocolVersion()); - } - public ClientCnx(InstrumentProvider instrumentProvider, ClientConfigurationData conf, EventLoopGroup eventLoopGroup) { this(instrumentProvider, conf, eventLoopGroup, Commands.getCurrentProtocolVersion()); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java index c20461f7e8c8d..e6999b777b88f 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java @@ -77,16 +77,16 @@ public HttpLookupService(InstrumentProvider instrumentProvider, ClientConfigurat Attributes attrs = Attributes.of(AttributeKey.stringKey("pulsar.lookup.transport-type"), "binary"); - histoGetBroker = instrumentProvider.newLatencyHistogram("pulsar.client.lookup", + histoGetBroker = instrumentProvider.newLatencyHistogram("pulsar.client.lookup.duration", "Duration of lookup operations", null, attrs.toBuilder().put("pulsar.lookup.type", "topic").build()); - histoGetTopicMetadata = instrumentProvider.newLatencyHistogram("pulsar.client.lookup", + histoGetTopicMetadata = instrumentProvider.newLatencyHistogram("pulsar.client.lookup.duration", "Duration of lookup operations", null, attrs.toBuilder().put("pulsar.lookup.type", "metadata").build()); - histoGetSchema = instrumentProvider.newLatencyHistogram("pulsar.client.lookup", + histoGetSchema = instrumentProvider.newLatencyHistogram("pulsar.client.lookup.duration", "Duration of lookup operations", null, attrs.toBuilder().put("pulsar.lookup.type", "schema").build()); - histoListTopics = instrumentProvider.newLatencyHistogram("pulsar.client.lookup", + histoListTopics = instrumentProvider.newLatencyHistogram("pulsar.client.lookup.duration", "Duration of lookup operations", null, attrs.toBuilder().put("pulsar.lookup.type", "list-topics").build()); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index 0b75098f9baec..22ef86fae47f7 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -182,7 +182,7 @@ private PulsarClientImpl(ClientConfigurationData conf, EventLoopGroup eventLoopG ExecutorProvider internalExecutorProvider, ScheduledExecutorProvider scheduledExecutorProvider) throws PulsarClientException { - this.instrumentProvider = new InstrumentProvider(conf); + this.instrumentProvider = new InstrumentProvider(conf.getOpenTelemetry()); EventLoopGroup eventLoopGroupReference = null; ConnectionPool connectionPoolReference = null; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/InstrumentProvider.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/InstrumentProvider.java index 1c2604b3f0e46..2b99787d59b4b 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/InstrumentProvider.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/InstrumentProvider.java @@ -29,16 +29,16 @@ public class InstrumentProvider { + public static final InstrumentProvider NOOP = new InstrumentProvider(OpenTelemetry.noop()); + private final Meter meter; - public InstrumentProvider(ClientConfigurationData conf) { - OpenTelemetry otel = conf.getOpenTelemetry(); + public InstrumentProvider(OpenTelemetry otel) { if (otel == null) { // By default, metrics are disabled, unless the OTel java agent is configured. // This allows to enable metrics without any code change. otel = GlobalOpenTelemetry.get(); } - this.meter = otel.getMeterProvider() .meterBuilder("org.apache.pulsar.client") .setInstrumentationVersion(PulsarVersion.getVersion()) diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AcknowledgementsGroupingTrackerTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AcknowledgementsGroupingTrackerTest.java index b57a3e3d7439e..514e3dde14070 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AcknowledgementsGroupingTrackerTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AcknowledgementsGroupingTrackerTest.java @@ -41,6 +41,7 @@ import org.apache.pulsar.client.api.MessageIdAdv; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.client.util.TimedCompletableFuture; import org.apache.pulsar.common.api.proto.CommandAck.AckType; import org.apache.pulsar.common.util.collections.ConcurrentBitSetRecyclable; @@ -423,7 +424,7 @@ public void testDoIndividualBatchAckAsync() throws Exception{ public class ClientCnxTest extends ClientCnx { public ClientCnxTest(ClientConfigurationData conf, EventLoopGroup eventLoopGroup) { - super(conf, eventLoopGroup); + super(InstrumentProvider.NOOP, conf, eventLoopGroup); } @Override diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientCnxRequestTimeoutQueueTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientCnxRequestTimeoutQueueTest.java index ca6114d2ed823..d573229fddefa 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientCnxRequestTimeoutQueueTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientCnxRequestTimeoutQueueTest.java @@ -26,6 +26,7 @@ import io.netty.util.concurrent.DefaultThreadFactory; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.client.util.TimedCompletableFuture; import org.apache.pulsar.common.util.netty.EventLoopUtil; @@ -60,7 +61,7 @@ void setupClientCnx() throws Exception { ClientConfigurationData conf = new ClientConfigurationData(); conf.setKeepAliveIntervalSeconds(0); conf.setOperationTimeoutMs(1); - cnx = new ClientCnx(conf, eventLoop); + cnx = new ClientCnx(InstrumentProvider.NOOP, conf, eventLoop); ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); Channel channel = mock(Channel.class); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientCnxTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientCnxTest.java index 4f657da82b289..bc1d940c76bbf 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientCnxTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientCnxTest.java @@ -42,6 +42,7 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.PulsarClientException.BrokerMetadataException; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.api.proto.CommandCloseConsumer; import org.apache.pulsar.common.api.proto.CommandCloseProducer; import org.apache.pulsar.common.api.proto.CommandConnected; @@ -63,7 +64,7 @@ public void testClientCnxTimeout() throws Exception { ClientConfigurationData conf = new ClientConfigurationData(); conf.setOperationTimeoutMs(10); conf.setKeepAliveIntervalSeconds(0); - ClientCnx cnx = new ClientCnx(conf, eventLoop); + ClientCnx cnx = new ClientCnx(InstrumentProvider.NOOP, conf, eventLoop); ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); Channel channel = mock(Channel.class); @@ -89,7 +90,7 @@ public void testPendingLookupRequestSemaphore() throws Exception { ClientConfigurationData conf = new ClientConfigurationData(); conf.setOperationTimeoutMs(10_000); conf.setKeepAliveIntervalSeconds(0); - ClientCnx cnx = new ClientCnx(conf, eventLoop); + ClientCnx cnx = new ClientCnx(InstrumentProvider.NOOP, conf, eventLoop); ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); Channel channel = mock(Channel.class); @@ -127,7 +128,7 @@ public void testPendingLookupRequestSemaphoreServiceNotReady() throws Exception ClientConfigurationData conf = new ClientConfigurationData(); conf.setOperationTimeoutMs(10_000); conf.setKeepAliveIntervalSeconds(0); - ClientCnx cnx = new ClientCnx(conf, eventLoop); + ClientCnx cnx = new ClientCnx(InstrumentProvider.NOOP, conf, eventLoop); ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); Channel channel = mock(Channel.class); @@ -170,7 +171,7 @@ public void testPendingWaitingLookupRequestSemaphore() throws Exception { ClientConfigurationData conf = new ClientConfigurationData(); conf.setOperationTimeoutMs(10_000); conf.setKeepAliveIntervalSeconds(0); - ClientCnx cnx = new ClientCnx(conf, eventLoop); + ClientCnx cnx = new ClientCnx(InstrumentProvider.NOOP, conf, eventLoop); ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); Channel channel = mock(Channel.class); @@ -196,7 +197,7 @@ public void testReceiveErrorAtSendConnectFrameState() throws Exception { EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(1, false, threadFactory); ClientConfigurationData conf = new ClientConfigurationData(); conf.setOperationTimeoutMs(10); - ClientCnx cnx = new ClientCnx(conf, eventLoop); + ClientCnx cnx = new ClientCnx(InstrumentProvider.NOOP, conf, eventLoop); ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); Channel channel = mock(Channel.class); @@ -230,7 +231,7 @@ public void testGetLastMessageIdWithError() throws Exception { ThreadFactory threadFactory = new DefaultThreadFactory("testReceiveErrorAtSendConnectFrameState"); EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(1, false, threadFactory); ClientConfigurationData conf = new ClientConfigurationData(); - ClientCnx cnx = new ClientCnx(conf, eventLoop); + ClientCnx cnx = new ClientCnx(InstrumentProvider.NOOP, conf, eventLoop); ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); Channel channel = mock(Channel.class); @@ -276,7 +277,7 @@ public void testHandleCloseConsumer() { ThreadFactory threadFactory = new DefaultThreadFactory("testHandleCloseConsumer"); EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(1, false, threadFactory); ClientConfigurationData conf = new ClientConfigurationData(); - ClientCnx cnx = new ClientCnx(conf, eventLoop); + ClientCnx cnx = new ClientCnx(InstrumentProvider.NOOP, conf, eventLoop); long consumerId = 1; PulsarClientImpl pulsarClient = mock(PulsarClientImpl.class); @@ -300,7 +301,7 @@ public void testHandleCloseProducer() { ThreadFactory threadFactory = new DefaultThreadFactory("testHandleCloseProducer"); EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(1, false, threadFactory); ClientConfigurationData conf = new ClientConfigurationData(); - ClientCnx cnx = new ClientCnx(conf, eventLoop); + ClientCnx cnx = new ClientCnx(InstrumentProvider.NOOP, conf, eventLoop); long producerId = 1; PulsarClientImpl pulsarClient = mock(PulsarClientImpl.class); @@ -393,7 +394,7 @@ private void withConnection(String testName, Consumer test) { try { ClientConfigurationData conf = new ClientConfigurationData(); - ClientCnx cnx = new ClientCnx(conf, eventLoop); + ClientCnx cnx = new ClientCnx(InstrumentProvider.NOOP, conf, eventLoop); ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); Channel channel = mock(Channel.class); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java index 3d1da9696f3c2..274b9b4f2d572 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java @@ -181,7 +181,7 @@ public void testInitializeWithTimer() throws PulsarClientException { ClientConfigurationData conf = new ClientConfigurationData(); @Cleanup("shutdownGracefully") EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(1, false, new DefaultThreadFactory("test")); - ConnectionPool pool = Mockito.spy(new ConnectionPool(new InstrumentProvider(conf), conf, eventLoop)); + ConnectionPool pool = Mockito.spy(new ConnectionPool(InstrumentProvider.NOOP, conf, eventLoop)); conf.setServiceUrl("pulsar://localhost:6650"); HashedWheelTimer timer = new HashedWheelTimer(); @@ -206,7 +206,7 @@ public void testResourceCleanup() throws Exception { ClientConfigurationData conf = new ClientConfigurationData(); conf.setServiceUrl(""); initializeEventLoopGroup(conf); - try (ConnectionPool connectionPool = new ConnectionPool(new InstrumentProvider(conf), conf, eventLoopGroup)) { + try (ConnectionPool connectionPool = new ConnectionPool(InstrumentProvider.NOOP, conf, eventLoopGroup)) { assertThrows(() -> new PulsarClientImpl(conf, eventLoopGroup, connectionPool)); } finally { // Externally passed eventLoopGroup should not be shutdown. From e840d06dcd6f6305378f1c7c065b250ab1f5d086 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Wed, 6 Mar 2024 11:26:07 -0800 Subject: [PATCH 06/22] Added unit tests --- pulsar-broker/pom.xml | 5 + .../client/metrics/ClientMetricsTest.java | 333 ++++++++++++++++++ .../client/impl/BinaryProtoLookupService.java | 6 - .../pulsar/client/impl/ConsumerImpl.java | 4 +- .../pulsar/client/impl/ProducerImpl.java | 7 +- .../pulsar/client/impl/metrics/Counter.java | 6 +- .../client/impl/metrics/LatencyHistogram.java | 16 +- .../client/impl/metrics/MetricsUtil.java | 2 +- .../client/impl/metrics/UpDownCounter.java | 11 +- 9 files changed, 369 insertions(+), 21 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index 18da38b43dc25..a822edd240cd8 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -149,6 +149,11 @@ ${project.version} + + io.opentelemetry + opentelemetry-sdk-testing + + ${project.groupId} pulsar-io-batch-discovery-triggerers diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java new file mode 100644 index 0000000000000..32a40c37390c2 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java @@ -0,0 +1,333 @@ +/* + * 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.client.metrics; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; +import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.sdk.OpenTelemetrySdk; +import io.opentelemetry.sdk.metrics.SdkMeterProvider; +import io.opentelemetry.sdk.metrics.data.MetricData; +import io.opentelemetry.sdk.metrics.data.MetricDataType; +import io.opentelemetry.sdk.testing.exporter.InMemoryMetricReader; +import java.util.Map; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; +import lombok.Cleanup; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionType; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +@Test(groups = "broker-api") +public class ClientMetricsTest extends ProducerConsumerBase { + + InMemoryMetricReader reader; + OpenTelemetry otel; + + @BeforeMethod + @Override + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + + this.reader = InMemoryMetricReader.create(); + SdkMeterProvider sdkMeterProvider = SdkMeterProvider.builder() + .registerMetricReader(reader) + .build(); + this.otel = OpenTelemetrySdk.builder().setMeterProvider(sdkMeterProvider).build(); + } + + @AfterMethod(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + private Map collectMetrics() { + Map metrics = new TreeMap<>(); + for (MetricData md : reader.collectAllMetrics()) { + metrics.put(md.getName(), md); + } + return metrics; + } + + private void assertCounterValue(Map metrics, String name, long expectedValue, + Attributes expectedAttributes) { + assertEquals(getCounterValue(metrics, name, expectedAttributes), expectedValue); + } + + private long getCounterValue(Map metrics, String name, + Attributes expectedAttributes) { + MetricData md = metrics.get(name); + assertNotNull(md, "metric not found: " + name); + assertEquals(md.getType(), MetricDataType.LONG_SUM); + + for (var ex : md.getLongSumData().getPoints()) { + if (ex.getAttributes().equals(expectedAttributes)) { + return ex.getValue(); + } + } + + fail("metric attributes not found: " + expectedAttributes); + return -1; + } + + private void assertHistoCountValue(Map metrics, String name, long expectedCount, + Attributes expectedAttributes) { + assertEquals(getHistoCountValue(metrics, name, expectedAttributes), expectedCount); + } + + private long getHistoCountValue(Map metrics, String name, + Attributes expectedAttributes) { + MetricData md = metrics.get(name); + assertNotNull(md, "metric not found: " + name); + assertEquals(md.getType(), MetricDataType.HISTOGRAM); + + for (var ex : md.getHistogramData().getPoints()) { + if (ex.getAttributes().equals(expectedAttributes)) { + return ex.getCount(); + } + } + + fail("metric attributes not found: " + expectedAttributes); + return -1; + } + + @Test + public void testProducerMetrics() throws Exception { + String topic = newTopicName(); + + PulsarClient client = PulsarClient.builder() + .serviceUrl(pulsar.getBrokerServiceUrl()) + .openTelemetry(otel) + .build(); + + Producer producer = client.newProducer(Schema.STRING) + .topic(topic) + .create(); + + for (int i = 0; i < 5; i++) { + producer.send("Hello"); + } + + Attributes nsAttrs = Attributes.builder() + .put("pulsar.tenant", "my-property") + .put("pulsar.namespace", "my-property/my-ns") + .build(); + Attributes nsAttrsSuccess = nsAttrs.toBuilder() + .put("success", true) + .build(); + + var metrics = collectMetrics(); + System.err.println("All metrics: " + metrics.keySet()); + + assertCounterValue(metrics, "pulsar.client.connections.opened", 1, Attributes.empty()); + assertCounterValue(metrics, "pulsar.client.producer.message.pending.count", 0, nsAttrs); + assertCounterValue(metrics, "pulsar.client.producer.message.pending.size", 0, nsAttrs); + + assertHistoCountValue(metrics, "pulsar.client.lookup.duration", 1, + Attributes.builder() + .put("pulsar.lookup.transport-type", "binary") + .put("pulsar.lookup.type", "topic") + .put("success", true) + .build()); + assertHistoCountValue(metrics, "pulsar.client.lookup.duration", 1, + Attributes.builder() + .put("pulsar.lookup.transport-type", "binary") + .put("pulsar.lookup.type", "metadata") + .put("success", true) + .build()); + + assertHistoCountValue(metrics, "pulsar.client.producer.message.send.duration", 5, nsAttrsSuccess); + assertHistoCountValue(metrics, "pulsar.client.producer.rpc.send.duration", 5, nsAttrsSuccess); + assertCounterValue(metrics, "pulsar.client.producer.message.send.size", "hello".length() * 5, nsAttrs); + + + assertCounterValue(metrics, "pulsar.client.producer.opened", 1, nsAttrs); + + producer.close(); + client.close(); + + metrics = collectMetrics(); + assertCounterValue(metrics, "pulsar.client.producer.closed", 1, nsAttrs); + assertCounterValue(metrics, "pulsar.client.connections.closed", 1, Attributes.empty()); + } + + @Test + public void testConnectionsFailedMetrics() throws Exception { + String topic = newTopicName(); + + @Cleanup + PulsarClient client = PulsarClient.builder() + .serviceUrl("pulsar://invalid-pulsar-address:1234") + .operationTimeout(3, TimeUnit.SECONDS) + .openTelemetry(otel) + .build(); + + try { + client.newProducer(Schema.STRING) + .topic(topic) + .create(); + fail("Should have failed the producer creation"); + } catch (Exception e) { + // Expected + } + + var metrics = collectMetrics(); + + assertTrue(getCounterValue(metrics, "pulsar.client.connections.failed", + Attributes.builder().put("pulsar.failure.type", "tcp-failed").build()) >= 1); + } + + @Test + public void testPublishFailedMetrics() throws Exception { + String topic = newTopicName(); + + @Cleanup + PulsarClient client = PulsarClient.builder() + .serviceUrl(admin.getServiceUrl()) + .operationTimeout(3, TimeUnit.SECONDS) + .openTelemetry(otel) + .build(); + + @Cleanup + Producer producer = client.newProducer(Schema.STRING) + .topic(topic) + .sendTimeout(3, TimeUnit.SECONDS) + .create(); + + // Make the client switch to non-existing broker to make publish fail + client.updateServiceUrl("pulsar://invalid-address:6650"); + + + try { + producer.send("Hello"); + fail("Should have failed to publish"); + } catch (Exception e) { + // expected + } + + var metrics = collectMetrics(); + + Attributes nsAttrs = Attributes.builder() + .put("pulsar.tenant", "my-property") + .put("pulsar.namespace", "my-property/my-ns") + .build(); + Attributes nsAttrsFailure = nsAttrs.toBuilder() + .put("success", false) + .build(); + + assertCounterValue(metrics, "pulsar.client.producer.message.pending.count", 0, nsAttrs); + assertCounterValue(metrics, "pulsar.client.producer.message.pending.size", 0, nsAttrs); + assertHistoCountValue(metrics, "pulsar.client.producer.message.send.duration", 1, nsAttrsFailure); + assertHistoCountValue(metrics, "pulsar.client.producer.rpc.send.duration", 1, nsAttrsFailure); + } + + @Test + public void testConsumerMetrics() throws Exception { + String topic = newTopicName(); + + PulsarClient client = PulsarClient.builder() + .serviceUrl(pulsar.getBrokerServiceUrl()) + .openTelemetry(otel) + .build(); + + @Cleanup + Producer producer = client.newProducer(Schema.STRING) + .topic(topic) + .create(); + + Consumer consumer = client.newConsumer(Schema.STRING) + .topic(topic) + .subscriptionName("my-sub") + .ackTimeout(1, TimeUnit.SECONDS) + .subscriptionType(SubscriptionType.Shared) + .subscribe(); + + for (int i = 0; i < 10; i++) { + producer.send("Hello"); + } + + Thread.sleep(1000); + + Attributes nsAttrs = Attributes.builder() + .put("pulsar.tenant", "my-property") + .put("pulsar.namespace", "my-property/my-ns") + .build(); + var metrics = collectMetrics(); + + assertCounterValue(metrics, "pulsar.client.connections.opened", 1, Attributes.empty()); + + assertHistoCountValue(metrics, "pulsar.client.lookup.duration", 2, + Attributes.builder() + .put("pulsar.lookup.transport-type", "binary") + .put("pulsar.lookup.type", "topic") + .put("success", true) + .build()); + assertHistoCountValue(metrics, "pulsar.client.lookup.duration", 2, + Attributes.builder() + .put("pulsar.lookup.transport-type", "binary") + .put("pulsar.lookup.type", "metadata") + .put("success", true) + .build()); + + assertCounterValue(metrics, "pulsar.client.consumer.prefetched.count", 10, nsAttrs); + assertCounterValue(metrics, "pulsar.client.consumer.prefetched.size", "hello".length() * 10, nsAttrs); + assertCounterValue(metrics, "pulsar.client.consumer.opened", 1, nsAttrs); + + Message msg1 = consumer.receive(); + consumer.acknowledge(msg1); + + Message msg2 = consumer.receive(); + consumer.negativeAcknowledge(msg2); + + /* Message msg3 = */ consumer.receive(); + + metrics = collectMetrics(); + assertCounterValue(metrics, "pulsar.client.consumer.prefetched.count", 7, nsAttrs); + assertCounterValue(metrics, "pulsar.client.consumer.prefetched.size", "hello".length() * 7, nsAttrs); + + // Let msg3 to reach ack-timeout + Thread.sleep(3000); + + metrics = collectMetrics(); + assertCounterValue(metrics, "pulsar.client.consumer.prefetched.count", 8, nsAttrs); + assertCounterValue(metrics, "pulsar.client.consumer.prefetched.size", "hello".length() * 8, nsAttrs); + + assertCounterValue(metrics, "pulsar.client.consumer.message.ack", 1, nsAttrs); + assertCounterValue(metrics, "pulsar.client.consumer.message.nack", 1, nsAttrs); + assertCounterValue(metrics, "pulsar.client.consumer.message.ack.timeout", 1, nsAttrs); + + client.close(); + + metrics = collectMetrics(); + assertCounterValue(metrics, "pulsar.client.consumer.closed", 1, nsAttrs); + assertCounterValue(metrics, "pulsar.client.connections.closed", 1, Attributes.empty()); + } +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java index 3d41265ce72f5..3acd4278a621c 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java @@ -159,12 +159,6 @@ public CompletableFuture getPartitionedTopicMetadata(T CompletableFuture newFuture = getPartitionedTopicMetadata(serviceNameResolver.resolveHost(), topicName); newFutureCreated.setValue(newFuture); - newFuture.thenRun(() -> { - histoGetTopicMetadata.recordSuccess(System.nanoTime() - startTime); - }).exceptionally(x -> { - histoGetTopicMetadata.recordFailure(System.nanoTime() - startTime); - return null; - }); return newFuture; }); } finally { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 33623038d271f..4977a3712f071 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -420,8 +420,8 @@ protected ConsumerImpl(PulsarClientImpl client, String topic, ConsumerConfigurat "The number of acknowledged messages", topic, Attributes.empty()); consumerNacksCounter = ip.newCounter("pulsar.client.consumer.message.nack", Unit.Messages, "The number of negatively acknowledged messages", topic, Attributes.empty()); - consumerDlqMessagesCounter = ip.newCounter("pulsar.client.consumer.dlq", Unit.Messages, - "Number of messages sent to DLQ", topic, Attributes.empty()); + consumerDlqMessagesCounter = ip.newCounter("pulsar.client.consumer.message.dlq", Unit.Messages, + "The number of messages sent to DLQ", topic, Attributes.empty()); grabCnx(); consumersOpenedCounter.increment(); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java index feda0b4e56917..a729f3f80258a 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java @@ -291,15 +291,15 @@ public ProducerImpl(PulsarClientImpl client, String topic, ProducerConfiguration "Publish RPC latency experienced internally by the client when sending data to receiving an ack", topic, Attributes.empty()); publishedBytesCounter = ip.newCounter("pulsar.client.producer.message.send.size", - Unit.Bytes, "Bytes published", topic, Attributes.empty()); + Unit.Bytes, "The number of bytes published", topic, Attributes.empty()); pendingMessagesCounter = ip.newUpDownCounter("pulsar.client.producer.message.pending.count", Unit.Messages, "Pending messages for this producer", topic, Attributes.empty()); pendingBytesCounter = ip.newUpDownCounter("pulsar.client.producer.message.pending.size", Unit.Bytes, "Pending bytes for this producer", topic, Attributes.empty()); producersOpenedCounter = ip.newCounter("pulsar.client.producer.opened", Unit.Sessions, - "Counter of producer sessions opened", topic, Attributes.empty()); + "The number of producer sessions opened", topic, Attributes.empty()); producersClosedCounter = ip.newCounter("pulsar.client.producer.closed", Unit.Sessions, - "Counter of producer sessions closed", topic, Attributes.empty()); + "The number of producer sessions closed", topic, Attributes.empty()); this.connectionHandler = new ConnectionHandler(this, new BackoffBuilder() @@ -2145,6 +2145,7 @@ private void failPendingMessages(ClientCnx cnx, PulsarClientException ex) { log.warn("[{}] [{}] Got exception while completing the callback for msg {}:", topic, producerName, op.sequenceId, t); } + client.getMemoryLimitController().releaseMemory(op.uncompressedSize); ReferenceCountUtil.safeRelease(op.cmd); op.recycle(); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Counter.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Counter.java index ab4c3016df303..1e1e27271007e 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Counter.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Counter.java @@ -36,8 +36,10 @@ public class Counter { .setUnit(unit.toString()); if (topic != null) { - ExtendedLongCounterBuilder eb = (ExtendedLongCounterBuilder) builder; - eb.setAttributesAdvice(getDefaultAggregationLabels(attributes)); + if (builder instanceof ExtendedLongCounterBuilder) { + ExtendedLongCounterBuilder eb = (ExtendedLongCounterBuilder) builder; + eb.setAttributesAdvice(getDefaultAggregationLabels(attributes)); + } attributes = getTopicAttributes(topic, attributes); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java index 688709cacc16f..1346a98317610 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java @@ -26,6 +26,7 @@ import io.opentelemetry.api.metrics.DoubleHistogram; import io.opentelemetry.api.metrics.DoubleHistogramBuilder; import io.opentelemetry.api.metrics.Meter; +import io.opentelemetry.api.metrics.MeterBuilder; import io.opentelemetry.extension.incubator.metrics.ExtendedDoubleHistogramBuilder; import java.util.List; import java.util.concurrent.TimeUnit; @@ -33,7 +34,7 @@ public class LatencyHistogram { // Used for tests - public final static LatencyHistogram NOOP = new LatencyHistogram(null, null, null, null, null) { + public final static LatencyHistogram NOOP = new LatencyHistogram() { public void recordSuccess(long latencyNanos) { } @@ -51,6 +52,12 @@ public void recordFailure(long latencyNanos) { private final Attributes failedAttributes; private final DoubleHistogram histogram; + private LatencyHistogram() { + successAttributes = null; + failedAttributes = null; + histogram = null; + } + LatencyHistogram(Meter meter, String name, String description, String topic, Attributes attributes) { DoubleHistogramBuilder builder = meter.histogramBuilder(name) .setDescription(description) @@ -58,8 +65,11 @@ public void recordFailure(long latencyNanos) { .setExplicitBucketBoundariesAdvice(latencyHistogramBuckets); if (topic != null) { - ExtendedDoubleHistogramBuilder eb = (ExtendedDoubleHistogramBuilder) builder; - eb.setAttributesAdvice(getDefaultAggregationLabels(attributes.toBuilder().put("success", true).build())); + if (builder instanceof ExtendedDoubleHistogramBuilder) { + ExtendedDoubleHistogramBuilder eb = (ExtendedDoubleHistogramBuilder) builder; + eb.setAttributesAdvice( + getDefaultAggregationLabels(attributes.toBuilder().put("success", true).build())); + } attributes = getTopicAttributes(topic, attributes); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/MetricsUtil.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/MetricsUtil.java index 1bc1847612230..b9802f4f32b5f 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/MetricsUtil.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/MetricsUtil.java @@ -47,7 +47,7 @@ static List> getDefaultAggregationLabels(Attributes attrs) { static Attributes getTopicAttributes(String topic, Attributes baseAttributes) { TopicName tn = TopicName.get(topic); - AttributesBuilder ab = Attributes.builder(); + AttributesBuilder ab = baseAttributes.toBuilder(); if (tn.isPartitioned()) { ab.put("pulsar.partition", tn.getPartitionIndex()); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/UpDownCounter.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/UpDownCounter.java index 50a3a654b9258..90f5ea9e25a92 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/UpDownCounter.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/UpDownCounter.java @@ -6,6 +6,7 @@ import io.opentelemetry.api.metrics.LongUpDownCounter; import io.opentelemetry.api.metrics.LongUpDownCounterBuilder; import io.opentelemetry.api.metrics.Meter; +import io.opentelemetry.extension.incubator.metrics.ExtendedLongCounterBuilder; import io.opentelemetry.extension.incubator.metrics.ExtendedLongUpDownCounterBuilder; public class UpDownCounter { @@ -19,8 +20,10 @@ public class UpDownCounter { .setUnit(unit.toString()); if (topic != null) { - ExtendedLongUpDownCounterBuilder eb = (ExtendedLongUpDownCounterBuilder) builder; - eb.setAttributesAdvice(getDefaultAggregationLabels(attributes)); + if (builder instanceof ExtendedLongUpDownCounterBuilder) { + ExtendedLongUpDownCounterBuilder eb = (ExtendedLongUpDownCounterBuilder) builder; + eb.setAttributesAdvice(getDefaultAggregationLabels(attributes)); + } attributes = getTopicAttributes(topic, attributes); } @@ -37,11 +40,11 @@ public void decrement() { add(-1); } - public void add(int delta) { + public void add(long delta) { counter.add(delta, attributes); } - public void subtract(int diff) { + public void subtract(long diff) { add(-diff); } } From a0cb78d215c353a881119e63c2d9c9581c7ea282 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Fri, 8 Mar 2024 08:10:10 -0800 Subject: [PATCH 07/22] Addressed more comments --- pulsar-broker/pom.xml | 1 + .../client/metrics/ClientMetricsTest.java | 55 ++++++++++--------- .../apache/pulsar/client/impl/ClientCnx.java | 4 +- .../pulsar/client/impl/ConnectionPool.java | 4 +- .../pulsar/client/impl/ConsumerImpl.java | 16 +++--- .../pulsar/client/impl/HttpLookupService.java | 2 +- .../pulsar/client/impl/ProducerImpl.java | 20 +++---- .../client/impl/UnAckedMessageTracker.java | 2 +- .../client/impl/metrics/LatencyHistogram.java | 6 +- 9 files changed, 57 insertions(+), 53 deletions(-) diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index a822edd240cd8..2b6ec7c8f5aeb 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -152,6 +152,7 @@ io.opentelemetry opentelemetry-sdk-testing + test diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java index 32a40c37390c2..62e613dc4beb4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java @@ -40,6 +40,7 @@ import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; +import org.assertj.core.api.Assertions; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -141,11 +142,10 @@ public void testProducerMetrics() throws Exception { .put("pulsar.namespace", "my-property/my-ns") .build(); Attributes nsAttrsSuccess = nsAttrs.toBuilder() - .put("success", true) + .put("pulsar.response.status", "success") .build(); var metrics = collectMetrics(); - System.err.println("All metrics: " + metrics.keySet()); assertCounterValue(metrics, "pulsar.client.connections.opened", 1, Attributes.empty()); assertCounterValue(metrics, "pulsar.client.producer.message.pending.count", 0, nsAttrs); @@ -155,13 +155,13 @@ public void testProducerMetrics() throws Exception { Attributes.builder() .put("pulsar.lookup.transport-type", "binary") .put("pulsar.lookup.type", "topic") - .put("success", true) + .put("pulsar.response.status", "success") .build()); assertHistoCountValue(metrics, "pulsar.client.lookup.duration", 1, Attributes.builder() .put("pulsar.lookup.transport-type", "binary") .put("pulsar.lookup.type", "metadata") - .put("success", true) + .put("pulsar.response.status", "success") .build()); assertHistoCountValue(metrics, "pulsar.client.producer.message.send.duration", 5, nsAttrsSuccess); @@ -176,7 +176,7 @@ public void testProducerMetrics() throws Exception { metrics = collectMetrics(); assertCounterValue(metrics, "pulsar.client.producer.closed", 1, nsAttrs); - assertCounterValue(metrics, "pulsar.client.connections.closed", 1, Attributes.empty()); + assertCounterValue(metrics, "pulsar.client.connection.closed", 1, Attributes.empty()); } @Test @@ -190,19 +190,19 @@ public void testConnectionsFailedMetrics() throws Exception { .openTelemetry(otel) .build(); - try { - client.newProducer(Schema.STRING) - .topic(topic) - .create(); - fail("Should have failed the producer creation"); - } catch (Exception e) { - // Expected - } + Assertions.assertThatThrownBy(() -> { + client.newProducer(Schema.STRING) + .topic(topic) + .create(); + }).isInstanceOf(Exception.class); + var metrics = collectMetrics(); - assertTrue(getCounterValue(metrics, "pulsar.client.connections.failed", - Attributes.builder().put("pulsar.failure.type", "tcp-failed").build()) >= 1); + Assertions.assertThat( + getCounterValue(metrics, "pulsar.client.connection.failed", + Attributes.builder().put("pulsar.failure.type", "tcp-failed").build())) + .isGreaterThanOrEqualTo(1L); } @Test @@ -240,7 +240,7 @@ public void testPublishFailedMetrics() throws Exception { .put("pulsar.namespace", "my-property/my-ns") .build(); Attributes nsAttrsFailure = nsAttrs.toBuilder() - .put("success", false) + .put("pulsar.response.status", "failed") .build(); assertCounterValue(metrics, "pulsar.client.producer.message.pending.count", 0, nsAttrs); @@ -282,23 +282,23 @@ public void testConsumerMetrics() throws Exception { .build(); var metrics = collectMetrics(); - assertCounterValue(metrics, "pulsar.client.connections.opened", 1, Attributes.empty()); + assertCounterValue(metrics, "pulsar.client.connection.opened", 1, Attributes.empty()); assertHistoCountValue(metrics, "pulsar.client.lookup.duration", 2, Attributes.builder() .put("pulsar.lookup.transport-type", "binary") .put("pulsar.lookup.type", "topic") - .put("success", true) + .put("pulsar.response.status", "success") .build()); assertHistoCountValue(metrics, "pulsar.client.lookup.duration", 2, Attributes.builder() .put("pulsar.lookup.transport-type", "binary") .put("pulsar.lookup.type", "metadata") - .put("success", true) + .put("pulsar.response.status", "success") .build()); - assertCounterValue(metrics, "pulsar.client.consumer.prefetched.count", 10, nsAttrs); - assertCounterValue(metrics, "pulsar.client.consumer.prefetched.size", "hello".length() * 10, nsAttrs); + assertCounterValue(metrics, "pulsar.client.consumer.receive_queue.count", 10, nsAttrs); + assertCounterValue(metrics, "pulsar.client.consumer.receive_queue.size", "hello".length() * 10, nsAttrs); assertCounterValue(metrics, "pulsar.client.consumer.opened", 1, nsAttrs); Message msg1 = consumer.receive(); @@ -310,15 +310,18 @@ public void testConsumerMetrics() throws Exception { /* Message msg3 = */ consumer.receive(); metrics = collectMetrics(); - assertCounterValue(metrics, "pulsar.client.consumer.prefetched.count", 7, nsAttrs); - assertCounterValue(metrics, "pulsar.client.consumer.prefetched.size", "hello".length() * 7, nsAttrs); + assertCounterValue(metrics, "pulsar.client.consumer.receive_queue.count", 7, nsAttrs); + assertCounterValue(metrics, "pulsar.client.consumer.receive_queue.size", "hello".length() * 7, nsAttrs); + assertCounterValue(metrics, "pulsar.client.consumer.message.received.count", 3, nsAttrs); + assertCounterValue(metrics, "pulsar.client.consumer.message.received.size", "hello".length() * 3, nsAttrs); + // Let msg3 to reach ack-timeout Thread.sleep(3000); metrics = collectMetrics(); - assertCounterValue(metrics, "pulsar.client.consumer.prefetched.count", 8, nsAttrs); - assertCounterValue(metrics, "pulsar.client.consumer.prefetched.size", "hello".length() * 8, nsAttrs); + assertCounterValue(metrics, "pulsar.client.consumer.receive_queue.count", 8, nsAttrs); + assertCounterValue(metrics, "pulsar.client.consumer.receive_queue.size", "hello".length() * 8, nsAttrs); assertCounterValue(metrics, "pulsar.client.consumer.message.ack", 1, nsAttrs); assertCounterValue(metrics, "pulsar.client.consumer.message.nack", 1, nsAttrs); @@ -328,6 +331,6 @@ public void testConsumerMetrics() throws Exception { metrics = collectMetrics(); assertCounterValue(metrics, "pulsar.client.consumer.closed", 1, nsAttrs); - assertCounterValue(metrics, "pulsar.client.connections.closed", 1, Attributes.empty()); + assertCounterValue(metrics, "pulsar.client.connection.closed", 1, Attributes.empty()); } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java index 0cdc388b5d6bc..6df6d41a11071 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java @@ -265,9 +265,9 @@ public ClientCnx(InstrumentProvider instrumentProvider, ClientConfigurationData this.idleState = new ClientCnxIdleState(this); this.clientVersion = "Pulsar-Java-v" + PulsarVersion.getVersion() + (conf.getDescription() == null ? "" : ("-" + conf.getDescription())); - this.connectionsOpenedCounter = instrumentProvider.newCounter("pulsar.client.connections.opened", Unit.Connections, + this.connectionsOpenedCounter = instrumentProvider.newCounter("pulsar.client.connection.opened", Unit.Connections, "The number of connections opened", null, Attributes.empty()); - this.connectionsClosedCounter = instrumentProvider.newCounter("pulsar.client.connections.closed", Unit.Connections, + this.connectionsClosedCounter = instrumentProvider.newCounter("pulsar.client.connection.closed", Unit.Connections, "The number of connections closed", null, Attributes.empty()); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java index 18cfc2cb6f6a2..d5adbdd7098ed 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java @@ -166,10 +166,10 @@ public ConnectionPool(InstrumentProvider instrumentProvider, } connectionsTcpFailureCounter = - instrumentProvider.newCounter("pulsar.client.connections.failed", Unit.Connections, + instrumentProvider.newCounter("pulsar.client.connection.failed", Unit.Connections, "The number of failed connection attempts", null, Attributes.builder().put("pulsar.failure.type", "tcp-failed").build()); - connectionsHandshakeFailureCounter = instrumentProvider.newCounter("pulsar.client.connections.failed", + connectionsHandshakeFailureCounter = instrumentProvider.newCounter("pulsar.client.connection.failed", Unit.Connections, "The number of failed connection attempts", null, Attributes.builder().put("pulsar.failure.type", "handshake").build()); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 4977a3712f071..ab7954c313227 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -404,17 +404,17 @@ protected ConsumerImpl(PulsarClientImpl client, String topic, ConsumerConfigurat InstrumentProvider ip = client.instrumentProvider(); consumersOpenedCounter = ip.newCounter("pulsar.client.consumer.opened", Unit.Sessions, - "Counter of sessions opened", topic, Attributes.empty()); + "The number of consumer sessions opened", topic, Attributes.empty()); consumersClosedCounter = ip.newCounter("pulsar.client.consumer.closed", Unit.Sessions, - "Counter of sessions closed", topic, Attributes.empty()); - messagesReceivedCounter = ip.newCounter("pulsar.client.received.count", Unit.Messages, + "The number of consumer sessions closed", topic, Attributes.empty()); + messagesReceivedCounter = ip.newCounter("pulsar.client.consumer.message.received.count", Unit.Messages, "The number of messages explicitly received by the consumer application", topic, Attributes.empty()); - bytesReceivedCounter = ip.newCounter("pulsar.client.received.size", Unit.Bytes, + bytesReceivedCounter = ip.newCounter("pulsar.client.consumer.message.received.size", Unit.Bytes, "The number of bytes explicitly received by the consumer application", topic, Attributes.empty()); - messagesPrefetchedGauge = ip.newUpDownCounter("pulsar.client.consumer.prefetched.count", Unit.Messages, - "Number of messages currently sitting in the consumer pre-fetch queue", topic, Attributes.empty()); - bytesPrefetchedGauge = ip.newUpDownCounter("pulsar.client.consumer.prefetched.size", Unit.Bytes, - "Total number of bytes currently sitting in the consumer pre-fetch queue", topic, Attributes.empty()); + messagesPrefetchedGauge = ip.newUpDownCounter("pulsar.client.consumer.receive_queue.count", Unit.Messages, + "Number of messages currently sitting in the consumer receive queue", topic, Attributes.empty()); + bytesPrefetchedGauge = ip.newUpDownCounter("pulsar.client.consumer.receive_queue.size", Unit.Bytes, + "Total number of bytes currently sitting in the consumer receive queue", topic, Attributes.empty()); consumerAcksCounter = ip.newCounter("pulsar.client.consumer.message.ack", Unit.Messages, "The number of acknowledged messages", topic, Attributes.empty()); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java index e6999b777b88f..0bfc5c78c111b 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java @@ -75,7 +75,7 @@ public HttpLookupService(InstrumentProvider instrumentProvider, ClientConfigurat this.useTls = conf.isUseTls(); this.listenerName = conf.getListenerName(); - Attributes attrs = Attributes.of(AttributeKey.stringKey("pulsar.lookup.transport-type"), "binary"); + Attributes attrs = Attributes.of(AttributeKey.stringKey("pulsar.lookup.transport-type"), "http"); histoGetBroker = instrumentProvider.newLatencyHistogram("pulsar.client.lookup.duration", "Duration of lookup operations", diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java index a729f3f80258a..27b76b8eba6b2 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java @@ -180,8 +180,8 @@ public class ProducerImpl extends ProducerBase implements TimerTask, Conne private final LatencyHistogram latencyHistogram; final LatencyHistogram rpcLatencyHistogram; private final Counter publishedBytesCounter; - private final UpDownCounter pendingMessagesCounter; - private final UpDownCounter pendingBytesCounter; + private final UpDownCounter pendingMessagesUpDownCounter; + private final UpDownCounter pendingBytesUpDownCounter; private final Counter producersOpenedCounter; private final Counter producersClosedCounter; @@ -292,10 +292,10 @@ public ProducerImpl(PulsarClientImpl client, String topic, ProducerConfiguration Attributes.empty()); publishedBytesCounter = ip.newCounter("pulsar.client.producer.message.send.size", Unit.Bytes, "The number of bytes published", topic, Attributes.empty()); - pendingMessagesCounter = ip.newUpDownCounter("pulsar.client.producer.message.pending.count", Unit.Messages, - "Pending messages for this producer", topic, Attributes.empty()); - pendingBytesCounter = ip.newUpDownCounter("pulsar.client.producer.message.pending.size", Unit.Bytes, - "Pending bytes for this producer", topic, Attributes.empty()); + pendingMessagesUpDownCounter = ip.newUpDownCounter("pulsar.client.producer.message.pending.count", Unit.Messages, + "The number of messages in the producer internal send queue, waiting to be sent", topic, Attributes.empty()); + pendingBytesUpDownCounter = ip.newUpDownCounter("pulsar.client.producer.message.pending.size", Unit.Bytes, + "The size of the messages in the producer internal queue, waiting to sent", topic, Attributes.empty()); producersOpenedCounter = ip.newCounter("pulsar.client.producer.opened", Unit.Sessions, "The number of producer sessions opened", topic, Attributes.empty()); producersClosedCounter = ip.newCounter("pulsar.client.producer.closed", Unit.Sessions, @@ -371,8 +371,8 @@ CompletableFuture internalSendAsync(Message message) { } int msgSize = interceptorMessage.getDataBuffer().readableBytes(); - pendingMessagesCounter.increment(); - pendingBytesCounter.add(msgSize); + pendingMessagesUpDownCounter.increment(); + pendingBytesUpDownCounter.add(msgSize); sendAsync(interceptorMessage, new SendCallback() { SendCallback nextCallback = null; @@ -397,8 +397,8 @@ public MessageImpl getNextMessage() { @Override public void sendComplete(Exception e) { long latencyNanos = System.nanoTime() - createdAt; - pendingMessagesCounter.decrement(); - pendingBytesCounter.subtract(msgSize); + pendingMessagesUpDownCounter.decrement(); + pendingBytesUpDownCounter.subtract(msgSize); try { if (e != null) { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java index ebed22ad78bcb..4ba3172fb48cc 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java @@ -124,7 +124,7 @@ public UnAckedMessageTracker(PulsarClientImpl client, ConsumerBase consumerBa InstrumentProvider ip = client.instrumentProvider(); consumerAckTimeoutsCounter = ip.newCounter("pulsar.client.consumer.message.ack.timeout", Unit.Messages, - "Number of ack timeouts events", consumerBase.getTopic(), Attributes.empty()); + "The number of messages that were not acknowledged in the configured timeout period, hence, were requested by the client to be redelivered", consumerBase.getTopic(), Attributes.empty()); if (conf.getAckTimeoutRedeliveryBackoff() == null) { this.messageIdPartitionMap = new HashMap<>(); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java index 1346a98317610..c792c29cbccf3 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java @@ -68,16 +68,16 @@ private LatencyHistogram() { if (builder instanceof ExtendedDoubleHistogramBuilder) { ExtendedDoubleHistogramBuilder eb = (ExtendedDoubleHistogramBuilder) builder; eb.setAttributesAdvice( - getDefaultAggregationLabels(attributes.toBuilder().put("success", true).build())); + getDefaultAggregationLabels(attributes.toBuilder().put("pulsar.response.status", "success").build())); } attributes = getTopicAttributes(topic, attributes); } successAttributes = attributes.toBuilder() - .put("success", true) + .put("pulsar.response.status", "success") .build(); failedAttributes = attributes.toBuilder() - .put("success", false) + .put("pulsar.response.status", "failed") .build(); this.histogram = builder.build(); } From 5af20183735f92ea89e7f766264cb66212a09072 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Fri, 8 Mar 2024 16:26:55 -0800 Subject: [PATCH 08/22] Share the histogram objects --- .../client/metrics/ClientMetricsTest.java | 2 +- .../client/impl/BinaryProtoLookupService.java | 20 ++++++----------- .../pulsar/client/impl/HttpLookupService.java | 22 +++++++------------ .../client/impl/metrics/LatencyHistogram.java | 20 ++++++++++++++++- 4 files changed, 35 insertions(+), 29 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java index 62e613dc4beb4..aa99c6e5e114e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java @@ -147,7 +147,7 @@ public void testProducerMetrics() throws Exception { var metrics = collectMetrics(); - assertCounterValue(metrics, "pulsar.client.connections.opened", 1, Attributes.empty()); + assertCounterValue(metrics, "pulsar.client.connection.opened", 1, Attributes.empty()); assertCounterValue(metrics, "pulsar.client.producer.message.pending.count", 0, nsAttrs); assertCounterValue(metrics, "pulsar.client.producer.message.pending.size", 0, nsAttrs); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java index 3acd4278a621c..7d97d02a37bd9 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java @@ -93,20 +93,14 @@ public BinaryProtoLookupService(PulsarClientImpl client, this.listenerName = listenerName; updateServiceUrl(serviceUrl); - Attributes attrs = Attributes.of(AttributeKey.stringKey("pulsar.lookup.transport-type"), "binary"); - - histoGetBroker = client.instrumentProvider().newLatencyHistogram("pulsar.client.lookup.duration", - "Duration of lookup operations", null, - attrs.toBuilder().put("pulsar.lookup.type", "topic").build()); - histoGetTopicMetadata = client.instrumentProvider().newLatencyHistogram("pulsar.client.lookup.duration", - "Duration of lookup operations", null, - attrs.toBuilder().put("pulsar.lookup.type", "metadata").build()); - histoGetSchema = client.instrumentProvider().newLatencyHistogram("pulsar.client.lookup.duration", - "Duration of lookup operations", null, - attrs.toBuilder().put("pulsar.lookup.type", "schema").build()); - histoListTopics = client.instrumentProvider().newLatencyHistogram("pulsar.client.lookup.duration", + LatencyHistogram histo = client.instrumentProvider().newLatencyHistogram("pulsar.client.lookup.duration", "Duration of lookup operations", null, - attrs.toBuilder().put("pulsar.lookup.type", "list-topics").build()); + Attributes.builder().put("pulsar.lookup.transport-type", "binary").build()); + histoGetBroker = histo.withAttributes(Attributes.builder().put("pulsar.lookup.type", "topic").build()); + histoGetTopicMetadata = + histo.withAttributes(Attributes.builder().put("pulsar.lookup.type", "metadata").build()); + histoGetSchema = histo.withAttributes(Attributes.builder().put("pulsar.lookup.type", "schema").build()); + histoListTopics = histo.withAttributes(Attributes.builder().put("pulsar.lookup.type", "list-topics").build()); } @Override diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java index 0bfc5c78c111b..1dff155b35111 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java @@ -75,20 +75,14 @@ public HttpLookupService(InstrumentProvider instrumentProvider, ClientConfigurat this.useTls = conf.isUseTls(); this.listenerName = conf.getListenerName(); - Attributes attrs = Attributes.of(AttributeKey.stringKey("pulsar.lookup.transport-type"), "http"); - - histoGetBroker = instrumentProvider.newLatencyHistogram("pulsar.client.lookup.duration", - "Duration of lookup operations", - null, attrs.toBuilder().put("pulsar.lookup.type", "topic").build()); - histoGetTopicMetadata = instrumentProvider.newLatencyHistogram("pulsar.client.lookup.duration", - "Duration of lookup operations", - null, attrs.toBuilder().put("pulsar.lookup.type", "metadata").build()); - histoGetSchema = instrumentProvider.newLatencyHistogram("pulsar.client.lookup.duration", - "Duration of lookup operations", - null, attrs.toBuilder().put("pulsar.lookup.type", "schema").build()); - histoListTopics = instrumentProvider.newLatencyHistogram("pulsar.client.lookup.duration", - "Duration of lookup operations", - null, attrs.toBuilder().put("pulsar.lookup.type", "list-topics").build()); + LatencyHistogram histo = instrumentProvider.newLatencyHistogram("pulsar.client.lookup.duration", + "Duration of lookup operations", null, + Attributes.builder().put("pulsar.lookup.transport-type", "http").build()); + histoGetBroker = histo.withAttributes(Attributes.builder().put("pulsar.lookup.type", "topic").build()); + histoGetTopicMetadata = + histo.withAttributes(Attributes.builder().put("pulsar.lookup.type", "metadata").build()); + histoGetSchema = histo.withAttributes(Attributes.builder().put("pulsar.lookup.type", "schema").build()); + histoListTopics = histo.withAttributes(Attributes.builder().put("pulsar.lookup.type", "list-topics").build()); } @Override diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java index c792c29cbccf3..97e7c7174b519 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java @@ -68,7 +68,8 @@ private LatencyHistogram() { if (builder instanceof ExtendedDoubleHistogramBuilder) { ExtendedDoubleHistogramBuilder eb = (ExtendedDoubleHistogramBuilder) builder; eb.setAttributesAdvice( - getDefaultAggregationLabels(attributes.toBuilder().put("pulsar.response.status", "success").build())); + getDefaultAggregationLabels( + attributes.toBuilder().put("pulsar.response.status", "success").build())); } attributes = getTopicAttributes(topic, attributes); } @@ -82,6 +83,23 @@ private LatencyHistogram() { this.histogram = builder.build(); } + private LatencyHistogram(DoubleHistogram histogram, Attributes successAttributes, Attributes failedAttributes) { + this.histogram = histogram; + this.successAttributes = successAttributes; + this.failedAttributes = failedAttributes; + } + + /** + * Create a new histograms that inherits the old histograms attributes and adds new ones. + */ + public LatencyHistogram withAttributes(Attributes attributes) { + return new LatencyHistogram( + histogram, + successAttributes.toBuilder().putAll(attributes).build(), + failedAttributes.toBuilder().putAll(attributes).build() + ); + } + public void recordSuccess(long latencyNanos) { histogram.record(latencyNanos / NANOS, successAttributes); From 218f57b4ffc0ff890315b7716bf9a97f176c63ca Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Mon, 11 Mar 2024 10:46:17 -0700 Subject: [PATCH 09/22] Added javadoc and reset old metrics default --- .../pulsar/client/api/ClientBuilder.java | 18 +++++++++++++++++- .../impl/conf/ClientConfigurationData.java | 2 +- 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java index 83bc6adf501d9..2d8c5becaefb3 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java @@ -452,7 +452,7 @@ ClientBuilder authentication(String authPluginClassName, Map aut ClientBuilder memoryLimit(long memoryLimit, SizeUnit unit); /** - * Set the interval between each stat info (default: disabled) Stats will be activated with positive + * Set the interval between each stat info (default: 60 seconds) Stats will be activated with positive * statsInterval It should be set to at least 1 second. * * @param statsInterval @@ -558,6 +558,22 @@ ClientBuilder authentication(String authPluginClassName, Map aut */ ClientBuilder enableBusyWait(boolean enableBusyWait); + /** + * Configure OpenTelemetry for Pulsar Client + *

+ * When you pass an OpenTelemetry instance, Pulsar client will emit metrics that can be exported in a variety + * of different methods. + *

+ * Refer to OpenTelemetry Java SDK documentation for + * how to configure OpenTelemetry and the metrics exporter. + *

+ * By default, Pulsar client will use the {@link io.opentelemetry.api.GlobalOpenTelemetry} instance. If an + * OpenTelemetry JVM agent is configured, the metrics will be reported, otherwise the metrics will be + * completely disabled. + * + * @param openTelemetry the OpenTelemetry instance + * @return the client builder instance + */ ClientBuilder openTelemetry(io.opentelemetry.api.OpenTelemetry openTelemetry); /** diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java index c50f1b542c6c7..6dcea7dc46672 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java @@ -108,7 +108,7 @@ public class ClientConfigurationData implements Serializable, Cloneable { name = "statsIntervalSeconds", value = "Interval to print client stats (in seconds)." ) - private long statsIntervalSeconds = 0; + private long statsIntervalSeconds = 60; @ApiModelProperty( name = "numIoThreads", From bd1fb3ff1583a1a30ff79b10970e1c91a9d7c50c Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Mon, 11 Mar 2024 11:36:18 -0700 Subject: [PATCH 10/22] Added missing license header --- .../client/impl/metrics/UpDownCounter.java | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/UpDownCounter.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/UpDownCounter.java index 90f5ea9e25a92..309749c548467 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/UpDownCounter.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/UpDownCounter.java @@ -1,3 +1,22 @@ +/* + * 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.client.impl.metrics; import static org.apache.pulsar.client.impl.metrics.MetricsUtil.getDefaultAggregationLabels; From 59bf64b3619c2c9b7c16e235a2eea310d14115cf Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Mon, 11 Mar 2024 12:25:56 -0700 Subject: [PATCH 11/22] Removed trailing spaces --- .../main/java/org/apache/pulsar/client/api/ClientBuilder.java | 2 +- .../main/java/org/apache/pulsar/client/api/ConsumerStats.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java index 2d8c5becaefb3..735aeeed55916 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java @@ -460,7 +460,7 @@ ClientBuilder authentication(String authPluginClassName, Map aut * @param unit * time unit for {@code statsInterval} * @return the client builder instance - * + * * @deprecated @see {@link #openTelemetry(OpenTelemetry)} */ @Deprecated diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerStats.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerStats.java index d36cb846c12c6..e488aa81151ce 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerStats.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerStats.java @@ -30,7 +30,7 @@ * *

All the stats are relative to the last recording period. The interval of the stats refreshes is configured with * {@link ClientBuilder#statsInterval(long, java.util.concurrent.TimeUnit)} with a default of 1 minute. - * + * * @deprecated use {@link ClientBuilder#openTelemetry(OpenTelemetry)} to enable stats */ @InterfaceAudience.Public From 857c0a3b8481f84a2e1368b0c831d53bdf296b61 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Mon, 11 Mar 2024 13:14:52 -0700 Subject: [PATCH 12/22] Fixed checkstyle --- .../client/impl/BinaryProtoLookupService.java | 1 - .../apache/pulsar/client/impl/ClientCnx.java | 13 +++++++---- .../pulsar/client/impl/HttpLookupService.java | 4 ++-- .../pulsar/client/impl/ProducerImpl.java | 6 +++-- .../pulsar/client/impl/PulsarClientImpl.java | 2 -- .../client/impl/UnAckedMessageTracker.java | 4 +++- .../pulsar/client/impl/metrics/Counter.java | 3 ++- .../impl/metrics/InstrumentProvider.java | 5 ++-- .../client/impl/metrics/LatencyHistogram.java | 3 +-- .../client/impl/metrics/UpDownCounter.java | 1 - .../client/impl/metrics/package-info.java | 23 +++++++++++++++++++ 11 files changed, 45 insertions(+), 20 deletions(-) create mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/package-info.java diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java index 7d97d02a37bd9..3506a8f6ce195 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java @@ -20,7 +20,6 @@ import static java.lang.String.format; import io.netty.buffer.ByteBuf; -import io.opentelemetry.api.common.AttributeKey; import io.opentelemetry.api.common.Attributes; import java.net.InetSocketAddress; import java.net.URI; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java index 6df6d41a11071..fd70e871135f4 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java @@ -249,7 +249,8 @@ public ClientCnx(InstrumentProvider instrumentProvider, this(instrumentProvider, conf, eventLoopGroup, Commands.getCurrentProtocolVersion()); } - public ClientCnx(InstrumentProvider instrumentProvider, ClientConfigurationData conf, EventLoopGroup eventLoopGroup, int protocolVersion) { + public ClientCnx(InstrumentProvider instrumentProvider, ClientConfigurationData conf, EventLoopGroup eventLoopGroup, + int protocolVersion) { super(conf.getKeepAliveIntervalSeconds(), TimeUnit.SECONDS); checkArgument(conf.getMaxLookupRequest() > conf.getConcurrentLookupRequest()); this.pendingLookupRequestSemaphore = new Semaphore(conf.getConcurrentLookupRequest(), false); @@ -265,10 +266,12 @@ public ClientCnx(InstrumentProvider instrumentProvider, ClientConfigurationData this.idleState = new ClientCnxIdleState(this); this.clientVersion = "Pulsar-Java-v" + PulsarVersion.getVersion() + (conf.getDescription() == null ? "" : ("-" + conf.getDescription())); - this.connectionsOpenedCounter = instrumentProvider.newCounter("pulsar.client.connection.opened", Unit.Connections, - "The number of connections opened", null, Attributes.empty()); - this.connectionsClosedCounter = instrumentProvider.newCounter("pulsar.client.connection.closed", Unit.Connections, - "The number of connections closed", null, Attributes.empty()); + this.connectionsOpenedCounter = + instrumentProvider.newCounter("pulsar.client.connection.opened", Unit.Connections, + "The number of connections opened", null, Attributes.empty()); + this.connectionsClosedCounter = + instrumentProvider.newCounter("pulsar.client.connection.closed", Unit.Connections, + "The number of connections closed", null, Attributes.empty()); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java index 1dff155b35111..8158b6d979efd 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java @@ -19,7 +19,6 @@ package org.apache.pulsar.client.impl; import io.netty.channel.EventLoopGroup; -import io.opentelemetry.api.common.AttributeKey; import io.opentelemetry.api.common.Attributes; import java.net.InetSocketAddress; import java.net.URI; @@ -69,7 +68,8 @@ public class HttpLookupService implements LookupService { private final LatencyHistogram histoGetSchema; private final LatencyHistogram histoListTopics; - public HttpLookupService(InstrumentProvider instrumentProvider, ClientConfigurationData conf, EventLoopGroup eventLoopGroup) + public HttpLookupService(InstrumentProvider instrumentProvider, ClientConfigurationData conf, + EventLoopGroup eventLoopGroup) throws PulsarClientException { this.httpClient = new HttpClient(conf, eventLoopGroup); this.useTls = conf.isUseTls(); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java index 27b76b8eba6b2..e40cefdb42b47 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java @@ -292,8 +292,10 @@ public ProducerImpl(PulsarClientImpl client, String topic, ProducerConfiguration Attributes.empty()); publishedBytesCounter = ip.newCounter("pulsar.client.producer.message.send.size", Unit.Bytes, "The number of bytes published", topic, Attributes.empty()); - pendingMessagesUpDownCounter = ip.newUpDownCounter("pulsar.client.producer.message.pending.count", Unit.Messages, - "The number of messages in the producer internal send queue, waiting to be sent", topic, Attributes.empty()); + pendingMessagesUpDownCounter = + ip.newUpDownCounter("pulsar.client.producer.message.pending.count", Unit.Messages, + "The number of messages in the producer internal send queue, waiting to be sent", topic, + Attributes.empty()); pendingBytesUpDownCounter = ip.newUpDownCounter("pulsar.client.producer.message.pending.size", Unit.Bytes, "The size of the messages in the producer internal queue, waiting to sent", topic, Attributes.empty()); producersOpenedCounter = ip.newCounter("pulsar.client.producer.opened", Unit.Sessions, diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index 22ef86fae47f7..35d1ab4567386 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -26,7 +26,6 @@ import io.netty.channel.EventLoopGroup; import io.netty.util.HashedWheelTimer; import io.netty.util.Timer; -import io.opentelemetry.api.metrics.Meter; import java.io.IOException; import java.net.InetSocketAddress; import java.time.Clock; @@ -71,7 +70,6 @@ import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; import org.apache.pulsar.client.impl.conf.ReaderConfigurationData; -import org.apache.pulsar.client.impl.metrics.Counter; import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.client.impl.schema.AutoConsumeSchema; import org.apache.pulsar.client.impl.schema.AutoProduceBytesSchema; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java index 4ba3172fb48cc..1937ed1dcfdb5 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java @@ -124,7 +124,9 @@ public UnAckedMessageTracker(PulsarClientImpl client, ConsumerBase consumerBa InstrumentProvider ip = client.instrumentProvider(); consumerAckTimeoutsCounter = ip.newCounter("pulsar.client.consumer.message.ack.timeout", Unit.Messages, - "The number of messages that were not acknowledged in the configured timeout period, hence, were requested by the client to be redelivered", consumerBase.getTopic(), Attributes.empty()); + "The number of messages that were not acknowledged in the configured timeout period, hence, were " + + "requested by the client to be redelivered", + consumerBase.getTopic(), Attributes.empty()); if (conf.getAckTimeoutRedeliveryBackoff() == null) { this.messageIdPartitionMap = new HashMap<>(); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Counter.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Counter.java index 1e1e27271007e..fffbab4217a86 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Counter.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Counter.java @@ -18,7 +18,8 @@ */ package org.apache.pulsar.client.impl.metrics; -import static org.apache.pulsar.client.impl.metrics.MetricsUtil.*; +import static org.apache.pulsar.client.impl.metrics.MetricsUtil.getDefaultAggregationLabels; +import static org.apache.pulsar.client.impl.metrics.MetricsUtil.getTopicAttributes; import io.opentelemetry.api.common.Attributes; import io.opentelemetry.api.metrics.LongCounter; import io.opentelemetry.api.metrics.LongCounterBuilder; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/InstrumentProvider.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/InstrumentProvider.java index 2b99787d59b4b..1e02af1fd37e1 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/InstrumentProvider.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/InstrumentProvider.java @@ -23,9 +23,7 @@ import io.opentelemetry.api.OpenTelemetry; import io.opentelemetry.api.common.Attributes; import io.opentelemetry.api.metrics.Meter; -import java.util.Optional; import org.apache.pulsar.PulsarVersion; -import org.apache.pulsar.client.impl.conf.ClientConfigurationData; public class InstrumentProvider { @@ -49,7 +47,8 @@ public Counter newCounter(String name, Unit unit, String description, String top return new Counter(meter, name, unit, description, topic, attributes); } - public UpDownCounter newUpDownCounter(String name, Unit unit, String description, String topic, Attributes attributes) { + public UpDownCounter newUpDownCounter(String name, Unit unit, String description, String topic, + Attributes attributes) { return new UpDownCounter(meter, name, unit, description, topic, attributes); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java index 97e7c7174b519..ed04eff03b39d 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java @@ -26,7 +26,6 @@ import io.opentelemetry.api.metrics.DoubleHistogram; import io.opentelemetry.api.metrics.DoubleHistogramBuilder; import io.opentelemetry.api.metrics.Meter; -import io.opentelemetry.api.metrics.MeterBuilder; import io.opentelemetry.extension.incubator.metrics.ExtendedDoubleHistogramBuilder; import java.util.List; import java.util.concurrent.TimeUnit; @@ -34,7 +33,7 @@ public class LatencyHistogram { // Used for tests - public final static LatencyHistogram NOOP = new LatencyHistogram() { + public static final LatencyHistogram NOOP = new LatencyHistogram() { public void recordSuccess(long latencyNanos) { } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/UpDownCounter.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/UpDownCounter.java index 309749c548467..3df0c2bb42302 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/UpDownCounter.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/UpDownCounter.java @@ -25,7 +25,6 @@ import io.opentelemetry.api.metrics.LongUpDownCounter; import io.opentelemetry.api.metrics.LongUpDownCounterBuilder; import io.opentelemetry.api.metrics.Meter; -import io.opentelemetry.extension.incubator.metrics.ExtendedLongCounterBuilder; import io.opentelemetry.extension.incubator.metrics.ExtendedLongUpDownCounterBuilder; public class UpDownCounter { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/package-info.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/package-info.java new file mode 100644 index 0000000000000..ee99bb3332c26 --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/package-info.java @@ -0,0 +1,23 @@ +/* + * 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. + */ + +/** + * Pulsar Client OTel metrics utilities + */ +package org.apache.pulsar.client.impl.metrics; From e103369c35402e7ee30778c90aded890325a5d9e Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Mon, 11 Mar 2024 13:55:40 -0700 Subject: [PATCH 13/22] Checkstyle --- .../java/org/apache/pulsar/client/impl/ProducerImpl.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java index e40cefdb42b47..7b2451871d9ba 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java @@ -1496,8 +1496,8 @@ void initialize() { chunkedMessageCtx = null; } - static OpSendMsg create(LatencyHistogram rpcLatencyHistogram, MessageImpl msg, ByteBufPair cmd, long sequenceId, - SendCallback callback) { + static OpSendMsg create(LatencyHistogram rpcLatencyHistogram, MessageImpl msg, ByteBufPair cmd, + long sequenceId, SendCallback callback) { OpSendMsg op = RECYCLER.get(); op.initialize(); op.rpcLatencyHistogram = rpcLatencyHistogram; @@ -1510,9 +1510,8 @@ static OpSendMsg create(LatencyHistogram rpcLatencyHistogram, MessageImpl msg return op; } - static OpSendMsg create(LatencyHistogram rpcLatencyHistogram, List> msgs, ByteBufPair cmd, long sequenceId, - SendCallback callback, - int batchAllocatedSize) { + static OpSendMsg create(LatencyHistogram rpcLatencyHistogram, List> msgs, ByteBufPair cmd, + long sequenceId, SendCallback callback, int batchAllocatedSize) { OpSendMsg op = RECYCLER.get(); op.initialize(); op.rpcLatencyHistogram = rpcLatencyHistogram; From 1f2d0c832524e7d1453d3578b5f7aaf9db709936 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Mon, 11 Mar 2024 14:01:11 -0700 Subject: [PATCH 14/22] Checkstyle --- .../api/PulsarMultiListenersWithInternalListenerNameTest.java | 1 - .../java/org/apache/pulsar/client/metrics/ClientMetricsTest.java | 1 - 2 files changed, 2 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PulsarMultiListenersWithInternalListenerNameTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PulsarMultiListenersWithInternalListenerNameTest.java index ced7fccd6b7ed..a076e20b33218 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PulsarMultiListenersWithInternalListenerNameTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PulsarMultiListenersWithInternalListenerNameTest.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar.client.api; -import io.opentelemetry.api.OpenTelemetry; import java.io.IOException; import java.io.UncheckedIOException; import java.net.InetAddress; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java index aa99c6e5e114e..3fd7716697520 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java @@ -20,7 +20,6 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; -import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; import io.opentelemetry.api.OpenTelemetry; import io.opentelemetry.api.common.Attributes; From 413360c1121011f394114035d3309a85bdc2df9e Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Mon, 11 Mar 2024 15:00:52 -0700 Subject: [PATCH 15/22] Fixed spotbugs --- .../apache/pulsar/client/impl/BinaryProtoLookupService.java | 1 - .../java/org/apache/pulsar/client/impl/PulsarClientImpl.java | 3 +-- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java index 3506a8f6ce195..81c196c731f70 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java @@ -145,7 +145,6 @@ public CompletableFuture getBroker(TopicName topicName) { * */ public CompletableFuture getPartitionedTopicMetadata(TopicName topicName) { - long startTime = System.nanoTime(); final MutableObject newFutureCreated = new MutableObject<>(); try { return partitionedMetadataInProgress.computeIfAbsent(topicName, tpName -> { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index 35d1ab4567386..a919eb19a7ff8 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -180,8 +180,6 @@ private PulsarClientImpl(ClientConfigurationData conf, EventLoopGroup eventLoopG ExecutorProvider internalExecutorProvider, ScheduledExecutorProvider scheduledExecutorProvider) throws PulsarClientException { - this.instrumentProvider = new InstrumentProvider(conf.getOpenTelemetry()); - EventLoopGroup eventLoopGroupReference = null; ConnectionPool connectionPoolReference = null; try { @@ -199,6 +197,7 @@ private PulsarClientImpl(ClientConfigurationData conf, EventLoopGroup eventLoopG throw new PulsarClientException.InvalidConfigurationException("Invalid client configuration"); } this.conf = conf; + this.instrumentProvider = new InstrumentProvider(conf.getOpenTelemetry()); clientClock = conf.getClock(); conf.getAuthentication().start(); connectionPoolReference = From 345f24060e484d99c1d0638bb11f70975022645c Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Mon, 11 Mar 2024 15:39:11 -0700 Subject: [PATCH 16/22] Fixed compilation --- .../org/apache/pulsar/proxy/server/ProxyConnection.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java index ba9247a085dff..594d6cbc3bb59 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java @@ -59,6 +59,7 @@ import org.apache.pulsar.client.impl.PulsarChannelInitializer; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.client.impl.conf.ConfigurationDataUtils; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.client.internal.PropertiesUtils; import org.apache.pulsar.common.api.AuthData; import org.apache.pulsar.common.api.proto.CommandAuthResponse; @@ -383,11 +384,12 @@ private synchronized void completeConnect() throws PulsarClientException { service.getConfiguration().isForwardAuthorizationCredentials(), this); } else { clientCnxSupplier = - () -> new ClientCnx(clientConf, service.getWorkerGroup(), protocolVersionToAdvertise); + () -> new ClientCnx(InstrumentProvider.NOOP, clientConf, service.getWorkerGroup(), + protocolVersionToAdvertise); } if (this.connectionPool == null) { - this.connectionPool = new ConnectionPool(clientConf, service.getWorkerGroup(), + this.connectionPool = new ConnectionPool(InstrumentProvider.NOOP, clientConf, service.getWorkerGroup(), clientCnxSupplier, Optional.of(dnsAddressResolverGroup.getResolver(service.getWorkerGroup().next()))); } else { From 0b60e7d9f8d98c511152dea837f2aa7a071a1a81 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Mon, 11 Mar 2024 16:24:06 -0700 Subject: [PATCH 17/22] Fix --- .../java/org/apache/pulsar/proxy/server/ProxyClientCnx.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyClientCnx.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyClientCnx.java index 782454022b1ed..d15d48b9209d0 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyClientCnx.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyClientCnx.java @@ -26,6 +26,7 @@ import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.client.impl.ClientCnx; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.api.AuthData; import org.apache.pulsar.common.api.proto.CommandAuthChallenge; import org.apache.pulsar.common.protocol.Commands; @@ -47,7 +48,7 @@ public class ProxyClientCnx extends ClientCnx { public ProxyClientCnx(ClientConfigurationData conf, EventLoopGroup eventLoopGroup, String clientAuthRole, String clientAuthMethod, int protocolVersion, boolean forwardClientAuthData, ProxyConnection proxyConnection) { - super(conf, eventLoopGroup, protocolVersion); + super(InstrumentProvider.NOOP, conf, eventLoopGroup, protocolVersion); this.clientAuthRole = clientAuthRole; this.clientAuthMethod = clientAuthMethod; this.forwardClientAuthData = forwardClientAuthData; From 70dd347db94118938f09991e9cb9aa67b2e9edd1 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Mon, 11 Mar 2024 17:03:58 -0700 Subject: [PATCH 18/22] Fixed test --- .../java/org/apache/pulsar/proxy/server/ProxyParserTest.java | 5 +++-- .../test/java/org/apache/pulsar/proxy/server/ProxyTest.java | 5 +++-- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java index 3f58250e6d68a..1a9459619ebe9 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java @@ -42,6 +42,7 @@ import org.apache.pulsar.client.impl.ConnectionPool; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.api.proto.CommandActiveConsumerChange; import org.apache.pulsar.common.api.proto.ProtocolVersion; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; @@ -251,8 +252,8 @@ private static PulsarClient getClientActiveConsumerChangeNotSupported(ClientConf final EventLoopGroup eventLoopGroup) throws Exception { - ConnectionPool cnxPool = new ConnectionPool(conf, eventLoopGroup, () -> { - return new ClientCnx(conf, eventLoopGroup, ProtocolVersion.v11_VALUE) { + ConnectionPool cnxPool = new ConnectionPool(InstrumentProvider.NOOP, conf, eventLoopGroup, () -> { + return new ClientCnx(InstrumentProvider.NOOP, conf, eventLoopGroup, ProtocolVersion.v11_VALUE) { @Override protected void handleActiveConsumerChange(CommandActiveConsumerChange change) { throw new UnsupportedOperationException(); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTest.java index 9bc12dcc6fcb2..e1e49f9e8c5f2 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTest.java @@ -50,6 +50,7 @@ import org.apache.pulsar.client.impl.ConnectionPool; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.api.proto.CommandActiveConsumerChange; import org.apache.pulsar.common.api.proto.ProtocolVersion; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; @@ -374,8 +375,8 @@ private PulsarClient getClientActiveConsumerChangeNotSupported(ClientConfigurati EventLoopGroup eventLoopGroup = EventLoopUtil.newEventLoopGroup(conf.getNumIoThreads(), false, threadFactory); registerCloseable(() -> eventLoopGroup.shutdownNow()); - ConnectionPool cnxPool = new ConnectionPool(conf, eventLoopGroup, () -> { - return new ClientCnx(conf, eventLoopGroup, ProtocolVersion.v11_VALUE) { + ConnectionPool cnxPool = new ConnectionPool(InstrumentProvider.NOOP, conf, eventLoopGroup, () -> { + return new ClientCnx(InstrumentProvider.NOOP, conf, eventLoopGroup, ProtocolVersion.v11_VALUE) { @Override protected void handleActiveConsumerChange(CommandActiveConsumerChange change) { throw new UnsupportedOperationException(); From d90f621a80c53ec9c5edf5f595b8fd77ea97a47c Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Wed, 13 Mar 2024 17:35:38 -0700 Subject: [PATCH 19/22] Fixed tests mock --- distribution/shell/src/assemble/LICENSE.bin.txt | 4 ++++ .../pulsar/client/impl/ConnectionPoolTest.java | 17 +++++++++++------ .../impl/BinaryProtoLookupServiceTest.java | 2 ++ .../impl/PartitionedProducerImplTest.java | 2 ++ 4 files changed, 19 insertions(+), 6 deletions(-) diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 8ddcbcfb1600d..44a3b3317c01e 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -384,6 +384,10 @@ The Apache Software License, Version 2.0 - log4j-core-2.18.0.jar - log4j-slf4j-impl-2.18.0.jar - log4j-web-2.18.0.jar + * OpenTelemetry + - opentelemetry-api-1.34.1.jar + - opentelemetry-context-1.34.1.jar + - opentelemetry-extension-incubator-1.34.1-alpha.jar * BookKeeper - bookkeeper-common-allocator-4.16.4.jar diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionPoolTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionPoolTest.java index 553aad294dd66..1037019d608ab 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionPoolTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionPoolTest.java @@ -69,7 +69,8 @@ protected void cleanup() throws Exception { public void testSingleIpAddress() throws Exception { ClientConfigurationData conf = new ClientConfigurationData(); EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(1, false, new DefaultThreadFactory("test")); - ConnectionPool pool = spyWithClassAndConstructorArgs(ConnectionPool.class, conf, eventLoop); + ConnectionPool pool = + spyWithClassAndConstructorArgs(ConnectionPool.class, InstrumentProvider.NOOP, conf, eventLoop); conf.setServiceUrl(serviceUrl); PulsarClientImpl client = new PulsarClientImpl(conf, eventLoop, pool); @@ -119,7 +120,7 @@ public void testSelectConnectionForSameProducer() throws Exception { public void testDoubleIpAddress() throws Exception { ClientConfigurationData conf = new ClientConfigurationData(); EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(1, false, new DefaultThreadFactory("test")); - ConnectionPool pool = spyWithClassAndConstructorArgs(ConnectionPool.class, conf, eventLoop); + ConnectionPool pool = spyWithClassAndConstructorArgs(ConnectionPool.class, InstrumentProvider.NOOP, conf, eventLoop); conf.setServiceUrl(serviceUrl); PulsarClientImpl client = new PulsarClientImpl(conf, eventLoop, pool); @@ -144,7 +145,8 @@ public void testNoConnectionPool() throws Exception { ClientConfigurationData conf = new ClientConfigurationData(); conf.setConnectionsPerBroker(0); EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(8, false, new DefaultThreadFactory("test")); - ConnectionPool pool = spyWithClassAndConstructorArgs(ConnectionPool.class, conf, eventLoop); + ConnectionPool pool = + spyWithClassAndConstructorArgs(ConnectionPool.class, InstrumentProvider.NOOP, conf, eventLoop); InetSocketAddress brokerAddress = InetSocketAddress.createUnresolved("127.0.0.1", brokerPort); @@ -167,7 +169,8 @@ public void testEnableConnectionPool() throws Exception { ClientConfigurationData conf = new ClientConfigurationData(); conf.setConnectionsPerBroker(5); EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(8, false, new DefaultThreadFactory("test")); - ConnectionPool pool = spyWithClassAndConstructorArgs(ConnectionPool.class, conf, eventLoop); + ConnectionPool pool = + spyWithClassAndConstructorArgs(ConnectionPool.class, InstrumentProvider.NOOP, conf, eventLoop); InetSocketAddress brokerAddress = InetSocketAddress.createUnresolved("127.0.0.1", brokerPort); @@ -234,8 +237,10 @@ protected void doResolveAll(SocketAddress socketAddress, Promise promise) throws } }; - ConnectionPool pool = spyWithClassAndConstructorArgs(ConnectionPool.class, conf, eventLoop, - (Supplier) () -> new ClientCnx(InstrumentProvider.NOOP, conf, eventLoop), Optional.of(resolver)); + ConnectionPool pool = + spyWithClassAndConstructorArgs(ConnectionPool.class, InstrumentProvider.NOOP, conf, eventLoop, + (Supplier) () -> new ClientCnx(InstrumentProvider.NOOP, conf, eventLoop), + Optional.of(resolver)); ClientCnx cnx = pool.getConnection( diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BinaryProtoLookupServiceTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BinaryProtoLookupServiceTest.java index 87188255b20b8..983cd21a7a9d8 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BinaryProtoLookupServiceTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BinaryProtoLookupServiceTest.java @@ -36,6 +36,7 @@ import org.apache.pulsar.client.api.PulsarClientException.LookupException; import org.apache.pulsar.client.impl.BinaryProtoLookupService.LookupDataResult; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.naming.TopicName; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -65,6 +66,7 @@ public void setup() throws Exception { doReturn(0).when(clientConfig).getMaxLookupRedirects(); PulsarClientImpl client = mock(PulsarClientImpl.class); + doReturn(InstrumentProvider.NOOP).when(client).instrumentProvider(); doReturn(cnxPool).when(client).getCnxPool(); doReturn(clientConfig).when(client).getConfiguration(); doReturn(1L).when(client).newRequestId(); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PartitionedProducerImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PartitionedProducerImplTest.java index b96f6a344a3dc..f96d2e2e0b0e9 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PartitionedProducerImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PartitionedProducerImplTest.java @@ -47,6 +47,7 @@ import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; import org.apache.pulsar.client.impl.customroute.PartialRoundRobinMessageRouterImpl; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.util.netty.EventLoopUtil; import org.assertj.core.util.Sets; @@ -78,6 +79,7 @@ public void setup() { producerBuilderImpl = new ProducerBuilderImpl(client, Schema.BYTES); + when(client.instrumentProvider()).thenReturn(InstrumentProvider.NOOP); when(client.getConfiguration()).thenReturn(clientConfigurationData); when(client.timer()).thenReturn(timer); when(client.newProducer()).thenReturn(producerBuilderImpl); From 22d46af6df3c34160d9a056ec12806883c27051e Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Wed, 13 Mar 2024 18:01:08 -0700 Subject: [PATCH 20/22] Fixed test --- .../apache/pulsar/client/impl/UnAckedMessageTrackerTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/UnAckedMessageTrackerTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/UnAckedMessageTrackerTest.java index 91ad321048226..c1666335e0f63 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/UnAckedMessageTrackerTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/UnAckedMessageTrackerTest.java @@ -36,6 +36,7 @@ import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.MessageIdAdv; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; import org.testng.annotations.Test; @@ -46,6 +47,7 @@ public class UnAckedMessageTrackerTest { public void testAddAndRemove() { PulsarClientImpl client = mock(PulsarClientImpl.class); ConnectionPool connectionPool = mock(ConnectionPool.class); + when(client.instrumentProvider()).thenReturn(InstrumentProvider.NOOP); when(client.getCnxPool()).thenReturn(connectionPool); Timer timer = new HashedWheelTimer(new DefaultThreadFactory("pulsar-timer", Thread.currentThread().isDaemon()), 1, TimeUnit.MILLISECONDS); From a9f3857cc3116f0540d36c08709173ff43dff54e Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Wed, 13 Mar 2024 18:08:46 -0700 Subject: [PATCH 21/22] Attach subscription attribute to consumer metrics --- .../client/metrics/ClientMetricsTest.java | 1 + .../pulsar/client/impl/ConsumerImpl.java | 19 ++++++++++--------- .../client/impl/UnAckedMessageTracker.java | 3 ++- 3 files changed, 13 insertions(+), 10 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java index 3fd7716697520..31305123c4148 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java @@ -278,6 +278,7 @@ public void testConsumerMetrics() throws Exception { Attributes nsAttrs = Attributes.builder() .put("pulsar.tenant", "my-property") .put("pulsar.namespace", "my-property/my-ns") + .put("pulsar.subscription", "my-sub") .build(); var metrics = collectMetrics(); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index ae7b20262cfae..fa5d6799723d6 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -405,25 +405,26 @@ protected ConsumerImpl(PulsarClientImpl client, String topic, ConsumerConfigurat topicNameWithoutPartition = topicName.getPartitionedTopicName(); InstrumentProvider ip = client.instrumentProvider(); + Attributes attrs = Attributes.builder().put("pulsar.subscription", subscription).build(); consumersOpenedCounter = ip.newCounter("pulsar.client.consumer.opened", Unit.Sessions, - "The number of consumer sessions opened", topic, Attributes.empty()); + "The number of consumer sessions opened", topic, attrs); consumersClosedCounter = ip.newCounter("pulsar.client.consumer.closed", Unit.Sessions, - "The number of consumer sessions closed", topic, Attributes.empty()); + "The number of consumer sessions closed", topic, attrs); messagesReceivedCounter = ip.newCounter("pulsar.client.consumer.message.received.count", Unit.Messages, - "The number of messages explicitly received by the consumer application", topic, Attributes.empty()); + "The number of messages explicitly received by the consumer application", topic, attrs); bytesReceivedCounter = ip.newCounter("pulsar.client.consumer.message.received.size", Unit.Bytes, - "The number of bytes explicitly received by the consumer application", topic, Attributes.empty()); + "The number of bytes explicitly received by the consumer application", topic, attrs); messagesPrefetchedGauge = ip.newUpDownCounter("pulsar.client.consumer.receive_queue.count", Unit.Messages, - "Number of messages currently sitting in the consumer receive queue", topic, Attributes.empty()); + "Number of messages currently sitting in the consumer receive queue", topic, attrs); bytesPrefetchedGauge = ip.newUpDownCounter("pulsar.client.consumer.receive_queue.size", Unit.Bytes, - "Total number of bytes currently sitting in the consumer receive queue", topic, Attributes.empty()); + "Total number of bytes currently sitting in the consumer receive queue", topic, attrs); consumerAcksCounter = ip.newCounter("pulsar.client.consumer.message.ack", Unit.Messages, - "The number of acknowledged messages", topic, Attributes.empty()); + "The number of acknowledged messages", topic, attrs); consumerNacksCounter = ip.newCounter("pulsar.client.consumer.message.nack", Unit.Messages, - "The number of negatively acknowledged messages", topic, Attributes.empty()); + "The number of negatively acknowledged messages", topic, attrs); consumerDlqMessagesCounter = ip.newCounter("pulsar.client.consumer.message.dlq", Unit.Messages, - "The number of messages sent to DLQ", topic, Attributes.empty()); + "The number of messages sent to DLQ", topic, attrs); grabCnx(); consumersOpenedCounter.increment(); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java index 1937ed1dcfdb5..e755b6ba1ee6d 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java @@ -126,7 +126,8 @@ public UnAckedMessageTracker(PulsarClientImpl client, ConsumerBase consumerBa consumerAckTimeoutsCounter = ip.newCounter("pulsar.client.consumer.message.ack.timeout", Unit.Messages, "The number of messages that were not acknowledged in the configured timeout period, hence, were " + "requested by the client to be redelivered", - consumerBase.getTopic(), Attributes.empty()); + consumerBase.getTopic(), + Attributes.builder().put("pulsar.subscription", consumerBase.getSubscription()).build()); if (conf.getAckTimeoutRedeliveryBackoff() == null) { this.messageIdPartitionMap = new HashMap<>(); From 980bae35ae4636fffff1736bf01c3a62cb089a0c Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Wed, 13 Mar 2024 18:29:19 -0700 Subject: [PATCH 22/22] More test fix --- .../main/java/org/apache/pulsar/client/impl/ConsumerImpl.java | 4 ++-- .../apache/pulsar/client/impl/UnAckedMessageTrackerTest.java | 1 + 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index fa5d6799723d6..66986585424bc 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -415,9 +415,9 @@ protected ConsumerImpl(PulsarClientImpl client, String topic, ConsumerConfigurat bytesReceivedCounter = ip.newCounter("pulsar.client.consumer.message.received.size", Unit.Bytes, "The number of bytes explicitly received by the consumer application", topic, attrs); messagesPrefetchedGauge = ip.newUpDownCounter("pulsar.client.consumer.receive_queue.count", Unit.Messages, - "Number of messages currently sitting in the consumer receive queue", topic, attrs); + "The number of messages currently sitting in the consumer receive queue", topic, attrs); bytesPrefetchedGauge = ip.newUpDownCounter("pulsar.client.consumer.receive_queue.size", Unit.Bytes, - "Total number of bytes currently sitting in the consumer receive queue", topic, attrs); + "The total size in bytes of messages currently sitting in the consumer receive queue", topic, attrs); consumerAcksCounter = ip.newCounter("pulsar.client.consumer.message.ack", Unit.Messages, "The number of acknowledged messages", topic, attrs); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/UnAckedMessageTrackerTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/UnAckedMessageTrackerTest.java index c1666335e0f63..b01fbcb879f80 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/UnAckedMessageTrackerTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/UnAckedMessageTrackerTest.java @@ -88,6 +88,7 @@ public void testAddAndRemove() { public void testTrackChunkedMessageId() { PulsarClientImpl client = mock(PulsarClientImpl.class); ConnectionPool connectionPool = mock(ConnectionPool.class); + when(client.instrumentProvider()).thenReturn(InstrumentProvider.NOOP); when(client.getCnxPool()).thenReturn(connectionPool); Timer timer = new HashedWheelTimer(new DefaultThreadFactory("pulsar-timer", Thread.currentThread().isDaemon()), 1, TimeUnit.MILLISECONDS);