From 2749ceee222696ef770f77ff4a1669a242a5c940 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 8 Jan 2024 18:05:45 -0800 Subject: [PATCH 001/133] Add OTel dependencies in pom --- pom.xml | 9 +++++++++ pulsar-common/pom.xml | 7 ++++++- 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 8b23966df32b3..85700efda45d8 100644 --- a/pom.xml +++ b/pom.xml @@ -248,6 +248,7 @@ flexible messaging model and an intuitive client API. 3.4.3 1.5.2-3 2.0.6 + 1.34.0 1.18.3 @@ -1425,6 +1426,14 @@ flexible messaging model and an intuitive client API. checker-qual ${checkerframework.version} + + + io.opentelemetry + opentelemetry-bom + ${opentelemetry.version} + pom + import + diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index 1b54a7aee2d8c..306a1966ff48a 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -198,7 +198,12 @@ com.google.protobuf - protobuf-java + protobuf-java + + + + io.opentelemetry + opentelemetry-api From ebc5e92f86485a14aa4b6eeda10bab5a9582b53b Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 8 Jan 2024 19:08:08 -0800 Subject: [PATCH 002/133] Add PulsarBrokerOpenTelemetry class --- .../stats/PulsarBrokerOpenTelemetry.java | 43 ++++++++++++++ pulsar-common/pom.xml | 8 +++ .../common/stats/OpenTelemetryService.java | 58 +++++++++++++++++++ 3 files changed, 109 insertions(+) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java new file mode 100644 index 0000000000000..a0b3e42261932 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.stats; + +import io.opentelemetry.api.metrics.Meter; +import java.io.Closeable; +import lombok.Getter; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.common.stats.OpenTelemetryService; + +public class PulsarBrokerOpenTelemetry implements Closeable { + + private final OpenTelemetryService openTelemetryService; + + @Getter + private final Meter meter; + + public PulsarBrokerOpenTelemetry(PulsarService pulsar) { + this.openTelemetryService = new OpenTelemetryService(pulsar.getConfiguration().getClusterName()); + this.meter = openTelemetryService.getMeter("pulsar.broker"); + } + + @Override + public void close() { + openTelemetryService.close(); + } +} diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index 306a1966ff48a..dfa509c2cb10b 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -204,6 +204,14 @@ io.opentelemetry opentelemetry-api + ${opentelemetry.version} + + + + + io.opentelemetry + opentelemetry-sdk-extension-autoconfigure + ${opentelemetry.version} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java new file mode 100644 index 0000000000000..1166ffd54a01b --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -0,0 +1,58 @@ +/* + * 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.common.stats; + +import io.opentelemetry.api.metrics.Meter; +import io.opentelemetry.sdk.OpenTelemetrySdk; +import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdk; +import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; +import io.opentelemetry.sdk.resources.Resource; +import java.io.Closeable; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + +public class OpenTelemetryService implements Closeable { + + public static final String CLUSTER_NAME_ATTRIBUTE = "pulsar.cluster"; + + private final OpenTelemetrySdk openTelemetrySdk; + + public OpenTelemetryService(String clusterName) { + Objects.requireNonNull(clusterName); + AutoConfiguredOpenTelemetrySdkBuilder builder = AutoConfiguredOpenTelemetrySdk.builder(); + builder.addPropertiesSupplier(() -> { + Map defaultOverrides = new HashMap<>(); + defaultOverrides.put("otel.experimental.metrics.cardinality.limit", "10000"); + return defaultOverrides; + }); + builder.addResourceCustomizer( + (resource, __) -> resource.merge(Resource.builder().put(CLUSTER_NAME_ATTRIBUTE, clusterName).build())); + openTelemetrySdk = builder.build().getOpenTelemetrySdk(); + } + + public Meter getMeter(String instrumentationScopeName) { + return openTelemetrySdk.getMeter(instrumentationScopeName); + } + + @Override + public void close() { + openTelemetrySdk.close(); + } +} From 68a5760ab0bbc0b9c88afc549e00b39430df6a64 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 8 Jan 2024 20:38:31 -0800 Subject: [PATCH 003/133] Add PulsarBrokerOpenTelemetry instance to PulsarService --- .../src/main/java/org/apache/pulsar/broker/PulsarService.java | 3 +++ .../apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index a04a4c137ccbc..9ddf7d249b287 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -108,6 +108,7 @@ import org.apache.pulsar.broker.service.schema.SchemaRegistryService; import org.apache.pulsar.broker.service.schema.SchemaStorageFactory; import org.apache.pulsar.broker.stats.MetricsGenerator; +import org.apache.pulsar.broker.stats.PulsarBrokerOpenTelemetry; import org.apache.pulsar.broker.stats.prometheus.PrometheusRawMetricsProvider; import org.apache.pulsar.broker.stats.prometheus.PulsarPrometheusMetricsServlet; import org.apache.pulsar.broker.storage.ManagedLedgerStorage; @@ -248,6 +249,7 @@ public class PulsarService implements AutoCloseable, ShutdownService { private final Timer brokerClientSharedTimer; private MetricsGenerator metricsGenerator; + private PulsarBrokerOpenTelemetry openTelemetry; private TransactionMetadataStoreService transactionMetadataStoreService; private TransactionBufferProvider transactionBufferProvider; @@ -890,6 +892,7 @@ public void start() throws PulsarServerException { } this.metricsGenerator = new MetricsGenerator(this); + this.openTelemetry = new PulsarBrokerOpenTelemetry(this); // Initialize the message protocol handlers. // start the protocol handlers only after the broker is ready, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java index a0b3e42261932..ab7b2c1c4c71e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java @@ -32,7 +32,7 @@ public class PulsarBrokerOpenTelemetry implements Closeable { private final Meter meter; public PulsarBrokerOpenTelemetry(PulsarService pulsar) { - this.openTelemetryService = new OpenTelemetryService(pulsar.getConfiguration().getClusterName()); + this.openTelemetryService = new OpenTelemetryService(pulsar.getConfig().getClusterName()); this.meter = openTelemetryService.getMeter("pulsar.broker"); } From b54628cafe24860848b68b5365a94c9145cd114d Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 8 Jan 2024 20:47:27 -0800 Subject: [PATCH 004/133] Cosmetic fix --- .../pulsar/common/stats/OpenTelemetryService.java | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index 1166ffd54a01b..c3aad5a34f9c4 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -24,8 +24,7 @@ import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; import io.opentelemetry.sdk.resources.Resource; import java.io.Closeable; -import java.util.HashMap; -import java.util.Map; +import java.util.Collections; import java.util.Objects; public class OpenTelemetryService implements Closeable { @@ -37,11 +36,8 @@ public class OpenTelemetryService implements Closeable { public OpenTelemetryService(String clusterName) { Objects.requireNonNull(clusterName); AutoConfiguredOpenTelemetrySdkBuilder builder = AutoConfiguredOpenTelemetrySdk.builder(); - builder.addPropertiesSupplier(() -> { - Map defaultOverrides = new HashMap<>(); - defaultOverrides.put("otel.experimental.metrics.cardinality.limit", "10000"); - return defaultOverrides; - }); + builder.addPropertiesSupplier( + () -> Collections.singletonMap("otel.experimental.metrics.cardinality.limit", "10000")); builder.addResourceCustomizer( (resource, __) -> resource.merge(Resource.builder().put(CLUSTER_NAME_ATTRIBUTE, clusterName).build())); openTelemetrySdk = builder.build().getOpenTelemetrySdk(); From b742550d5225475c8a24fb8a84f5ed3b84642460 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 8 Jan 2024 21:40:15 -0800 Subject: [PATCH 005/133] Add worker and proxy providers --- .../stats/PulsarBrokerOpenTelemetry.java | 4 +- .../functions/instance/ContextImpl.java | 9 +++- .../stats/PulsarWorkerOpenTelemetry.java | 42 ++++++++++++++++++ .../pulsar/proxy/server/ProxyService.java | 7 +++ .../proxy/stats/PulsarProxyOpenTelemetry.java | 43 +++++++++++++++++++ 5 files changed, 102 insertions(+), 3 deletions(-) create mode 100644 pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/PulsarWorkerOpenTelemetry.java create mode 100644 pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java index ab7b2c1c4c71e..40673bb70efad 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java @@ -32,8 +32,8 @@ public class PulsarBrokerOpenTelemetry implements Closeable { private final Meter meter; public PulsarBrokerOpenTelemetry(PulsarService pulsar) { - this.openTelemetryService = new OpenTelemetryService(pulsar.getConfig().getClusterName()); - this.meter = openTelemetryService.getMeter("pulsar.broker"); + openTelemetryService = new OpenTelemetryService(pulsar.getConfig().getClusterName()); + meter = openTelemetryService.getMeter("pulsar.broker"); } @Override diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java index 6664a00510e56..a7a70a9da7105 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java @@ -69,6 +69,7 @@ import org.apache.pulsar.functions.instance.stats.ComponentStatsManager; import org.apache.pulsar.functions.instance.stats.FunctionCollectorRegistry; import org.apache.pulsar.functions.instance.stats.FunctionStatsManager; +import org.apache.pulsar.functions.instance.stats.PulsarWorkerOpenTelemetry; import org.apache.pulsar.functions.instance.stats.SinkStatsManager; import org.apache.pulsar.functions.instance.stats.SourceStatsManager; import org.apache.pulsar.functions.proto.Function; @@ -118,6 +119,7 @@ class ContextImpl implements Context, SinkContext, SourceContext, AutoCloseable Map userMetricsLabels = new HashMap<>(); private final String[] metricsLabels; private final Summary userMetricsSummary; + private final PulsarWorkerOpenTelemetry openTelemetry; private final SubscriptionType subscriptionType; @@ -223,6 +225,7 @@ public ContextImpl(InstanceConfig config, Logger logger, PulsarClient client, .quantile(0.99, 0.01) .quantile(0.999, 0.01) .create()); + this.openTelemetry = new PulsarWorkerOpenTelemetry(config); this.componentType = componentType; this.stateManager = stateManager; this.defaultStateStore = (DefaultStateStore) stateManager.getStore( @@ -731,7 +734,7 @@ public void setUnderlyingBuilder(TypedMessageBuilder underlyingBuilder) { @Override public void close() { - List futures = new LinkedList<>(); + List> futures = new LinkedList<>(); if (publishProducers != null) { for (Producer producer : publishProducers.values()) { @@ -754,6 +757,10 @@ public void close() { } catch (InterruptedException | ExecutionException e) { logger.warn("Failed to close producers", e); } + + if (openTelemetry != null) { + openTelemetry.close(); + } } @Override diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/PulsarWorkerOpenTelemetry.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/PulsarWorkerOpenTelemetry.java new file mode 100644 index 0000000000000..d8b6761372526 --- /dev/null +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/PulsarWorkerOpenTelemetry.java @@ -0,0 +1,42 @@ +/* + * 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.functions.instance.stats; + +import io.opentelemetry.api.metrics.Meter; +import java.io.Closeable; +import lombok.Getter; +import org.apache.pulsar.common.stats.OpenTelemetryService; +import org.apache.pulsar.functions.instance.InstanceConfig; + +public class PulsarWorkerOpenTelemetry implements Closeable { + + private final OpenTelemetryService openTelemetryService; + + @Getter + private final Meter meter; + + public PulsarWorkerOpenTelemetry(InstanceConfig instanceConfig) { + openTelemetryService = new OpenTelemetryService(instanceConfig.getClusterName()); + meter = openTelemetryService.getMeter("pulsar.worker"); + } + + public void close() { + openTelemetryService.close(); + } +} diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java index 719c7c2cbdade..925ca503c77e9 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java @@ -73,6 +73,7 @@ import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; import org.apache.pulsar.proxy.extensions.ProxyExtensions; +import org.apache.pulsar.proxy.stats.PulsarProxyOpenTelemetry; import org.apache.pulsar.proxy.stats.TopicStats; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -148,6 +149,8 @@ public class ProxyService implements Closeable { private PrometheusMetricsServlet metricsServlet; private List pendingMetricsProviders; + @Getter + private PulsarProxyOpenTelemetry openTelemetry; @Getter private final ConnectionController connectionController; @@ -284,6 +287,7 @@ public void start() throws Exception { } createMetricsServlet(); + openTelemetry = new PulsarProxyOpenTelemetry(this); // Initialize the message protocol handlers. // start the protocol handlers only after the broker is ready, @@ -399,6 +403,9 @@ public void close() throws IOException { proxyAdditionalServlets = null; } + if (openTelemetry != null) { + openTelemetry.close(); + } resetMetricsServlet(); if (localMetadataStore != null) { diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java new file mode 100644 index 0000000000000..475b0dd22fae4 --- /dev/null +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java @@ -0,0 +1,43 @@ +/* + * 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.proxy.stats; + +import io.opentelemetry.api.metrics.Meter; +import java.io.Closeable; +import lombok.Getter; +import org.apache.pulsar.common.stats.OpenTelemetryService; +import org.apache.pulsar.proxy.server.ProxyService; + +public class PulsarProxyOpenTelemetry implements Closeable { + + private final OpenTelemetryService openTelemetryService; + + @Getter + private final Meter meter; + + public PulsarProxyOpenTelemetry(ProxyService proxyService) { + openTelemetryService = new OpenTelemetryService(proxyService.getConfiguration().getClusterName()); + meter = openTelemetryService.getMeter("pulsar.proxy"); + } + + @Override + public void close() { + openTelemetryService.close(); + } +} From 079e429e5be20620ae6c1a9e4de9ba911d96505b Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 8 Jan 2024 22:59:23 -0800 Subject: [PATCH 006/133] Add exporters to pom --- pulsar-common/pom.xml | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index dfa509c2cb10b..e4f3bc0f621de 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -207,6 +207,17 @@ ${opentelemetry.version} + + io.opentelemetry + opentelemetry-exporter-otlp + ${opentelemetry.version} + + + + io.opentelemetry + opentelemetry-exporter-prometheus + ${opentelemetry.version}-alpha + io.opentelemetry From 5bd9ed300e5f94c65ae02460a0dfced132062ae6 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 8 Jan 2024 23:49:15 -0800 Subject: [PATCH 007/133] Add dummy unit test --- pulsar-common/pom.xml | 7 +++ .../common/stats/OpenTelemetryService.java | 16 +++++ .../stats/OpenTelemetryServiceTest.java | 47 ++++++++++++++ .../common/stats/TestMetricProvider.java | 61 +++++++++++++++++++ 4 files changed, 131 insertions(+) create mode 100644 pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java create mode 100644 pulsar-common/src/test/java/org/apache/pulsar/common/stats/TestMetricProvider.java diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index e4f3bc0f621de..258b361553ca7 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -219,6 +219,13 @@ ${opentelemetry.version}-alpha + + io.opentelemetry + opentelemetry-exporter-logging + ${opentelemetry.version} + test + + io.opentelemetry opentelemetry-sdk-extension-autoconfigure diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index c3aad5a34f9c4..6285333e872df 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -22,10 +22,15 @@ import io.opentelemetry.sdk.OpenTelemetrySdk; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdk; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; +import io.opentelemetry.sdk.autoconfigure.spi.ConfigProperties; +import io.opentelemetry.sdk.metrics.export.MetricExporter; +import io.opentelemetry.sdk.metrics.export.PeriodicMetricReader; import io.opentelemetry.sdk.resources.Resource; import java.io.Closeable; import java.util.Collections; +import java.util.Map; import java.util.Objects; +import java.util.function.BiFunction; public class OpenTelemetryService implements Closeable { @@ -34,12 +39,23 @@ public class OpenTelemetryService implements Closeable { private final OpenTelemetrySdk openTelemetrySdk; public OpenTelemetryService(String clusterName) { + this(clusterName, Collections.emptyMap()); + } + + public OpenTelemetryService(String clusterName, Map extraProperties) { Objects.requireNonNull(clusterName); AutoConfiguredOpenTelemetrySdkBuilder builder = AutoConfiguredOpenTelemetrySdk.builder(); builder.addPropertiesSupplier( () -> Collections.singletonMap("otel.experimental.metrics.cardinality.limit", "10000")); + builder.addPropertiesSupplier(() -> extraProperties); builder.addResourceCustomizer( (resource, __) -> resource.merge(Resource.builder().put(CLUSTER_NAME_ATTRIBUTE, clusterName).build())); + builder.addMetricExporterCustomizer(new BiFunction() { + @Override + public MetricExporter apply(MetricExporter metricExporter, ConfigProperties configProperties) { + return metricExporter; + } + }); openTelemetrySdk = builder.build().getOpenTelemetrySdk(); } diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java new file mode 100644 index 0000000000000..1fca53b506c63 --- /dev/null +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.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.common.stats; + +import io.opentelemetry.api.metrics.LongCounter; +import io.opentelemetry.api.metrics.Meter; +import io.opentelemetry.exporter.logging.LoggingMetricExporter; +import java.util.HashMap; +import java.util.Map; +import lombok.Cleanup; +import org.testng.annotations.Test; + +public class OpenTelemetryServiceTest { + + @Test + public void testA() throws Exception { + Object ignore = LoggingMetricExporter.class; + Map extraProperties = new HashMap<>(); + extraProperties.put("otel.metrics.exporter", "test"); + extraProperties.put("otel.metric.export.interval", "1000"); + + @Cleanup + OpenTelemetryService ots = new OpenTelemetryService("clusterName", extraProperties); + + Meter meter = ots.getMeter("pulsar.test"); + LongCounter longCounter = meter.counterBuilder("counter.A").build(); + longCounter.add(1); + + Thread.sleep(10000); + } +} diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/TestMetricProvider.java b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/TestMetricProvider.java new file mode 100644 index 0000000000000..20d49ce804a0d --- /dev/null +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/TestMetricProvider.java @@ -0,0 +1,61 @@ +/* + * 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.common.stats; + +import io.opentelemetry.sdk.autoconfigure.spi.ConfigProperties; +import io.opentelemetry.sdk.autoconfigure.spi.metrics.ConfigurableMetricExporterProvider; +import io.opentelemetry.sdk.common.CompletableResultCode; +import io.opentelemetry.sdk.metrics.InstrumentType; +import io.opentelemetry.sdk.metrics.data.AggregationTemporality; +import io.opentelemetry.sdk.metrics.data.MetricData; +import io.opentelemetry.sdk.metrics.export.MetricExporter; +import java.util.Collection; + +public class TestMetricProvider implements ConfigurableMetricExporterProvider { + + @Override + public MetricExporter createExporter(ConfigProperties config) { + return new MetricExporter() { + @Override + public CompletableResultCode export(Collection metrics) { + return CompletableResultCode.ofSuccess(); + } + + @Override + public CompletableResultCode flush() { + return CompletableResultCode.ofSuccess(); + } + + @Override + public CompletableResultCode shutdown() { + return CompletableResultCode.ofSuccess(); + } + + @Override + public AggregationTemporality getAggregationTemporality(InstrumentType instrumentType) { + return AggregationTemporality.CUMULATIVE; + } + }; + } + + @Override + public String getName() { + return "test"; + } +} From 540c6c507b38f5f0dbe1e2f6941c6a47887166e5 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 9 Jan 2024 08:48:50 -0800 Subject: [PATCH 008/133] Add test utility --- .../common/stats/OpenTelemetryService.java | 1 - .../stats/OpenTelemetryServiceTest.java | 17 ++++++++++++---- .../common/stats/TestMetricProvider.java | 20 +++++++++++++++++++ ...metrics.ConfigurableMetricExporterProvider | 1 + 4 files changed, 34 insertions(+), 5 deletions(-) create mode 100644 pulsar-common/src/test/resources/META-INF/services/io.opentelemetry.sdk.autoconfigure.spi.metrics.ConfigurableMetricExporterProvider diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index 6285333e872df..5b54da94c2371 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -24,7 +24,6 @@ import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; import io.opentelemetry.sdk.autoconfigure.spi.ConfigProperties; import io.opentelemetry.sdk.metrics.export.MetricExporter; -import io.opentelemetry.sdk.metrics.export.PeriodicMetricReader; import io.opentelemetry.sdk.resources.Resource; import java.io.Closeable; import java.util.Collections; diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java index 1fca53b506c63..06144c663870c 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java @@ -20,20 +20,28 @@ import io.opentelemetry.api.metrics.LongCounter; import io.opentelemetry.api.metrics.Meter; -import io.opentelemetry.exporter.logging.LoggingMetricExporter; +import io.opentelemetry.sdk.metrics.data.MetricData; import java.util.HashMap; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Consumer; import lombok.Cleanup; +import org.mockito.Mockito; import org.testng.annotations.Test; public class OpenTelemetryServiceTest { + public static final Map> metricConsumers = new ConcurrentHashMap<>(); + @Test public void testA() throws Exception { - Object ignore = LoggingMetricExporter.class; + Consumer consumer = (Consumer) Mockito.mock(Consumer.class); + String consumerUuid = TestMetricProvider.registerMetricConsumer(consumer); + Map extraProperties = new HashMap<>(); extraProperties.put("otel.metrics.exporter", "test"); - extraProperties.put("otel.metric.export.interval", "1000"); + extraProperties.put("otel.metric.export.interval", "100"); + extraProperties.put(TestMetricProvider.METRIC_CONSUMER_CONFIG_KEY, consumerUuid); @Cleanup OpenTelemetryService ots = new OpenTelemetryService("clusterName", extraProperties); @@ -42,6 +50,7 @@ public void testA() throws Exception { LongCounter longCounter = meter.counterBuilder("counter.A").build(); longCounter.add(1); - Thread.sleep(10000); + Mockito.verify(consumer, Mockito.timeout(1000).atLeastOnce()). + accept(Mockito.argThat((MetricData md) -> "counter.A".equals(md.getName()))); } } diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/TestMetricProvider.java b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/TestMetricProvider.java index 20d49ce804a0d..0f467db516772 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/TestMetricProvider.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/TestMetricProvider.java @@ -26,14 +26,33 @@ import io.opentelemetry.sdk.metrics.data.MetricData; import io.opentelemetry.sdk.metrics.export.MetricExporter; import java.util.Collection; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Consumer; public class TestMetricProvider implements ConfigurableMetricExporterProvider { + public static final String METRIC_CONSUMER_CONFIG_KEY = "otel.metric.export.pulsartest.consumer"; + + private static final Map> metricConsumers = new ConcurrentHashMap<>(); + + public static String registerMetricConsumer(Consumer consumer) { + String uuid = UUID.randomUUID().toString(); + metricConsumers.put(uuid, consumer); + return uuid; + } + @Override public MetricExporter createExporter(ConfigProperties config) { + final String consumerKey = config.getString(METRIC_CONSUMER_CONFIG_KEY); + final Consumer consumer = consumerKey != null ? metricConsumers.get(consumerKey) : null; return new MetricExporter() { @Override public CompletableResultCode export(Collection metrics) { + if (consumer != null) { + metrics.forEach(consumer); + } return CompletableResultCode.ofSuccess(); } @@ -44,6 +63,7 @@ public CompletableResultCode flush() { @Override public CompletableResultCode shutdown() { + metricConsumers.remove(consumerKey); return CompletableResultCode.ofSuccess(); } diff --git a/pulsar-common/src/test/resources/META-INF/services/io.opentelemetry.sdk.autoconfigure.spi.metrics.ConfigurableMetricExporterProvider b/pulsar-common/src/test/resources/META-INF/services/io.opentelemetry.sdk.autoconfigure.spi.metrics.ConfigurableMetricExporterProvider new file mode 100644 index 0000000000000..528047af4826f --- /dev/null +++ b/pulsar-common/src/test/resources/META-INF/services/io.opentelemetry.sdk.autoconfigure.spi.metrics.ConfigurableMetricExporterProvider @@ -0,0 +1 @@ +org.apache.pulsar.common.stats.TestMetricProvider \ No newline at end of file From 6268af50614c3b549a95f13a08dac2372fab80a8 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 9 Jan 2024 08:49:24 -0800 Subject: [PATCH 009/133] Remove metric exporter customizer --- .../apache/pulsar/common/stats/OpenTelemetryService.java | 9 --------- 1 file changed, 9 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index 5b54da94c2371..d7ea4ec544d43 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -22,14 +22,11 @@ import io.opentelemetry.sdk.OpenTelemetrySdk; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdk; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; -import io.opentelemetry.sdk.autoconfigure.spi.ConfigProperties; -import io.opentelemetry.sdk.metrics.export.MetricExporter; import io.opentelemetry.sdk.resources.Resource; import java.io.Closeable; import java.util.Collections; import java.util.Map; import java.util.Objects; -import java.util.function.BiFunction; public class OpenTelemetryService implements Closeable { @@ -49,12 +46,6 @@ public OpenTelemetryService(String clusterName, Map extraPropert builder.addPropertiesSupplier(() -> extraProperties); builder.addResourceCustomizer( (resource, __) -> resource.merge(Resource.builder().put(CLUSTER_NAME_ATTRIBUTE, clusterName).build())); - builder.addMetricExporterCustomizer(new BiFunction() { - @Override - public MetricExporter apply(MetricExporter metricExporter, ConfigProperties configProperties) { - return metricExporter; - } - }); openTelemetrySdk = builder.build().getOpenTelemetrySdk(); } From 7f39c0524434c164403051775311beb3edfe10de Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 9 Jan 2024 16:00:31 -0800 Subject: [PATCH 010/133] Add argument matcher for MetricData --- .../stats/OpenTelemetryServiceTest.java | 34 ++++++++++++++++--- 1 file changed, 29 insertions(+), 5 deletions(-) diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java index 06144c663870c..c402dfe2275aa 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java @@ -20,22 +20,44 @@ import io.opentelemetry.api.metrics.LongCounter; import io.opentelemetry.api.metrics.Meter; +import io.opentelemetry.sdk.metrics.data.LongPointData; import io.opentelemetry.sdk.metrics.data.MetricData; +import io.opentelemetry.sdk.metrics.data.MetricDataType; import java.util.HashMap; +import java.util.List; import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; import java.util.function.Consumer; +import lombok.Builder; import lombok.Cleanup; +import lombok.Singular; +import org.mockito.ArgumentMatcher; import org.mockito.Mockito; import org.testng.annotations.Test; public class OpenTelemetryServiceTest { - public static final Map> metricConsumers = new ConcurrentHashMap<>(); + @Builder + public static final class MetricDataArgumentMatcher implements ArgumentMatcher { + final String name; + final MetricDataType type; + + @Singular + final List longs; + + @Override + public boolean matches(MetricData md) { + boolean m = longs == null || longs.stream().mapToLong(LongPointData::getValue).allMatch( + value -> md.getLongSumData().getPoints().stream().mapToLong(LongPointData::getValue).anyMatch( + valueA -> value == valueA)); + + return (type == null || type.equals(md.getType())) + && (name == null || name.equals(md.getName())); + } + } @Test public void testA() throws Exception { - Consumer consumer = (Consumer) Mockito.mock(Consumer.class); + Consumer consumer = Mockito.mock(Consumer.class); String consumerUuid = TestMetricProvider.registerMetricConsumer(consumer); Map extraProperties = new HashMap<>(); @@ -50,7 +72,9 @@ public void testA() throws Exception { LongCounter longCounter = meter.counterBuilder("counter.A").build(); longCounter.add(1); - Mockito.verify(consumer, Mockito.timeout(1000).atLeastOnce()). - accept(Mockito.argThat((MetricData md) -> "counter.A".equals(md.getName()))); + ArgumentMatcher matcher = + MetricDataArgumentMatcher.builder().name("counter.A").type(MetricDataType.LONG_SUM).build(); + + Mockito.verify(consumer, Mockito.timeout(1000).atLeastOnce()).accept(Mockito.argThat(matcher)); } } From 70588492a1a9e20ee231668d3a626349f00ce559 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 10 Jan 2024 16:16:26 -0800 Subject: [PATCH 011/133] Add builder for OpenTelemetryService --- .../stats/PulsarBrokerOpenTelemetry.java | 2 +- .../common/stats/OpenTelemetryService.java | 12 ++++---- .../stats/OpenTelemetryServiceTest.java | 30 +++++++++---------- .../common/stats/TestMetricProvider.java | 3 +- .../stats/PulsarWorkerOpenTelemetry.java | 2 +- .../proxy/stats/PulsarProxyOpenTelemetry.java | 3 +- 6 files changed, 28 insertions(+), 24 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java index 40673bb70efad..589c4b24f4845 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java @@ -32,7 +32,7 @@ public class PulsarBrokerOpenTelemetry implements Closeable { private final Meter meter; public PulsarBrokerOpenTelemetry(PulsarService pulsar) { - openTelemetryService = new OpenTelemetryService(pulsar.getConfig().getClusterName()); + openTelemetryService = OpenTelemetryService.builder().clusterName(pulsar.getConfig().getClusterName()).build(); meter = openTelemetryService.getMeter("pulsar.broker"); } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index d7ea4ec544d43..4299150edaf93 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -27,6 +27,8 @@ import java.util.Collections; import java.util.Map; import java.util.Objects; +import lombok.Builder; +import lombok.Singular; public class OpenTelemetryService implements Closeable { @@ -34,11 +36,11 @@ public class OpenTelemetryService implements Closeable { private final OpenTelemetrySdk openTelemetrySdk; - public OpenTelemetryService(String clusterName) { - this(clusterName, Collections.emptyMap()); - } - - public OpenTelemetryService(String clusterName, Map extraProperties) { + @lombok.Builder + public OpenTelemetryService( + String clusterName, + @Singular Map extraProperties, + @Singular Map extraResources) { Objects.requireNonNull(clusterName); AutoConfiguredOpenTelemetrySdkBuilder builder = AutoConfiguredOpenTelemetrySdk.builder(); builder.addPropertiesSupplier( diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java index c402dfe2275aa..2a22af5bae17d 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java @@ -23,9 +23,7 @@ import io.opentelemetry.sdk.metrics.data.LongPointData; import io.opentelemetry.sdk.metrics.data.MetricData; import io.opentelemetry.sdk.metrics.data.MetricDataType; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.function.Consumer; import lombok.Builder; import lombok.Cleanup; @@ -40,18 +38,20 @@ public class OpenTelemetryServiceTest { public static final class MetricDataArgumentMatcher implements ArgumentMatcher { final String name; final MetricDataType type; - - @Singular - final List longs; + @Singular final List longs; @Override public boolean matches(MetricData md) { - boolean m = longs == null || longs.stream().mapToLong(LongPointData::getValue).allMatch( + return (type == null || type.equals(md.getType())) + && (name == null || name.equals(md.getName())) + && matchesLongPointData(md); + } + + private boolean matchesLongPointData(MetricData md) { + return longs == null + || longs.stream().mapToLong(LongPointData::getValue).allMatch( value -> md.getLongSumData().getPoints().stream().mapToLong(LongPointData::getValue).anyMatch( valueA -> value == valueA)); - - return (type == null || type.equals(md.getType())) - && (name == null || name.equals(md.getName())); } } @@ -60,13 +60,13 @@ public void testA() throws Exception { Consumer consumer = Mockito.mock(Consumer.class); String consumerUuid = TestMetricProvider.registerMetricConsumer(consumer); - Map extraProperties = new HashMap<>(); - extraProperties.put("otel.metrics.exporter", "test"); - extraProperties.put("otel.metric.export.interval", "100"); - extraProperties.put(TestMetricProvider.METRIC_CONSUMER_CONFIG_KEY, consumerUuid); - @Cleanup - OpenTelemetryService ots = new OpenTelemetryService("clusterName", extraProperties); + OpenTelemetryService ots = OpenTelemetryService.builder(). + clusterName("clusterName"). + extraProperty("otel.metric.export.interval", "100"). + extraProperty("otel.metrics.exporter", TestMetricProvider.METRIC_PROVIDER_NAME). + extraProperty(TestMetricProvider.METRIC_CONSUMER_CONFIG_KEY, consumerUuid). + build(); Meter meter = ots.getMeter("pulsar.test"); LongCounter longCounter = meter.counterBuilder("counter.A").build(); diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/TestMetricProvider.java b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/TestMetricProvider.java index 0f467db516772..0e5cb64caf5bb 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/TestMetricProvider.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/TestMetricProvider.java @@ -33,6 +33,7 @@ public class TestMetricProvider implements ConfigurableMetricExporterProvider { + public static final String METRIC_PROVIDER_NAME = "test"; public static final String METRIC_CONSUMER_CONFIG_KEY = "otel.metric.export.pulsartest.consumer"; private static final Map> metricConsumers = new ConcurrentHashMap<>(); @@ -76,6 +77,6 @@ public AggregationTemporality getAggregationTemporality(InstrumentType instrumen @Override public String getName() { - return "test"; + return METRIC_PROVIDER_NAME; } } diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/PulsarWorkerOpenTelemetry.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/PulsarWorkerOpenTelemetry.java index d8b6761372526..9f2a28dda8d9d 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/PulsarWorkerOpenTelemetry.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/PulsarWorkerOpenTelemetry.java @@ -32,7 +32,7 @@ public class PulsarWorkerOpenTelemetry implements Closeable { private final Meter meter; public PulsarWorkerOpenTelemetry(InstanceConfig instanceConfig) { - openTelemetryService = new OpenTelemetryService(instanceConfig.getClusterName()); + openTelemetryService = OpenTelemetryService.builder().clusterName(instanceConfig.getClusterName()).build(); meter = openTelemetryService.getMeter("pulsar.worker"); } diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java index 475b0dd22fae4..da31a52bf78df 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java @@ -32,7 +32,8 @@ public class PulsarProxyOpenTelemetry implements Closeable { private final Meter meter; public PulsarProxyOpenTelemetry(ProxyService proxyService) { - openTelemetryService = new OpenTelemetryService(proxyService.getConfiguration().getClusterName()); + openTelemetryService = + OpenTelemetryService.builder().clusterName(proxyService.getConfiguration().getClusterName()).build(); meter = openTelemetryService.getMeter("pulsar.proxy"); } From 96006c416c29b8987b17ab0f1398aaed29b06a45 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 10 Jan 2024 16:46:31 -0800 Subject: [PATCH 012/133] Add cardinality limit test --- .../common/stats/OpenTelemetryService.java | 5 ++- .../stats/OpenTelemetryServiceTest.java | 42 ++++++++++++++++++- 2 files changed, 45 insertions(+), 2 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index 4299150edaf93..76957575d5d3c 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -36,6 +36,9 @@ public class OpenTelemetryService implements Closeable { private final OpenTelemetrySdk openTelemetrySdk; + private static final String MAX_CARDINALITY_LIMIT_KEY = "otel.experimental.metrics.cardinality.limit"; + public static final int MAX_CARDINALITY_LIMIT = 10000; + @lombok.Builder public OpenTelemetryService( String clusterName, @@ -44,7 +47,7 @@ public OpenTelemetryService( Objects.requireNonNull(clusterName); AutoConfiguredOpenTelemetrySdkBuilder builder = AutoConfiguredOpenTelemetrySdk.builder(); builder.addPropertiesSupplier( - () -> Collections.singletonMap("otel.experimental.metrics.cardinality.limit", "10000")); + () -> Collections.singletonMap(MAX_CARDINALITY_LIMIT_KEY, Integer.toString(MAX_CARDINALITY_LIMIT + 1))); builder.addPropertiesSupplier(() -> extraProperties); builder.addResourceCustomizer( (resource, __) -> resource.merge(Resource.builder().put(CLUSTER_NAME_ATTRIBUTE, clusterName).build())); diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java index 2a22af5bae17d..e31b13c47e205 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java @@ -18,11 +18,15 @@ */ package org.apache.pulsar.common.stats; +import io.opentelemetry.api.common.AttributeKey; +import io.opentelemetry.api.common.Attributes; import io.opentelemetry.api.metrics.LongCounter; import io.opentelemetry.api.metrics.Meter; import io.opentelemetry.sdk.metrics.data.LongPointData; import io.opentelemetry.sdk.metrics.data.MetricData; import io.opentelemetry.sdk.metrics.data.MetricDataType; +import io.opentelemetry.sdk.metrics.data.PointData; +import io.opentelemetry.sdk.metrics.internal.state.MetricStorage; import java.util.List; import java.util.function.Consumer; import lombok.Builder; @@ -38,15 +42,22 @@ public class OpenTelemetryServiceTest { public static final class MetricDataArgumentMatcher implements ArgumentMatcher { final String name; final MetricDataType type; + final Attributes attributes; @Singular final List longs; @Override public boolean matches(MetricData md) { return (type == null || type.equals(md.getType())) && (name == null || name.equals(md.getName())) + && matchesAttributes(md) && matchesLongPointData(md); } + private boolean matchesAttributes(MetricData md) { + return attributes == null + || md.getData().getPoints().stream().map(PointData::getAttributes).anyMatch(attributes::equals); + } + private boolean matchesLongPointData(MetricData md) { return longs == null || longs.stream().mapToLong(LongPointData::getValue).allMatch( @@ -56,7 +67,7 @@ private boolean matchesLongPointData(MetricData md) { } @Test - public void testA() throws Exception { + public void testMetricExport() throws Exception { Consumer consumer = Mockito.mock(Consumer.class); String consumerUuid = TestMetricProvider.registerMetricConsumer(consumer); @@ -77,4 +88,33 @@ public void testA() throws Exception { Mockito.verify(consumer, Mockito.timeout(1000).atLeastOnce()).accept(Mockito.argThat(matcher)); } + + @Test + public void testMetricCardinality() throws Exception { + Consumer consumer = Mockito.mock(Consumer.class); + String consumerUuid = TestMetricProvider.registerMetricConsumer(consumer); + + @Cleanup + OpenTelemetryService ots = OpenTelemetryService.builder(). + clusterName("clusterName"). + extraProperty("otel.metric.export.interval", "100"). + extraProperty("otel.metrics.exporter", TestMetricProvider.METRIC_PROVIDER_NAME). + extraProperty(TestMetricProvider.METRIC_CONSUMER_CONFIG_KEY, consumerUuid). + build(); + + Meter meter = ots.getMeter("pulsar.testMetricCardinality"); + LongCounter longCounter = meter.counterBuilder("count").build(); + ArgumentMatcher matcher = + MetricDataArgumentMatcher.builder().name("count").attributes(MetricStorage.CARDINALITY_OVERFLOW).build(); + + for (int i = 0; i < OpenTelemetryService.MAX_CARDINALITY_LIMIT; i++) { + longCounter.add(1, Attributes.of(AttributeKey.stringKey("attribute"), "value" + i)); + } + Mockito.verify(consumer, Mockito.never()).accept(Mockito.argThat(matcher)); + + for (int i = 0; i < OpenTelemetryService.MAX_CARDINALITY_LIMIT + 1; i++) { + longCounter.add(1, Attributes.of(AttributeKey.stringKey("attribute"), "value" + i)); + } + Mockito.verify(consumer, Mockito.timeout(1000).atLeastOnce()).accept(Mockito.argThat(matcher)); + } } From 800f31b6e1e9ca9c16c762303acedf3756941e86 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 10 Jan 2024 19:03:40 -0800 Subject: [PATCH 013/133] Fix test --- .../pulsar/common/stats/OpenTelemetryServiceTest.java | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java index e31b13c47e205..b1d1d6b39853a 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java @@ -28,12 +28,15 @@ import io.opentelemetry.sdk.metrics.data.PointData; import io.opentelemetry.sdk.metrics.internal.state.MetricStorage; import java.util.List; +import java.util.concurrent.CountDownLatch; import java.util.function.Consumer; import lombok.Builder; import lombok.Cleanup; import lombok.Singular; import org.mockito.ArgumentMatcher; import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; import org.testng.annotations.Test; public class OpenTelemetryServiceTest { @@ -110,6 +113,13 @@ public void testMetricCardinality() throws Exception { for (int i = 0; i < OpenTelemetryService.MAX_CARDINALITY_LIMIT; i++) { longCounter.add(1, Attributes.of(AttributeKey.stringKey("attribute"), "value" + i)); } + + CountDownLatch cdl = new CountDownLatch(1); // Wait for at least one metric batch to be exported + Mockito.doAnswer(invocation -> { + cdl.countDown(); + return null; + }).when(consumer).accept(Mockito.any()); + cdl.await(); Mockito.verify(consumer, Mockito.never()).accept(Mockito.argThat(matcher)); for (int i = 0; i < OpenTelemetryService.MAX_CARDINALITY_LIMIT + 1; i++) { From 89c094eb9138c3768d6ea20ca9a1404c885ddf44 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 11 Jan 2024 20:24:07 -0800 Subject: [PATCH 014/133] Add OpenTelemetry SDK testing pom --- pulsar-common/pom.xml | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index 258b361553ca7..3265ba2aa24b0 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -269,6 +269,13 @@ awaitility test + + + io.opentelemetry + opentelemetry-sdk-testing + ${opentelemetry.version} + test + From e31e4284b0cdfce1205630f7a2fa34a63949693e Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 17 Jan 2024 20:59:49 -0800 Subject: [PATCH 015/133] Use InMemoryReader for tests --- .../common/stats/OpenTelemetryService.java | 14 +++- .../stats/OpenTelemetryServiceTest.java | 77 +++++++++---------- 2 files changed, 48 insertions(+), 43 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index 76957575d5d3c..dc6a3ea91a769 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -22,12 +22,15 @@ import io.opentelemetry.sdk.OpenTelemetrySdk; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdk; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; +import io.opentelemetry.sdk.metrics.export.MetricReader; +import io.opentelemetry.sdk.metrics.internal.SdkMeterProviderUtil; +import io.opentelemetry.sdk.metrics.internal.export.CardinalityLimitSelector; import io.opentelemetry.sdk.resources.Resource; import java.io.Closeable; import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.Objects; -import lombok.Builder; import lombok.Singular; public class OpenTelemetryService implements Closeable { @@ -43,7 +46,8 @@ public class OpenTelemetryService implements Closeable { public OpenTelemetryService( String clusterName, @Singular Map extraProperties, - @Singular Map extraResources) { + @Singular Map extraResources, + @Singular List extraMetricReaders) { Objects.requireNonNull(clusterName); AutoConfiguredOpenTelemetrySdkBuilder builder = AutoConfiguredOpenTelemetrySdk.builder(); builder.addPropertiesSupplier( @@ -51,6 +55,12 @@ public OpenTelemetryService( builder.addPropertiesSupplier(() -> extraProperties); builder.addResourceCustomizer( (resource, __) -> resource.merge(Resource.builder().put(CLUSTER_NAME_ATTRIBUTE, clusterName).build())); + final CardinalityLimitSelector cardinalityLimitSelector = __ -> MAX_CARDINALITY_LIMIT + 1; + extraMetricReaders.forEach(metricReader -> builder.addMeterProviderCustomizer((sdkMeterProviderBuilder, __) -> { + SdkMeterProviderUtil.registerMetricReaderWithCardinalitySelector( + sdkMeterProviderBuilder, metricReader, cardinalityLimitSelector); + return sdkMeterProviderBuilder; + })); openTelemetrySdk = builder.build().getOpenTelemetrySdk(); } diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java index b1d1d6b39853a..3346119696b76 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java @@ -27,33 +27,31 @@ import io.opentelemetry.sdk.metrics.data.MetricDataType; import io.opentelemetry.sdk.metrics.data.PointData; import io.opentelemetry.sdk.metrics.internal.state.MetricStorage; +import io.opentelemetry.sdk.testing.exporter.InMemoryMetricReader; +import java.util.Collection; import java.util.List; -import java.util.concurrent.CountDownLatch; -import java.util.function.Consumer; +import java.util.function.Predicate; import lombok.Builder; import lombok.Cleanup; import lombok.Singular; -import org.mockito.ArgumentMatcher; -import org.mockito.Mockito; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; +import org.testng.Assert; import org.testng.annotations.Test; public class OpenTelemetryServiceTest { @Builder - public static final class MetricDataArgumentMatcher implements ArgumentMatcher { + public static final class MetricDataMatchPredicate implements Predicate { final String name; final MetricDataType type; final Attributes attributes; - @Singular final List longs; + @Singular final List longValues; @Override - public boolean matches(MetricData md) { + public boolean test(MetricData md) { return (type == null || type.equals(md.getType())) && (name == null || name.equals(md.getName())) && matchesAttributes(md) - && matchesLongPointData(md); + && matchesLongValues(md); } private boolean matchesAttributes(MetricData md) { @@ -61,70 +59,67 @@ private boolean matchesAttributes(MetricData md) { || md.getData().getPoints().stream().map(PointData::getAttributes).anyMatch(attributes::equals); } - private boolean matchesLongPointData(MetricData md) { - return longs == null - || longs.stream().mapToLong(LongPointData::getValue).allMatch( + private boolean matchesLongValues(MetricData md) { + return longValues == null || longValues.stream().allMatch( value -> md.getLongSumData().getPoints().stream().mapToLong(LongPointData::getValue).anyMatch( valueA -> value == valueA)); } } @Test - public void testMetricExport() throws Exception { - Consumer consumer = Mockito.mock(Consumer.class); - String consumerUuid = TestMetricProvider.registerMetricConsumer(consumer); + public void testInMemoryReader() throws Exception { + @Cleanup + InMemoryMetricReader reader = InMemoryMetricReader.create(); @Cleanup - OpenTelemetryService ots = OpenTelemetryService.builder(). - clusterName("clusterName"). - extraProperty("otel.metric.export.interval", "100"). - extraProperty("otel.metrics.exporter", TestMetricProvider.METRIC_PROVIDER_NAME). - extraProperty(TestMetricProvider.METRIC_CONSUMER_CONFIG_KEY, consumerUuid). - build(); + OpenTelemetryService ots = + OpenTelemetryService.builder().clusterName("clusterName").extraMetricReader(reader).build(); Meter meter = ots.getMeter("pulsar.test"); - LongCounter longCounter = meter.counterBuilder("counter.A").build(); + LongCounter longCounter = meter.counterBuilder("counter.inMemory").build(); longCounter.add(1); - ArgumentMatcher matcher = - MetricDataArgumentMatcher.builder().name("counter.A").type(MetricDataType.LONG_SUM).build(); + Predicate predicate = MetricDataMatchPredicate.builder(). + name("counter.inMemory"). + type(MetricDataType.LONG_SUM). + longValue(1L). + build(); - Mockito.verify(consumer, Mockito.timeout(1000).atLeastOnce()).accept(Mockito.argThat(matcher)); + Collection metricData = reader.collectAllMetrics(); + Assert.assertTrue(metricData.stream().anyMatch(predicate)); } @Test public void testMetricCardinality() throws Exception { - Consumer consumer = Mockito.mock(Consumer.class); - String consumerUuid = TestMetricProvider.registerMetricConsumer(consumer); + @Cleanup + InMemoryMetricReader reader = InMemoryMetricReader.create(); @Cleanup OpenTelemetryService ots = OpenTelemetryService.builder(). clusterName("clusterName"). - extraProperty("otel.metric.export.interval", "100"). - extraProperty("otel.metrics.exporter", TestMetricProvider.METRIC_PROVIDER_NAME). - extraProperty(TestMetricProvider.METRIC_CONSUMER_CONFIG_KEY, consumerUuid). + extraMetricReader(reader). build(); Meter meter = ots.getMeter("pulsar.testMetricCardinality"); LongCounter longCounter = meter.counterBuilder("count").build(); - ArgumentMatcher matcher = - MetricDataArgumentMatcher.builder().name("count").attributes(MetricStorage.CARDINALITY_OVERFLOW).build(); for (int i = 0; i < OpenTelemetryService.MAX_CARDINALITY_LIMIT; i++) { longCounter.add(1, Attributes.of(AttributeKey.stringKey("attribute"), "value" + i)); } - CountDownLatch cdl = new CountDownLatch(1); // Wait for at least one metric batch to be exported - Mockito.doAnswer(invocation -> { - cdl.countDown(); - return null; - }).when(consumer).accept(Mockito.any()); - cdl.await(); - Mockito.verify(consumer, Mockito.never()).accept(Mockito.argThat(matcher)); + Predicate hasOverflowAttribute = MetricDataMatchPredicate.builder(). + name("count"). + attributes(MetricStorage.CARDINALITY_OVERFLOW). + build(); + + Collection metricData = reader.collectAllMetrics(); + Assert.assertTrue(metricData.stream().noneMatch(hasOverflowAttribute)); for (int i = 0; i < OpenTelemetryService.MAX_CARDINALITY_LIMIT + 1; i++) { longCounter.add(1, Attributes.of(AttributeKey.stringKey("attribute"), "value" + i)); } - Mockito.verify(consumer, Mockito.timeout(1000).atLeastOnce()).accept(Mockito.argThat(matcher)); + + metricData = reader.collectAllMetrics(); + Assert.assertTrue(metricData.stream().anyMatch(hasOverflowAttribute)); } } From 67d022647494346db81a9fd7c3cf3c2a477d9b5f Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 17 Jan 2024 21:00:42 -0800 Subject: [PATCH 016/133] Remove TestMetricProvider --- .../common/stats/TestMetricProvider.java | 82 ------------------- ...metrics.ConfigurableMetricExporterProvider | 1 - 2 files changed, 83 deletions(-) delete mode 100644 pulsar-common/src/test/java/org/apache/pulsar/common/stats/TestMetricProvider.java delete mode 100644 pulsar-common/src/test/resources/META-INF/services/io.opentelemetry.sdk.autoconfigure.spi.metrics.ConfigurableMetricExporterProvider diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/TestMetricProvider.java b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/TestMetricProvider.java deleted file mode 100644 index 0e5cb64caf5bb..0000000000000 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/TestMetricProvider.java +++ /dev/null @@ -1,82 +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.common.stats; - -import io.opentelemetry.sdk.autoconfigure.spi.ConfigProperties; -import io.opentelemetry.sdk.autoconfigure.spi.metrics.ConfigurableMetricExporterProvider; -import io.opentelemetry.sdk.common.CompletableResultCode; -import io.opentelemetry.sdk.metrics.InstrumentType; -import io.opentelemetry.sdk.metrics.data.AggregationTemporality; -import io.opentelemetry.sdk.metrics.data.MetricData; -import io.opentelemetry.sdk.metrics.export.MetricExporter; -import java.util.Collection; -import java.util.Map; -import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; -import java.util.function.Consumer; - -public class TestMetricProvider implements ConfigurableMetricExporterProvider { - - public static final String METRIC_PROVIDER_NAME = "test"; - public static final String METRIC_CONSUMER_CONFIG_KEY = "otel.metric.export.pulsartest.consumer"; - - private static final Map> metricConsumers = new ConcurrentHashMap<>(); - - public static String registerMetricConsumer(Consumer consumer) { - String uuid = UUID.randomUUID().toString(); - metricConsumers.put(uuid, consumer); - return uuid; - } - - @Override - public MetricExporter createExporter(ConfigProperties config) { - final String consumerKey = config.getString(METRIC_CONSUMER_CONFIG_KEY); - final Consumer consumer = consumerKey != null ? metricConsumers.get(consumerKey) : null; - return new MetricExporter() { - @Override - public CompletableResultCode export(Collection metrics) { - if (consumer != null) { - metrics.forEach(consumer); - } - return CompletableResultCode.ofSuccess(); - } - - @Override - public CompletableResultCode flush() { - return CompletableResultCode.ofSuccess(); - } - - @Override - public CompletableResultCode shutdown() { - metricConsumers.remove(consumerKey); - return CompletableResultCode.ofSuccess(); - } - - @Override - public AggregationTemporality getAggregationTemporality(InstrumentType instrumentType) { - return AggregationTemporality.CUMULATIVE; - } - }; - } - - @Override - public String getName() { - return METRIC_PROVIDER_NAME; - } -} diff --git a/pulsar-common/src/test/resources/META-INF/services/io.opentelemetry.sdk.autoconfigure.spi.metrics.ConfigurableMetricExporterProvider b/pulsar-common/src/test/resources/META-INF/services/io.opentelemetry.sdk.autoconfigure.spi.metrics.ConfigurableMetricExporterProvider deleted file mode 100644 index 528047af4826f..0000000000000 --- a/pulsar-common/src/test/resources/META-INF/services/io.opentelemetry.sdk.autoconfigure.spi.metrics.ConfigurableMetricExporterProvider +++ /dev/null @@ -1 +0,0 @@ -org.apache.pulsar.common.stats.TestMetricProvider \ No newline at end of file From f7844b6c4ba99aa36697bb3cd84fcedc5ee6b86a Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 17 Jan 2024 22:46:52 -0800 Subject: [PATCH 017/133] Add serviceName parameter --- .../common/stats/OpenTelemetryService.java | 20 +++++-- .../stats/OpenTelemetryServiceTest.java | 60 ++++++++++++++----- 2 files changed, 60 insertions(+), 20 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index dc6a3ea91a769..9599259294137 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.common.stats; +import io.opentelemetry.api.common.AttributeKey; import io.opentelemetry.api.metrics.Meter; import io.opentelemetry.sdk.OpenTelemetrySdk; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdk; @@ -26,6 +27,7 @@ import io.opentelemetry.sdk.metrics.internal.SdkMeterProviderUtil; import io.opentelemetry.sdk.metrics.internal.export.CardinalityLimitSelector; import io.opentelemetry.sdk.resources.Resource; +import io.opentelemetry.sdk.resources.ResourceBuilder; import java.io.Closeable; import java.util.Collections; import java.util.List; @@ -35,18 +37,19 @@ public class OpenTelemetryService implements Closeable { - public static final String CLUSTER_NAME_ATTRIBUTE = "pulsar.cluster"; - - private final OpenTelemetrySdk openTelemetrySdk; + private static final AttributeKey CLUSTER_NAME_ATTRIBUTE = AttributeKey.stringKey("pulsar.cluster"); + private static final AttributeKey SERVICE_NAME_ATTRIBUTE = AttributeKey.stringKey("service.name"); private static final String MAX_CARDINALITY_LIMIT_KEY = "otel.experimental.metrics.cardinality.limit"; public static final int MAX_CARDINALITY_LIMIT = 10000; + private final OpenTelemetrySdk openTelemetrySdk; + @lombok.Builder public OpenTelemetryService( String clusterName, + String serviceName, @Singular Map extraProperties, - @Singular Map extraResources, @Singular List extraMetricReaders) { Objects.requireNonNull(clusterName); AutoConfiguredOpenTelemetrySdkBuilder builder = AutoConfiguredOpenTelemetrySdk.builder(); @@ -54,7 +57,14 @@ public OpenTelemetryService( () -> Collections.singletonMap(MAX_CARDINALITY_LIMIT_KEY, Integer.toString(MAX_CARDINALITY_LIMIT + 1))); builder.addPropertiesSupplier(() -> extraProperties); builder.addResourceCustomizer( - (resource, __) -> resource.merge(Resource.builder().put(CLUSTER_NAME_ATTRIBUTE, clusterName).build())); + (resource, __) -> { + ResourceBuilder resourceBuilder = Resource.builder(); + resourceBuilder.put(CLUSTER_NAME_ATTRIBUTE, clusterName); + if (serviceName != null) { + resourceBuilder.put(SERVICE_NAME_ATTRIBUTE, serviceName); + } + return resource.merge(resourceBuilder.build()); + }); final CardinalityLimitSelector cardinalityLimitSelector = __ -> MAX_CARDINALITY_LIMIT + 1; extraMetricReaders.forEach(metricReader -> builder.addMeterProviderCustomizer((sdkMeterProviderBuilder, __) -> { SdkMeterProviderUtil.registerMetricReaderWithCardinalitySelector( diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java index 3346119696b76..cc954f9d7194e 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java @@ -22,15 +22,18 @@ import io.opentelemetry.api.common.Attributes; import io.opentelemetry.api.metrics.LongCounter; import io.opentelemetry.api.metrics.Meter; +import io.opentelemetry.sdk.common.InstrumentationScopeInfo; import io.opentelemetry.sdk.metrics.data.LongPointData; import io.opentelemetry.sdk.metrics.data.MetricData; import io.opentelemetry.sdk.metrics.data.MetricDataType; import io.opentelemetry.sdk.metrics.data.PointData; import io.opentelemetry.sdk.metrics.internal.state.MetricStorage; +import io.opentelemetry.sdk.resources.Resource; import io.opentelemetry.sdk.testing.exporter.InMemoryMetricReader; import java.util.Collection; import java.util.List; import java.util.function.Predicate; +import java.util.stream.Collectors; import lombok.Builder; import lombok.Cleanup; import lombok.Singular; @@ -43,26 +46,46 @@ public class OpenTelemetryServiceTest { public static final class MetricDataMatchPredicate implements Predicate { final String name; final MetricDataType type; - final Attributes attributes; + final InstrumentationScopeInfo instrumentationScopeInfo; + final Resource resource; + @Singular final List resourceAttributes; + @Singular final List dataAttributes; @Singular final List longValues; @Override public boolean test(MetricData md) { - return (type == null || type.equals(md.getType())) - && (name == null || name.equals(md.getName())) - && matchesAttributes(md) - && matchesLongValues(md); + return (name == null || name.equals(md.getName())) + && (type == null || type.equals(md.getType())) + && (instrumentationScopeInfo == null + || instrumentationScopeInfo.equals(md.getInstrumentationScopeInfo())) + && (resource == null || resource.equals(md.getResource())) + && matchesResourceAttributes(md) + && (dataAttributes == null || matchesDataAttributes(md)) + && matchesLongValues(md); } - private boolean matchesAttributes(MetricData md) { - return attributes == null - || md.getData().getPoints().stream().map(PointData::getAttributes).anyMatch(attributes::equals); + private boolean matchesResourceAttributes(MetricData md) { + Attributes actual = md.getResource().getAttributes(); + return resourceAttributes.stream().allMatch(expected -> matchesAttributes(actual, expected)); + } + + private boolean matchesDataAttributes(MetricData md) { + Collection actuals = + md.getData().getPoints().stream().map(PointData::getAttributes).collect(Collectors.toSet()); + return dataAttributes.stream(). + allMatch(expected -> actuals.stream().anyMatch(actual -> matchesAttributes(actual, expected))); + } + + private boolean matchesAttributes(Attributes actual, Attributes expected) { + // Return true iff all attribute pairs in expected are a subset of those in actual. Allows tests to specify + // just the attributes they care about, insted of exhaustively having to list all of them. + return expected.asMap().entrySet().stream().allMatch(e -> e.getValue().equals(actual.get(e.getKey()))); } private boolean matchesLongValues(MetricData md) { - return longValues == null || longValues.stream().allMatch( - value -> md.getLongSumData().getPoints().stream().mapToLong(LongPointData::getValue).anyMatch( - valueA -> value == valueA)); + Collection actualData = + md.getLongSumData().getPoints().stream().map(LongPointData::getValue).collect(Collectors.toSet()); + return actualData.containsAll(longValues); } } @@ -72,15 +95,22 @@ public void testInMemoryReader() throws Exception { InMemoryMetricReader reader = InMemoryMetricReader.create(); @Cleanup - OpenTelemetryService ots = - OpenTelemetryService.builder().clusterName("clusterName").extraMetricReader(reader).build(); + OpenTelemetryService ots = OpenTelemetryService.builder(). + clusterName("clusterName"). + serviceName("testInMemoryReader"). + extraMetricReader(reader). + build(); Meter meter = ots.getMeter("pulsar.test"); LongCounter longCounter = meter.counterBuilder("counter.inMemory").build(); - longCounter.add(1); + longCounter.add(1, Attributes.of(AttributeKey.stringKey("dummyAttr"), "dummyValue")); Predicate predicate = MetricDataMatchPredicate.builder(). name("counter.inMemory"). + instrumentationScopeInfo(InstrumentationScopeInfo.create("pulsar.test")). + resourceAttribute(Attributes.of(AttributeKey.stringKey("pulsar.cluster"), "clusterName")). + resourceAttribute(Attributes.of(AttributeKey.stringKey("service.name"), "testInMemoryReader")). + dataAttribute(Attributes.of(AttributeKey.stringKey("dummyAttr"), "dummyValue")). type(MetricDataType.LONG_SUM). longValue(1L). build(); @@ -109,7 +139,7 @@ public void testMetricCardinality() throws Exception { Predicate hasOverflowAttribute = MetricDataMatchPredicate.builder(). name("count"). - attributes(MetricStorage.CARDINALITY_OVERFLOW). + dataAttribute(MetricStorage.CARDINALITY_OVERFLOW). build(); Collection metricData = reader.collectAllMetrics(); From 7b485ca8d988b448128f8cadd3170da413a95b39 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 18 Jan 2024 00:02:34 -0800 Subject: [PATCH 018/133] Update tests --- .../stats/OpenTelemetryServiceTest.java | 122 +++++++++++++++--- 1 file changed, 102 insertions(+), 20 deletions(-) diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java index cc954f9d7194e..a365d6fc2ab4e 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java @@ -20,9 +20,11 @@ import io.opentelemetry.api.common.AttributeKey; import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.metrics.DoubleCounter; import io.opentelemetry.api.metrics.LongCounter; import io.opentelemetry.api.metrics.Meter; import io.opentelemetry.sdk.common.InstrumentationScopeInfo; +import io.opentelemetry.sdk.metrics.data.DoublePointData; import io.opentelemetry.sdk.metrics.data.LongPointData; import io.opentelemetry.sdk.metrics.data.MetricData; import io.opentelemetry.sdk.metrics.data.MetricDataType; @@ -38,6 +40,8 @@ import lombok.Cleanup; import lombok.Singular; import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; public class OpenTelemetryServiceTest { @@ -51,6 +55,7 @@ public static final class MetricDataMatchPredicate implements Predicate resourceAttributes; @Singular final List dataAttributes; @Singular final List longValues; + @Singular final List doubleValues; @Override public boolean test(MetricData md) { @@ -61,7 +66,8 @@ public boolean test(MetricData md) { && (resource == null || resource.equals(md.getResource())) && matchesResourceAttributes(md) && (dataAttributes == null || matchesDataAttributes(md)) - && matchesLongValues(md); + && matchesLongValues(md) + && matchesDoubleValues(md); } private boolean matchesResourceAttributes(MetricData md) { @@ -87,32 +93,53 @@ private boolean matchesLongValues(MetricData md) { md.getLongSumData().getPoints().stream().map(LongPointData::getValue).collect(Collectors.toSet()); return actualData.containsAll(longValues); } + + private boolean matchesDoubleValues(MetricData md) { + Collection actualData = + md.getDoubleSumData().getPoints().stream().map(DoublePointData::getValue).collect(Collectors.toSet()); + return actualData.containsAll(doubleValues); + } + } + + private OpenTelemetryService openTelemetryService; + private InMemoryMetricReader reader; + private Meter meter; + + @BeforeMethod + public void setup() throws Exception { + reader = InMemoryMetricReader.create(); + openTelemetryService = OpenTelemetryService.builder(). + clusterName("openTelemetryServiceTestCluster"). + extraMetricReader(reader). + build(); + meter = openTelemetryService.getMeter("openTelemetryServiceTestInstrument"); + } + + @AfterMethod + public void teardown() throws Exception { + openTelemetryService.close(); + reader.close(); + } + + @Test(expectedExceptions = NullPointerException.class) + public void testIsClusterNameRequired() throws Exception { + @Cleanup + OpenTelemetryService ots = OpenTelemetryService.builder().build(); } @Test - public void testInMemoryReader() throws Exception { + public void testIsClusterNameSet() throws Exception { @Cleanup InMemoryMetricReader reader = InMemoryMetricReader.create(); @Cleanup OpenTelemetryService ots = OpenTelemetryService.builder(). - clusterName("clusterName"). - serviceName("testInMemoryReader"). + clusterName("testCluster"). extraMetricReader(reader). build(); - Meter meter = ots.getMeter("pulsar.test"); - LongCounter longCounter = meter.counterBuilder("counter.inMemory").build(); - longCounter.add(1, Attributes.of(AttributeKey.stringKey("dummyAttr"), "dummyValue")); - Predicate predicate = MetricDataMatchPredicate.builder(). - name("counter.inMemory"). - instrumentationScopeInfo(InstrumentationScopeInfo.create("pulsar.test")). - resourceAttribute(Attributes.of(AttributeKey.stringKey("pulsar.cluster"), "clusterName")). - resourceAttribute(Attributes.of(AttributeKey.stringKey("service.name"), "testInMemoryReader")). - dataAttribute(Attributes.of(AttributeKey.stringKey("dummyAttr"), "dummyValue")). - type(MetricDataType.LONG_SUM). - longValue(1L). + resourceAttribute(Attributes.of(AttributeKey.stringKey("pulsar.cluster"), "testCluster")). build(); Collection metricData = reader.collectAllMetrics(); @@ -120,25 +147,47 @@ public void testInMemoryReader() throws Exception { } @Test - public void testMetricCardinality() throws Exception { + public void testIsServiceNameSet() throws Exception { @Cleanup InMemoryMetricReader reader = InMemoryMetricReader.create(); @Cleanup OpenTelemetryService ots = OpenTelemetryService.builder(). - clusterName("clusterName"). + clusterName("testCluster"). + serviceName("testServiceName"). extraMetricReader(reader). build(); - Meter meter = ots.getMeter("pulsar.testMetricCardinality"); - LongCounter longCounter = meter.counterBuilder("count").build(); + Predicate predicate = MetricDataMatchPredicate.builder(). + resourceAttribute(Attributes.of(AttributeKey.stringKey("service.name"), "testServiceName")). + build(); + + Collection metricData = reader.collectAllMetrics(); + Assert.assertTrue(metricData.stream().anyMatch(predicate)); + } + + @Test + public void testIsInstrumentationNameSetOnMeter() throws Exception { + Meter meter = openTelemetryService.getMeter("testInstrumentationScope"); + meter.counterBuilder("dummyCounter").build().add(1); + MetricDataMatchPredicate predicate = MetricDataMatchPredicate.builder(). + name("dummyCounter"). + instrumentationScopeInfo(InstrumentationScopeInfo.create("testInstrumentationScope")). + build(); + Collection metricData = reader.collectAllMetrics(); + Assert.assertTrue(metricData.stream().anyMatch(predicate)); + } + + @Test + public void testMetricCardinality() throws Exception { + LongCounter longCounter = meter.counterBuilder("dummyMetricCardinalityTest").build(); for (int i = 0; i < OpenTelemetryService.MAX_CARDINALITY_LIMIT; i++) { longCounter.add(1, Attributes.of(AttributeKey.stringKey("attribute"), "value" + i)); } Predicate hasOverflowAttribute = MetricDataMatchPredicate.builder(). - name("count"). + name("dummyMetricCardinalityTest"). dataAttribute(MetricStorage.CARDINALITY_OVERFLOW). build(); @@ -152,4 +201,37 @@ public void testMetricCardinality() throws Exception { metricData = reader.collectAllMetrics(); Assert.assertTrue(metricData.stream().anyMatch(hasOverflowAttribute)); } + + @Test + public void testLongCounter() throws Exception { + LongCounter longCounter = meter.counterBuilder("dummyLongCounter").build(); + longCounter.add(1, Attributes.of(AttributeKey.stringKey("dummyAttr"), "dummyValue")); + longCounter.add(2, Attributes.of(AttributeKey.stringKey("dummyAttr"), "dummyValue")); + + Predicate predicate = MetricDataMatchPredicate.builder(). + name("dummyLongCounter"). + dataAttribute(Attributes.of(AttributeKey.stringKey("dummyAttr"), "dummyValue")). + type(MetricDataType.LONG_SUM). + longValue(3L). + build(); + + Collection metricData = reader.collectAllMetrics(); + Assert.assertTrue(metricData.stream().anyMatch(predicate)); + } + + @Test + public void testDoubleCounter() throws Exception { + DoubleCounter doubleCounter = meter.counterBuilder("dummyDoubleCounter").ofDoubles().build(); + doubleCounter.add(3.14, Attributes.of(AttributeKey.stringKey("dummyAttr"), "dummyValue")); + doubleCounter.add(2.71, Attributes.of(AttributeKey.stringKey("dummyAttr"), "dummyValue")); + + Predicate predicate = MetricDataMatchPredicate.builder(). + name("dummyDoubleCounter"). + dataAttribute(Attributes.of(AttributeKey.stringKey("dummyAttr"), "dummyValue")). + doubleValue(5.85). + build(); + + Collection metricData = reader.collectAllMetrics(); + Assert.assertTrue(metricData.stream().anyMatch(predicate)); + } } From cb6c131c8808d1e7390337abd2d72b118fc0475d Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 18 Jan 2024 11:35:07 -0800 Subject: [PATCH 019/133] Add VisibleForTesting annotation for extraReaders field --- .../org/apache/pulsar/common/stats/OpenTelemetryService.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index 9599259294137..1036a166cf91a 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.common.stats; +import com.google.common.annotations.VisibleForTesting; import io.opentelemetry.api.common.AttributeKey; import io.opentelemetry.api.metrics.Meter; import io.opentelemetry.sdk.OpenTelemetrySdk; @@ -50,7 +51,7 @@ public OpenTelemetryService( String clusterName, String serviceName, @Singular Map extraProperties, - @Singular List extraMetricReaders) { + @VisibleForTesting @Singular List extraMetricReaders) { Objects.requireNonNull(clusterName); AutoConfiguredOpenTelemetrySdkBuilder builder = AutoConfiguredOpenTelemetrySdk.builder(); builder.addPropertiesSupplier( From fd1adaa578c055a4e49ff0db00c3cfd07e3a8dfd Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 18 Jan 2024 11:35:32 -0800 Subject: [PATCH 020/133] Rename class MetricDataMatcher under tests --- .../common/stats/OpenTelemetryServiceTest.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java index a365d6fc2ab4e..49e3831b6c036 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java @@ -47,7 +47,7 @@ public class OpenTelemetryServiceTest { @Builder - public static final class MetricDataMatchPredicate implements Predicate { + public static final class MetricDataMatcher implements Predicate { final String name; final MetricDataType type; final InstrumentationScopeInfo instrumentationScopeInfo; @@ -138,7 +138,7 @@ public void testIsClusterNameSet() throws Exception { extraMetricReader(reader). build(); - Predicate predicate = MetricDataMatchPredicate.builder(). + Predicate predicate = MetricDataMatcher.builder(). resourceAttribute(Attributes.of(AttributeKey.stringKey("pulsar.cluster"), "testCluster")). build(); @@ -158,7 +158,7 @@ public void testIsServiceNameSet() throws Exception { extraMetricReader(reader). build(); - Predicate predicate = MetricDataMatchPredicate.builder(). + Predicate predicate = MetricDataMatcher.builder(). resourceAttribute(Attributes.of(AttributeKey.stringKey("service.name"), "testServiceName")). build(); @@ -170,7 +170,7 @@ public void testIsServiceNameSet() throws Exception { public void testIsInstrumentationNameSetOnMeter() throws Exception { Meter meter = openTelemetryService.getMeter("testInstrumentationScope"); meter.counterBuilder("dummyCounter").build().add(1); - MetricDataMatchPredicate predicate = MetricDataMatchPredicate.builder(). + MetricDataMatcher predicate = MetricDataMatcher.builder(). name("dummyCounter"). instrumentationScopeInfo(InstrumentationScopeInfo.create("testInstrumentationScope")). build(); @@ -186,7 +186,7 @@ public void testMetricCardinality() throws Exception { longCounter.add(1, Attributes.of(AttributeKey.stringKey("attribute"), "value" + i)); } - Predicate hasOverflowAttribute = MetricDataMatchPredicate.builder(). + Predicate hasOverflowAttribute = MetricDataMatcher.builder(). name("dummyMetricCardinalityTest"). dataAttribute(MetricStorage.CARDINALITY_OVERFLOW). build(); @@ -208,7 +208,7 @@ public void testLongCounter() throws Exception { longCounter.add(1, Attributes.of(AttributeKey.stringKey("dummyAttr"), "dummyValue")); longCounter.add(2, Attributes.of(AttributeKey.stringKey("dummyAttr"), "dummyValue")); - Predicate predicate = MetricDataMatchPredicate.builder(). + Predicate predicate = MetricDataMatcher.builder(). name("dummyLongCounter"). dataAttribute(Attributes.of(AttributeKey.stringKey("dummyAttr"), "dummyValue")). type(MetricDataType.LONG_SUM). @@ -225,7 +225,7 @@ public void testDoubleCounter() throws Exception { doubleCounter.add(3.14, Attributes.of(AttributeKey.stringKey("dummyAttr"), "dummyValue")); doubleCounter.add(2.71, Attributes.of(AttributeKey.stringKey("dummyAttr"), "dummyValue")); - Predicate predicate = MetricDataMatchPredicate.builder(). + Predicate predicate = MetricDataMatcher.builder(). name("dummyDoubleCounter"). dataAttribute(Attributes.of(AttributeKey.stringKey("dummyAttr"), "dummyValue")). doubleValue(5.85). From bc6e23d49c934c2c7682b52c7217d449a60ac431 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 18 Jan 2024 11:40:08 -0800 Subject: [PATCH 021/133] Move MetricDataMatcher to own class file --- .../common/stats/MetricDataMatcher.java | 93 +++++++++++++++++++ .../stats/OpenTelemetryServiceTest.java | 63 ------------- 2 files changed, 93 insertions(+), 63 deletions(-) create mode 100644 pulsar-common/src/test/java/org/apache/pulsar/common/stats/MetricDataMatcher.java diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/MetricDataMatcher.java b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/MetricDataMatcher.java new file mode 100644 index 0000000000000..e602b10783d18 --- /dev/null +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/MetricDataMatcher.java @@ -0,0 +1,93 @@ +/* + * 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.common.stats; + +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.sdk.common.InstrumentationScopeInfo; +import io.opentelemetry.sdk.metrics.data.DoublePointData; +import io.opentelemetry.sdk.metrics.data.LongPointData; +import io.opentelemetry.sdk.metrics.data.MetricData; +import io.opentelemetry.sdk.metrics.data.MetricDataType; +import io.opentelemetry.sdk.metrics.data.PointData; +import io.opentelemetry.sdk.resources.Resource; +import java.util.Collection; +import java.util.List; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import lombok.Builder; +import lombok.Singular; + +@Builder +public class MetricDataMatcher implements Predicate { + private final String name; + private final MetricDataType type; + private final InstrumentationScopeInfo instrumentationScopeInfo; + private final Resource resource; + @Singular + private final List resourceAttributes; + @Singular + private final List dataAttributes; + @Singular + private final List longValues; + @Singular + private final List doubleValues; + + @Override + public boolean test(MetricData md) { + return (name == null || name.equals(md.getName())) + && (type == null || type.equals(md.getType())) + && (instrumentationScopeInfo == null + || instrumentationScopeInfo.equals(md.getInstrumentationScopeInfo())) + && (resource == null || resource.equals(md.getResource())) + && matchesResourceAttributes(md) + && (dataAttributes == null || matchesDataAttributes(md)) + && matchesLongValues(md) + && matchesDoubleValues(md); + } + + private boolean matchesResourceAttributes(MetricData md) { + Attributes actual = md.getResource().getAttributes(); + return resourceAttributes.stream().allMatch(expected -> matchesAttributes(actual, expected)); + } + + private boolean matchesDataAttributes(MetricData md) { + Collection actuals = + md.getData().getPoints().stream().map(PointData::getAttributes).collect(Collectors.toSet()); + return dataAttributes.stream(). + allMatch(expected -> actuals.stream().anyMatch(actual -> matchesAttributes(actual, expected))); + } + + private boolean matchesAttributes(Attributes actual, Attributes expected) { + // Return true iff all attribute pairs in expected are a subset of those in actual. Allows tests to specify + // just the attributes they care about, insted of exhaustively having to list all of them. + return expected.asMap().entrySet().stream().allMatch(e -> e.getValue().equals(actual.get(e.getKey()))); + } + + private boolean matchesLongValues(MetricData md) { + Collection actualData = + md.getLongSumData().getPoints().stream().map(LongPointData::getValue).collect(Collectors.toSet()); + return actualData.containsAll(longValues); + } + + private boolean matchesDoubleValues(MetricData md) { + Collection actualData = + md.getDoubleSumData().getPoints().stream().map(DoublePointData::getValue).collect(Collectors.toSet()); + return actualData.containsAll(doubleValues); + } +} diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java index 49e3831b6c036..d201d7a64648b 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java @@ -24,21 +24,13 @@ import io.opentelemetry.api.metrics.LongCounter; import io.opentelemetry.api.metrics.Meter; import io.opentelemetry.sdk.common.InstrumentationScopeInfo; -import io.opentelemetry.sdk.metrics.data.DoublePointData; -import io.opentelemetry.sdk.metrics.data.LongPointData; import io.opentelemetry.sdk.metrics.data.MetricData; import io.opentelemetry.sdk.metrics.data.MetricDataType; -import io.opentelemetry.sdk.metrics.data.PointData; import io.opentelemetry.sdk.metrics.internal.state.MetricStorage; -import io.opentelemetry.sdk.resources.Resource; import io.opentelemetry.sdk.testing.exporter.InMemoryMetricReader; import java.util.Collection; -import java.util.List; import java.util.function.Predicate; -import java.util.stream.Collectors; -import lombok.Builder; import lombok.Cleanup; -import lombok.Singular; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -46,61 +38,6 @@ public class OpenTelemetryServiceTest { - @Builder - public static final class MetricDataMatcher implements Predicate { - final String name; - final MetricDataType type; - final InstrumentationScopeInfo instrumentationScopeInfo; - final Resource resource; - @Singular final List resourceAttributes; - @Singular final List dataAttributes; - @Singular final List longValues; - @Singular final List doubleValues; - - @Override - public boolean test(MetricData md) { - return (name == null || name.equals(md.getName())) - && (type == null || type.equals(md.getType())) - && (instrumentationScopeInfo == null - || instrumentationScopeInfo.equals(md.getInstrumentationScopeInfo())) - && (resource == null || resource.equals(md.getResource())) - && matchesResourceAttributes(md) - && (dataAttributes == null || matchesDataAttributes(md)) - && matchesLongValues(md) - && matchesDoubleValues(md); - } - - private boolean matchesResourceAttributes(MetricData md) { - Attributes actual = md.getResource().getAttributes(); - return resourceAttributes.stream().allMatch(expected -> matchesAttributes(actual, expected)); - } - - private boolean matchesDataAttributes(MetricData md) { - Collection actuals = - md.getData().getPoints().stream().map(PointData::getAttributes).collect(Collectors.toSet()); - return dataAttributes.stream(). - allMatch(expected -> actuals.stream().anyMatch(actual -> matchesAttributes(actual, expected))); - } - - private boolean matchesAttributes(Attributes actual, Attributes expected) { - // Return true iff all attribute pairs in expected are a subset of those in actual. Allows tests to specify - // just the attributes they care about, insted of exhaustively having to list all of them. - return expected.asMap().entrySet().stream().allMatch(e -> e.getValue().equals(actual.get(e.getKey()))); - } - - private boolean matchesLongValues(MetricData md) { - Collection actualData = - md.getLongSumData().getPoints().stream().map(LongPointData::getValue).collect(Collectors.toSet()); - return actualData.containsAll(longValues); - } - - private boolean matchesDoubleValues(MetricData md) { - Collection actualData = - md.getDoubleSumData().getPoints().stream().map(DoublePointData::getValue).collect(Collectors.toSet()); - return actualData.containsAll(doubleValues); - } - } - private OpenTelemetryService openTelemetryService; private InMemoryMetricReader reader; private Meter meter; From a14def03d773c4b23c9b72070410fdc9cef8f957 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 18 Jan 2024 13:42:50 -0800 Subject: [PATCH 022/133] Draft integration test containers --- .../OpenTelemetryCollectorContainer.java | 45 +++++++++++++++++++ .../containers/PrometheusContainer.java | 42 +++++++++++++++++ .../resources/containers/otel-collector.yaml | 31 +++++++++++++ .../test/resources/containers/prometheus.yaml | 6 +++ 4 files changed, 124 insertions(+) create mode 100644 tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/OpenTelemetryCollectorContainer.java create mode 100644 tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PrometheusContainer.java create mode 100644 tests/integration/src/test/resources/containers/otel-collector.yaml create mode 100644 tests/integration/src/test/resources/containers/prometheus.yaml diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/OpenTelemetryCollectorContainer.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/OpenTelemetryCollectorContainer.java new file mode 100644 index 0000000000000..e0c80dc1c68b8 --- /dev/null +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/OpenTelemetryCollectorContainer.java @@ -0,0 +1,45 @@ +/* + * 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.tests.integration.containers; + +import org.testcontainers.utility.MountableFile; + +public class OpenTelemetryCollectorContainer extends ChaosContainer { + + private static final String IMAGE_NAME = "otel/opentelemetry-collector-contrib:latest"; + + private static final int PROMETHEUS_COLLECTOR_PORT = 8888; + private static final int PROMETHEUS_EXPORTER_PORT = 8889; + private static final int OTLP_RECEIVER_PORT = 4317; + + public OpenTelemetryCollectorContainer(String clusterName) { + super(clusterName, IMAGE_NAME); + } + + @Override + protected void configure() { + super.configure(); + + this.withCopyFileToContainer( + MountableFile.forClasspathResource("containers/otel-collector-config.yaml", 0644), + "/etc/otel-collector-config.yaml") + .withCommand("--config=/etc/otel-collector-config.yaml") + .withExposedPorts(OTLP_RECEIVER_PORT, PROMETHEUS_COLLECTOR_PORT, PROMETHEUS_EXPORTER_PORT); + } +} diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PrometheusContainer.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PrometheusContainer.java new file mode 100644 index 0000000000000..13cdea934e196 --- /dev/null +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PrometheusContainer.java @@ -0,0 +1,42 @@ +/* + * 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.tests.integration.containers; + +import org.testcontainers.images.builder.Transferable; +import org.testcontainers.utility.MountableFile; + +public class PrometheusContainer extends ChaosContainer { + + private static final String IMAGE_NAME = "prom/prometheus:latest"; + + public PrometheusContainer(String clusterName) { + super(clusterName, IMAGE_NAME); + + this.withCopyToContainer( + MountableFile.forClasspathResource("prometheus.yml"), + "/etc/prometheus/prometheus.yml"); + + Transferable.of("yolo"); + + this.withCopyFileToContainer( + MountableFile.forClasspathResource("prometheus.yml"), + "/etc/prometheus/prometheus.yml"); + this.withExposedPorts(9090); + } +} diff --git a/tests/integration/src/test/resources/containers/otel-collector.yaml b/tests/integration/src/test/resources/containers/otel-collector.yaml new file mode 100644 index 0000000000000..4511d64def4fc --- /dev/null +++ b/tests/integration/src/test/resources/containers/otel-collector.yaml @@ -0,0 +1,31 @@ +receivers: + otlp: + protocols: + grpc: + +exporters: + prometheus: + endpoint: "0.0.0.0:8889" + namespace: promexample + const_labels: + label1: value1 + logging: + loglevel: debug + +processors: + batch: + +extensions: + health_check: + pprof: + endpoint: :1888 + zpages: + endpoint: :55679 + +service: + extensions: [pprof, zpages, health_check] + pipelines: + metrics: + receivers: [otlp] + processors: [batch] + exporters: [logging, prometheus] \ No newline at end of file diff --git a/tests/integration/src/test/resources/containers/prometheus.yaml b/tests/integration/src/test/resources/containers/prometheus.yaml new file mode 100644 index 0000000000000..4791a62ec8dca --- /dev/null +++ b/tests/integration/src/test/resources/containers/prometheus.yaml @@ -0,0 +1,6 @@ +scrape_configs: + - job_name: 'otel-collector' + scrape_interval: 2s + static_configs: + - targets: ['otel-collector:8889'] + - targets: ['otel-collector:8888'] \ No newline at end of file From 4d32e7f7de65e28ff0884a2b4a836b87c2992a0a Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 18 Jan 2024 17:02:57 -0800 Subject: [PATCH 023/133] Update container definitions --- .../OpenTelemetryCollectorContainer.java | 4 +++- .../containers/PrometheusContainer.java | 21 ++++++++++--------- 2 files changed, 14 insertions(+), 11 deletions(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/OpenTelemetryCollectorContainer.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/OpenTelemetryCollectorContainer.java index e0c80dc1c68b8..af744029f1893 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/OpenTelemetryCollectorContainer.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/OpenTelemetryCollectorContainer.java @@ -23,6 +23,7 @@ public class OpenTelemetryCollectorContainer extends ChaosContainer { private static final String IMAGE_NAME = "otel/opentelemetry-collector-contrib:latest"; + private static final String NAME = "otel-collector"; private static final int PROMETHEUS_COLLECTOR_PORT = 8888; private static final int PROMETHEUS_EXPORTER_PORT = 8889; @@ -36,7 +37,8 @@ public OpenTelemetryCollectorContainer(String clusterName) { protected void configure() { super.configure(); - this.withCopyFileToContainer( + this.withNetworkAliases(NAME) + .withCopyFileToContainer( MountableFile.forClasspathResource("containers/otel-collector-config.yaml", 0644), "/etc/otel-collector-config.yaml") .withCommand("--config=/etc/otel-collector-config.yaml") diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PrometheusContainer.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PrometheusContainer.java index 13cdea934e196..5ba9579a73f77 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PrometheusContainer.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PrometheusContainer.java @@ -18,25 +18,26 @@ */ package org.apache.pulsar.tests.integration.containers; -import org.testcontainers.images.builder.Transferable; import org.testcontainers.utility.MountableFile; public class PrometheusContainer extends ChaosContainer { private static final String IMAGE_NAME = "prom/prometheus:latest"; + private static final String NAME = "prometheus"; + private static final int PROMETHEUS_PORT = 9090; public PrometheusContainer(String clusterName) { super(clusterName, IMAGE_NAME); + } - this.withCopyToContainer( - MountableFile.forClasspathResource("prometheus.yml"), - "/etc/prometheus/prometheus.yml"); - - Transferable.of("yolo"); + @Override + protected void configure() { + super.configure(); - this.withCopyFileToContainer( - MountableFile.forClasspathResource("prometheus.yml"), - "/etc/prometheus/prometheus.yml"); - this.withExposedPorts(9090); + this.withNetworkAliases(NAME) + .withCopyToContainer( + MountableFile.forClasspathResource("containers/prometheus.yml"), + "/etc/prometheus/prometheus.yml") + .withExposedPorts(PROMETHEUS_PORT); } } From a93d9dc3dd861e20947d67a64d18db857e40a261 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 18 Jan 2024 17:13:23 -0800 Subject: [PATCH 024/133] Fix config file name typo for Prometheus --- .../tests/integration/containers/PrometheusContainer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PrometheusContainer.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PrometheusContainer.java index 5ba9579a73f77..d98d50f249ebf 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PrometheusContainer.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PrometheusContainer.java @@ -36,7 +36,7 @@ protected void configure() { this.withNetworkAliases(NAME) .withCopyToContainer( - MountableFile.forClasspathResource("containers/prometheus.yml"), + MountableFile.forClasspathResource("containers/prometheus.yaml"), "/etc/prometheus/prometheus.yml") .withExposedPorts(PROMETHEUS_PORT); } From 0148c59a42dbd79dd593a706914dc8f1c457fb1a Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 18 Jan 2024 17:49:41 -0800 Subject: [PATCH 025/133] Add MetricsTest draft --- .../apache/pulsar/tests/integration/metrics/MetricsTest.java | 2 ++ 1 file changed, 2 insertions(+) create mode 100644 tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java new file mode 100644 index 0000000000000..c9b65547f6cd0 --- /dev/null +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java @@ -0,0 +1,2 @@ +package org.apache.pulsar.tests.integration.metrics;public class MetricsTest { +} From 504181e29d72f8ead0133c15f333f03615a4112a Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 18 Jan 2024 17:52:28 -0800 Subject: [PATCH 026/133] Add missing licenses --- .../integration/metrics/MetricsTest.java | 57 ++++++++++++++++++- .../resources/containers/otel-collector.yaml | 19 +++++++ .../test/resources/containers/prometheus.yaml | 19 +++++++ 3 files changed, 94 insertions(+), 1 deletion(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java index c9b65547f6cd0..9fa6417a2a7b1 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java @@ -1,2 +1,57 @@ -package org.apache.pulsar.tests.integration.metrics;public class MetricsTest { +/* + * 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.tests.integration.metrics; + +import java.time.Duration; +import java.time.LocalDateTime; +import java.util.UUID; +import lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.tests.integration.containers.OpenTelemetryCollectorContainer; +import org.apache.pulsar.tests.integration.containers.PrometheusContainer; +import org.apache.pulsar.tests.integration.topologies.PulsarCluster; +import org.apache.pulsar.tests.integration.topologies.PulsarClusterSpec; +import org.awaitility.Awaitility; +import org.testng.annotations.Test; + +@Slf4j +public class MetricsTest { + + @Test + public void testBrokerMetrics() throws Exception { + var clusterName = MetricsTest.class.getSimpleName() + UUID.randomUUID(); + var spec = PulsarClusterSpec.builder() + .numBookies(1) + .numBrokers(1) + .numProxies(1) + .externalService("otel-collector", new OpenTelemetryCollectorContainer(clusterName)) + .externalService("prometheus", new PrometheusContainer(clusterName)) + .build(); + @Cleanup("stop") + var pulsarCluster = PulsarCluster.forSpec(spec); + pulsarCluster.start(); + + var start = LocalDateTime.now(); + Awaitility.waitAtMost(Duration.ofMinutes(30)).pollDelay(Duration.ofSeconds(30)).until(() -> { + var duration = Duration.between(LocalDateTime.now(), start); + log.info("Time since start: {}", duration); + return false; + }); + } } diff --git a/tests/integration/src/test/resources/containers/otel-collector.yaml b/tests/integration/src/test/resources/containers/otel-collector.yaml index 4511d64def4fc..5425105db4cbf 100644 --- a/tests/integration/src/test/resources/containers/otel-collector.yaml +++ b/tests/integration/src/test/resources/containers/otel-collector.yaml @@ -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. +# + receivers: otlp: protocols: diff --git a/tests/integration/src/test/resources/containers/prometheus.yaml b/tests/integration/src/test/resources/containers/prometheus.yaml index 4791a62ec8dca..6407df7915c78 100644 --- a/tests/integration/src/test/resources/containers/prometheus.yaml +++ b/tests/integration/src/test/resources/containers/prometheus.yaml @@ -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. +# + scrape_configs: - job_name: 'otel-collector' scrape_interval: 2s From 903677ad6f49026d655f30702e2f49e5fda5289f Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 18 Jan 2024 18:07:14 -0800 Subject: [PATCH 027/133] Rename otel-collector-config yaml file --- .../{otel-collector.yaml => otel-collector-config.yaml} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename tests/integration/src/test/resources/containers/{otel-collector.yaml => otel-collector-config.yaml} (100%) diff --git a/tests/integration/src/test/resources/containers/otel-collector.yaml b/tests/integration/src/test/resources/containers/otel-collector-config.yaml similarity index 100% rename from tests/integration/src/test/resources/containers/otel-collector.yaml rename to tests/integration/src/test/resources/containers/otel-collector-config.yaml From d84e2d0ea2d9d2bbcf5619f4645ce421f24c82c1 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 18 Jan 2024 18:07:23 -0800 Subject: [PATCH 028/133] Configure cluster name for MetricsTest --- .../org/apache/pulsar/tests/integration/metrics/MetricsTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java index 9fa6417a2a7b1..fedec798c18a2 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java @@ -37,6 +37,7 @@ public class MetricsTest { public void testBrokerMetrics() throws Exception { var clusterName = MetricsTest.class.getSimpleName() + UUID.randomUUID(); var spec = PulsarClusterSpec.builder() + .clusterName(clusterName) .numBookies(1) .numBrokers(1) .numProxies(1) From c1ed51df6c6b2be683336992adb37aa4bfd0c415 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 18 Jan 2024 18:36:15 -0800 Subject: [PATCH 029/133] Fix container names and add wait strategies --- .../OpenTelemetryCollectorContainer.java | 20 ++++++++++++++++++- .../containers/PrometheusContainer.java | 13 +++++++++++- .../integration/metrics/MetricsTest.java | 6 +++--- 3 files changed, 34 insertions(+), 5 deletions(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/OpenTelemetryCollectorContainer.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/OpenTelemetryCollectorContainer.java index af744029f1893..c6a8e2d9d9790 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/OpenTelemetryCollectorContainer.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/OpenTelemetryCollectorContainer.java @@ -18,6 +18,9 @@ */ package org.apache.pulsar.tests.integration.containers; +import java.time.Duration; +import org.apache.http.HttpStatus; +import org.testcontainers.containers.wait.strategy.HttpWaitStrategy; import org.testcontainers.utility.MountableFile; public class OpenTelemetryCollectorContainer extends ChaosContainer { @@ -28,6 +31,7 @@ public class OpenTelemetryCollectorContainer extends ChaosContainer { + createContainerCmd.withHostName(NAME); + createContainerCmd.withName(getContainerName()); + }) + .waitingFor(new HttpWaitStrategy() + .forPath("/debug/servicez") + .forPort(ZPAGES_PORT) + .forStatusCode(HttpStatus.SC_OK) + .withStartupTimeout(Duration.ofSeconds(300))); + } + + @Override + public String getContainerName() { + return clusterName + "-" + NAME; } } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PrometheusContainer.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PrometheusContainer.java index d98d50f249ebf..0adb6378ca703 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PrometheusContainer.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PrometheusContainer.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.tests.integration.containers; +import org.testcontainers.containers.wait.strategy.HostPortWaitStrategy; import org.testcontainers.utility.MountableFile; public class PrometheusContainer extends ChaosContainer { @@ -38,6 +39,16 @@ protected void configure() { .withCopyToContainer( MountableFile.forClasspathResource("containers/prometheus.yaml"), "/etc/prometheus/prometheus.yml") - .withExposedPorts(PROMETHEUS_PORT); + .withExposedPorts(PROMETHEUS_PORT) + .withCreateContainerCmdModifier(createContainerCmd -> { + createContainerCmd.withHostName(NAME); + createContainerCmd.withName(getContainerName()); + }) + .waitingFor(new HostPortWaitStrategy()); + } + + @Override + public String getContainerName() { + return clusterName + "-" + NAME; } } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java index fedec798c18a2..13a8d03f46b80 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java @@ -35,7 +35,7 @@ public class MetricsTest { @Test public void testBrokerMetrics() throws Exception { - var clusterName = MetricsTest.class.getSimpleName() + UUID.randomUUID(); + var clusterName = MetricsTest.class.getSimpleName() + "-" + UUID.randomUUID(); var spec = PulsarClusterSpec.builder() .clusterName(clusterName) .numBookies(1) @@ -49,8 +49,8 @@ public void testBrokerMetrics() throws Exception { pulsarCluster.start(); var start = LocalDateTime.now(); - Awaitility.waitAtMost(Duration.ofMinutes(30)).pollDelay(Duration.ofSeconds(30)).until(() -> { - var duration = Duration.between(LocalDateTime.now(), start); + Awaitility.waitAtMost(Duration.ofMinutes(30)).pollInterval(Duration.ofSeconds(30)).until(() -> { + var duration = Duration.between(start, LocalDateTime.now()); log.info("Time since start: {}", duration); return false; }); From 50e33a9127d7f37d25d4e546ea709f4a961c14c0 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 18 Jan 2024 19:58:55 -0800 Subject: [PATCH 030/133] Add opentelemetry java agent pom --- pom.xml | 1 + pulsar-common/pom.xml | 8 ++++++++ 2 files changed, 9 insertions(+) diff --git a/pom.xml b/pom.xml index 08b74d38166e6..7e65b7b8e4f72 100644 --- a/pom.xml +++ b/pom.xml @@ -249,6 +249,7 @@ flexible messaging model and an intuitive client API. 1.5.2-3 2.0.6 1.34.0 + 1.32.0 1.18.3 diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index 3265ba2aa24b0..0766af0a9978f 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -219,6 +219,7 @@ ${opentelemetry.version}-alpha + io.opentelemetry opentelemetry-exporter-logging @@ -226,6 +227,13 @@ test + + io.opentelemetry.javaagent + opentelemetry-javaagent + ${opentelemetry.javaagent.version} + runtime + + io.opentelemetry opentelemetry-sdk-extension-autoconfigure From 6fcbf6ad8c5452c93dd66ef966c7500c6c7d11e0 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 18 Jan 2024 20:27:56 -0800 Subject: [PATCH 031/133] Move javaagent out of runtime scope --- pulsar-common/pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index 0766af0a9978f..e95206f4b37e8 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -231,7 +231,6 @@ io.opentelemetry.javaagent opentelemetry-javaagent ${opentelemetry.javaagent.version} - runtime From 16c7e1b294a57d28592f70bf66b3cd617f0f65a1 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Fri, 19 Jan 2024 09:24:24 -0800 Subject: [PATCH 032/133] Fix missing dependencies issue --- distribution/server/src/assemble/bin.xml | 2 +- pom.xml | 11 +---- pulsar-common/pom.xml | 61 +++++++++++++++++------- 3 files changed, 47 insertions(+), 27 deletions(-) diff --git a/distribution/server/src/assemble/bin.xml b/distribution/server/src/assemble/bin.xml index 949c265706929..23c2a33f8c306 100644 --- a/distribution/server/src/assemble/bin.xml +++ b/distribution/server/src/assemble/bin.xml @@ -110,7 +110,7 @@ lib false - compile + runtime false diff --git a/pom.xml b/pom.xml index 7e65b7b8e4f72..42efa175094bb 100644 --- a/pom.xml +++ b/pom.xml @@ -248,7 +248,8 @@ flexible messaging model and an intuitive client API. 3.4.3 1.5.2-3 2.0.6 - 1.34.0 + 1.34.1 + 1.23.1-alpha 1.32.0 @@ -1435,14 +1436,6 @@ flexible messaging model and an intuitive client API. ${restassured.version} test - - - io.opentelemetry - opentelemetry-bom - ${opentelemetry.version} - pom - import - diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index e95206f4b37e8..abddfacca30ce 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -34,6 +34,26 @@ Pulsar Common Common libraries needed by client/broker/tools + + + + io.opentelemetry + opentelemetry-bom + ${opentelemetry.version} + pom + import + + + + io.opentelemetry + opentelemetry-bom-alpha + ${opentelemetry.version}-alpha + pom + import + + + + ${project.groupId} @@ -201,42 +221,49 @@ protobuf-java + io.opentelemetry opentelemetry-api - ${opentelemetry.version} - io.opentelemetry - opentelemetry-exporter-otlp - ${opentelemetry.version} + opentelemetry-api-events - io.opentelemetry - opentelemetry-exporter-prometheus - ${opentelemetry.version}-alpha + opentelemetry-sdk + + + io.opentelemetry + opentelemetry-sdk-metrics - - io.opentelemetry opentelemetry-exporter-logging - ${opentelemetry.version} - test - - io.opentelemetry.javaagent - opentelemetry-javaagent - ${opentelemetry.javaagent.version} + io.opentelemetry + opentelemetry-exporter-otlp - io.opentelemetry opentelemetry-sdk-extension-autoconfigure - ${opentelemetry.version} + + + io.opentelemetry + opentelemetry-sdk-extension-autoconfigure-spi + + + + io.opentelemetry.semconv + opentelemetry-semconv + ${opentelemetry.semconv.version} + + + io.opentelemetry.javaagent + opentelemetry-javaagent + ${opentelemetry.javaagent.version} From d567fdffdad5741c76287616ffab1769546a18a6 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Fri, 19 Jan 2024 14:21:37 -0800 Subject: [PATCH 033/133] Add pulsar-broker test-jar to integration tests --- tests/integration/pom.xml | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/tests/integration/pom.xml b/tests/integration/pom.xml index df36c35a19113..cf163acdfe28e 100644 --- a/tests/integration/pom.xml +++ b/tests/integration/pom.xml @@ -55,6 +55,13 @@ ${project.version} test + + org.apache.pulsar + pulsar-broker + ${project.version} + test-jar + test + org.apache.pulsar pulsar-common @@ -189,6 +196,12 @@ test + + io.rest-assured + rest-assured + test + + org.testcontainers From 70026c8dc3fb54c6159a5b0df199ae794ca44cfc Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Fri, 19 Jan 2024 14:22:10 -0800 Subject: [PATCH 034/133] Cleanup otel collector config --- .../resources/containers/otel-collector-config.yaml | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/tests/integration/src/test/resources/containers/otel-collector-config.yaml b/tests/integration/src/test/resources/containers/otel-collector-config.yaml index 5425105db4cbf..bd332f0428307 100644 --- a/tests/integration/src/test/resources/containers/otel-collector-config.yaml +++ b/tests/integration/src/test/resources/containers/otel-collector-config.yaml @@ -25,26 +25,19 @@ receivers: exporters: prometheus: endpoint: "0.0.0.0:8889" - namespace: promexample - const_labels: - label1: value1 - logging: - loglevel: debug processors: batch: extensions: health_check: - pprof: - endpoint: :1888 zpages: endpoint: :55679 service: - extensions: [pprof, zpages, health_check] + extensions: [zpages, health_check] pipelines: metrics: receivers: [otlp] processors: [batch] - exporters: [logging, prometheus] \ No newline at end of file + exporters: [prometheus] \ No newline at end of file From 6d91cedbf9cd61a9c3907cc30046e7ff46194881 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Fri, 19 Jan 2024 14:22:28 -0800 Subject: [PATCH 035/133] Refactor tests --- .../OpenTelemetryCollectorContainer.java | 15 ++++-- .../containers/PrometheusContainer.java | 1 + .../integration/metrics/MetricsTest.java | 47 +++++++++++++++---- 3 files changed, 49 insertions(+), 14 deletions(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/OpenTelemetryCollectorContainer.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/OpenTelemetryCollectorContainer.java index c6a8e2d9d9790..44f9f5d17299e 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/OpenTelemetryCollectorContainer.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/OpenTelemetryCollectorContainer.java @@ -20,6 +20,7 @@ import java.time.Duration; import org.apache.http.HttpStatus; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.testcontainers.containers.wait.strategy.HttpWaitStrategy; import org.testcontainers.utility.MountableFile; @@ -28,7 +29,6 @@ public class OpenTelemetryCollectorContainer extends ChaosContainer { createContainerCmd.withHostName(NAME); createContainerCmd.withName(getContainerName()); @@ -62,4 +61,12 @@ protected void configure() { public String getContainerName() { return clusterName + "-" + NAME; } + + public PrometheusMetricsClient getMetricsClient() { + return new PrometheusMetricsClient(getHost(), getMappedPort(PROMETHEUS_EXPORTER_PORT)); + } + + public String getOtlpEndpoint() { + return String.format("http://%s:%d", NAME, OTLP_RECEIVER_PORT); + } } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PrometheusContainer.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PrometheusContainer.java index 0adb6378ca703..e96378f171e3d 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PrometheusContainer.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PrometheusContainer.java @@ -45,6 +45,7 @@ protected void configure() { createContainerCmd.withName(getContainerName()); }) .waitingFor(new HostPortWaitStrategy()); + tailContainerLog(); } @Override diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java index 13a8d03f46b80..6a132dde3d2bd 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java @@ -18,13 +18,14 @@ */ package org.apache.pulsar.tests.integration.metrics; +import static org.testng.Assert.assertEquals; import java.time.Duration; -import java.time.LocalDateTime; +import java.util.Map; import java.util.UUID; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.tests.integration.containers.OpenTelemetryCollectorContainer; -import org.apache.pulsar.tests.integration.containers.PrometheusContainer; import org.apache.pulsar.tests.integration.topologies.PulsarCluster; import org.apache.pulsar.tests.integration.topologies.PulsarClusterSpec; import org.awaitility.Awaitility; @@ -34,25 +35,51 @@ public class MetricsTest { @Test - public void testBrokerMetrics() throws Exception { + public void testOpenTelemetryMetrics() throws Exception { var clusterName = MetricsTest.class.getSimpleName() + "-" + UUID.randomUUID(); + var openTelemetryCollectorContainer = new OpenTelemetryCollectorContainer(clusterName); + + assertEquals(openTelemetryCollectorContainer.getOtlpEndpoint(), "http://otel-collector:4317"); + + var brokerOtelServiceName = clusterName + "-broker"; + var localCollectorProps = Map.of( + "OTEL_SDK_DISABLED", "false", + "OTEL_METRICS_EXPORTER", "otlp", + "OTEL_METRIC_EXPORT_INTERVAL", "1000", + "OTEL_EXPORTER_OTLP_ENDPOINT", openTelemetryCollectorContainer.getOtlpEndpoint(), + "OTEL_SERVICE_NAME", brokerOtelServiceName + ); + + var proxyOtelServiceName = clusterName + "-proxy"; + var proxyCollectorProps = Map.of( + "OTEL_SDK_DISABLED", "false", + "OTEL_METRICS_EXPORTER", "otlp", + "OTEL_METRIC_EXPORT_INTERVAL", "1000", + "OTEL_EXPORTER_OTLP_ENDPOINT", openTelemetryCollectorContainer.getOtlpEndpoint(), + "OTEL_SERVICE_NAME", proxyOtelServiceName + ); + var spec = PulsarClusterSpec.builder() .clusterName(clusterName) .numBookies(1) .numBrokers(1) + .brokerEnvs(localCollectorProps) .numProxies(1) - .externalService("otel-collector", new OpenTelemetryCollectorContainer(clusterName)) - .externalService("prometheus", new PrometheusContainer(clusterName)) + .proxyEnvs(proxyCollectorProps) + .externalService("otel-collector", openTelemetryCollectorContainer) .build(); @Cleanup("stop") var pulsarCluster = PulsarCluster.forSpec(spec); pulsarCluster.start(); - var start = LocalDateTime.now(); - Awaitility.waitAtMost(Duration.ofMinutes(30)).pollInterval(Duration.ofSeconds(30)).until(() -> { - var duration = Duration.between(start, LocalDateTime.now()); - log.info("Time since start: {}", duration); - return false; + var prometheusClient = openTelemetryCollectorContainer.getMetricsClient(); + + Awaitility.waitAtMost(Duration.ofMinutes(5)).pollInterval(Duration.ofSeconds(1)).until(() -> { + var metricName = "queueSize_ratio"; // Sent automatically by the OpenTelemetry SDK. + var metrics = prometheusClient.getMetrics(); + var brokerMetrics = metrics.findByNameAndLabels(metricName, Pair.of("job", brokerOtelServiceName)); + var proxyMetrics = metrics.findByNameAndLabels(metricName, Pair.of("job", proxyOtelServiceName)); + return !brokerMetrics.isEmpty() && !proxyMetrics.isEmpty(); }); } } From 6ec7869512576d8bd69362b3a81e620fa5622bf5 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Fri, 19 Jan 2024 14:29:45 -0800 Subject: [PATCH 036/133] Cosmetic fix --- .../apache/pulsar/tests/integration/metrics/MetricsTest.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java index 6a132dde3d2bd..3ede77345680e 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java @@ -39,8 +39,6 @@ public void testOpenTelemetryMetrics() throws Exception { var clusterName = MetricsTest.class.getSimpleName() + "-" + UUID.randomUUID(); var openTelemetryCollectorContainer = new OpenTelemetryCollectorContainer(clusterName); - assertEquals(openTelemetryCollectorContainer.getOtlpEndpoint(), "http://otel-collector:4317"); - var brokerOtelServiceName = clusterName + "-broker"; var localCollectorProps = Map.of( "OTEL_SDK_DISABLED", "false", From 661a384e515bcd3136b20275e982d6da4762c310 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Fri, 19 Jan 2024 14:33:28 -0800 Subject: [PATCH 037/133] Cosmetic fix --- .../pulsar/tests/integration/metrics/MetricsTest.java | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java index 3ede77345680e..20ba048273fe1 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java @@ -18,10 +18,9 @@ */ package org.apache.pulsar.tests.integration.metrics; -import static org.testng.Assert.assertEquals; -import java.time.Duration; import java.util.Map; import java.util.UUID; +import java.util.concurrent.TimeUnit; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.tuple.Pair; @@ -70,11 +69,9 @@ public void testOpenTelemetryMetrics() throws Exception { var pulsarCluster = PulsarCluster.forSpec(spec); pulsarCluster.start(); - var prometheusClient = openTelemetryCollectorContainer.getMetricsClient(); - - Awaitility.waitAtMost(Duration.ofMinutes(5)).pollInterval(Duration.ofSeconds(1)).until(() -> { + Awaitility.waitAtMost(180, TimeUnit.SECONDS).pollInterval(1, TimeUnit.SECONDS).until(() -> { var metricName = "queueSize_ratio"; // Sent automatically by the OpenTelemetry SDK. - var metrics = prometheusClient.getMetrics(); + var metrics = openTelemetryCollectorContainer.getMetricsClient().getMetrics(); var brokerMetrics = metrics.findByNameAndLabels(metricName, Pair.of("job", brokerOtelServiceName)); var proxyMetrics = metrics.findByNameAndLabels(metricName, Pair.of("job", proxyOtelServiceName)); return !brokerMetrics.isEmpty() && !proxyMetrics.isEmpty(); From a7d86be825c3c1760589302189bc7e151f130d1f Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Fri, 19 Jan 2024 14:49:40 -0800 Subject: [PATCH 038/133] Allow specifying of function worker env for test PulsarCluster --- .../pulsar/tests/integration/topologies/PulsarCluster.java | 4 ++++ .../tests/integration/topologies/PulsarClusterSpec.java | 6 ++++++ 2 files changed, 10 insertions(+) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarCluster.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarCluster.java index 88ff778732ed3..a6603838cca56 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarCluster.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarCluster.java @@ -94,6 +94,7 @@ public static PulsarCluster forSpec(PulsarClusterSpec spec, CSContainer csContai private final ProxyContainer proxyContainer; private Map> externalServices = Collections.emptyMap(); private Map> externalServiceEnvs; + private Map> functionWorkerEnvs; private PulsarCluster(PulsarClusterSpec spec, CSContainer csContainer, boolean sharedCsContainer) { @@ -326,6 +327,8 @@ public void start() throws Exception { log.info("Successfully started external service {}.", service.getKey()); }); } + + this.functionWorkerEnvs = spec.functionWorkerEnvs; } public void startService(String networkAlias, @@ -435,6 +438,7 @@ private void startFunctionWorkersWithProcessContainerFactory(String suffix, int .withEnv("zookeeperServers", ZKContainer.NAME) // bookkeeper tools .withEnv("zkServers", ZKContainer.NAME) + .withEnv(functionWorkerEnvs.getOrDefault(suffix, Collections.emptyMap())) )); this.startWorkers(); } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java index 81e7ae70efffa..d0e9d4272e7a4 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java @@ -81,6 +81,12 @@ public class PulsarClusterSpec { @Default int numFunctionWorkers = 0; + /** + * Specify envs for function workers. + */ + @Singular + Map functionWorkerEnvs; + /** * Allow to query the last message */ From 4606bc12a9fe940f006b8f8b09f12c4fadcdb67f Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Fri, 19 Jan 2024 15:40:40 -0800 Subject: [PATCH 039/133] Test fix --- .../integration/metrics/MetricsTest.java | 21 ++++++++++++++++--- .../topologies/PulsarClusterSpec.java | 2 +- 2 files changed, 19 insertions(+), 4 deletions(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java index 20ba048273fe1..4e7609ed9dbe1 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java @@ -25,8 +25,10 @@ import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.tests.integration.containers.OpenTelemetryCollectorContainer; +import org.apache.pulsar.tests.integration.topologies.FunctionRuntimeType; import org.apache.pulsar.tests.integration.topologies.PulsarCluster; import org.apache.pulsar.tests.integration.topologies.PulsarClusterSpec; +import org.apache.pulsar.tests.integration.topologies.PulsarTestBase; import org.awaitility.Awaitility; import org.testng.annotations.Test; @@ -39,7 +41,7 @@ public void testOpenTelemetryMetrics() throws Exception { var openTelemetryCollectorContainer = new OpenTelemetryCollectorContainer(clusterName); var brokerOtelServiceName = clusterName + "-broker"; - var localCollectorProps = Map.of( + var brokerCollectorProps = Map.of( "OTEL_SDK_DISABLED", "false", "OTEL_METRICS_EXPORTER", "otlp", "OTEL_METRIC_EXPORT_INTERVAL", "1000", @@ -56,25 +58,38 @@ public void testOpenTelemetryMetrics() throws Exception { "OTEL_SERVICE_NAME", proxyOtelServiceName ); + var functionWorkerServiceNameSuffix = PulsarTestBase.randomName(); + var functionWorkerCollectorProps = Map.of( + "OTEL_SDK_DISABLED", "false", + "OTEL_METRICS_EXPORTER", "otlp", + "OTEL_METRIC_EXPORT_INTERVAL", "1000", + "OTEL_EXPORTER_OTLP_ENDPOINT", openTelemetryCollectorContainer.getOtlpEndpoint(), + "OTEL_SERVICE_NAME", functionWorkerServiceNameSuffix + ); + var spec = PulsarClusterSpec.builder() .clusterName(clusterName) .numBookies(1) .numBrokers(1) - .brokerEnvs(localCollectorProps) + .brokerEnvs(brokerCollectorProps) .numProxies(1) .proxyEnvs(proxyCollectorProps) .externalService("otel-collector", openTelemetryCollectorContainer) + .functionWorkerEnv(functionWorkerServiceNameSuffix, functionWorkerCollectorProps) .build(); @Cleanup("stop") var pulsarCluster = PulsarCluster.forSpec(spec); pulsarCluster.start(); + pulsarCluster.setupFunctionWorkers(functionWorkerServiceNameSuffix, FunctionRuntimeType.PROCESS, 1); Awaitility.waitAtMost(180, TimeUnit.SECONDS).pollInterval(1, TimeUnit.SECONDS).until(() -> { var metricName = "queueSize_ratio"; // Sent automatically by the OpenTelemetry SDK. var metrics = openTelemetryCollectorContainer.getMetricsClient().getMetrics(); var brokerMetrics = metrics.findByNameAndLabels(metricName, Pair.of("job", brokerOtelServiceName)); var proxyMetrics = metrics.findByNameAndLabels(metricName, Pair.of("job", proxyOtelServiceName)); - return !brokerMetrics.isEmpty() && !proxyMetrics.isEmpty(); + var functionWorkerMetrics = + metrics.findByNameAndLabels(metricName, Pair.of("job", functionWorkerServiceNameSuffix)); + return !brokerMetrics.isEmpty() && !proxyMetrics.isEmpty() && !functionWorkerMetrics.isEmpty(); }); } } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java index d0e9d4272e7a4..b0b56c8c6e3d8 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java @@ -85,7 +85,7 @@ public class PulsarClusterSpec { * Specify envs for function workers. */ @Singular - Map functionWorkerEnvs; + Map> functionWorkerEnvs; /** * Allow to query the last message From 00ad59bcbcce969e79d12b71c95d711f63374e86 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Fri, 19 Jan 2024 17:10:27 -0800 Subject: [PATCH 040/133] Disable function worker test --- .../pulsar/tests/integration/metrics/MetricsTest.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java index 4e7609ed9dbe1..bce80131f5cda 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java @@ -25,7 +25,6 @@ import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.tests.integration.containers.OpenTelemetryCollectorContainer; -import org.apache.pulsar.tests.integration.topologies.FunctionRuntimeType; import org.apache.pulsar.tests.integration.topologies.PulsarCluster; import org.apache.pulsar.tests.integration.topologies.PulsarClusterSpec; import org.apache.pulsar.tests.integration.topologies.PulsarTestBase; @@ -37,7 +36,7 @@ public class MetricsTest { @Test public void testOpenTelemetryMetrics() throws Exception { - var clusterName = MetricsTest.class.getSimpleName() + "-" + UUID.randomUUID(); + var clusterName = "testOpenTelemetryMetrics-" + UUID.randomUUID(); var openTelemetryCollectorContainer = new OpenTelemetryCollectorContainer(clusterName); var brokerOtelServiceName = clusterName + "-broker"; @@ -80,7 +79,7 @@ public void testOpenTelemetryMetrics() throws Exception { @Cleanup("stop") var pulsarCluster = PulsarCluster.forSpec(spec); pulsarCluster.start(); - pulsarCluster.setupFunctionWorkers(functionWorkerServiceNameSuffix, FunctionRuntimeType.PROCESS, 1); + // pulsarCluster.setupFunctionWorkers(functionWorkerServiceNameSuffix, FunctionRuntimeType.PROCESS, 1); Awaitility.waitAtMost(180, TimeUnit.SECONDS).pollInterval(1, TimeUnit.SECONDS).until(() -> { var metricName = "queueSize_ratio"; // Sent automatically by the OpenTelemetry SDK. @@ -89,7 +88,7 @@ public void testOpenTelemetryMetrics() throws Exception { var proxyMetrics = metrics.findByNameAndLabels(metricName, Pair.of("job", proxyOtelServiceName)); var functionWorkerMetrics = metrics.findByNameAndLabels(metricName, Pair.of("job", functionWorkerServiceNameSuffix)); - return !brokerMetrics.isEmpty() && !proxyMetrics.isEmpty() && !functionWorkerMetrics.isEmpty(); + return !brokerMetrics.isEmpty() && !proxyMetrics.isEmpty(); }); } } From 6e072e6dabffc2aa77e035ca40fad3a7c66833ba Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 22 Jan 2024 16:04:25 -0800 Subject: [PATCH 041/133] Disable OTel by default --- .../common/stats/OpenTelemetryService.java | 22 ++++++++++++------- .../stats/OpenTelemetryServiceTest.java | 5 ++++- 2 files changed, 18 insertions(+), 9 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index 1036a166cf91a..45fb8e3ed3a68 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.common.stats; +import static com.google.common.base.Preconditions.checkArgument; import com.google.common.annotations.VisibleForTesting; import io.opentelemetry.api.common.AttributeKey; import io.opentelemetry.api.metrics.Meter; @@ -30,18 +31,20 @@ import io.opentelemetry.sdk.resources.Resource; import io.opentelemetry.sdk.resources.ResourceBuilder; import java.io.Closeable; -import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Objects; import lombok.Singular; +import org.apache.commons.lang3.StringUtils; public class OpenTelemetryService implements Closeable { private static final AttributeKey CLUSTER_NAME_ATTRIBUTE = AttributeKey.stringKey("pulsar.cluster"); private static final AttributeKey SERVICE_NAME_ATTRIBUTE = AttributeKey.stringKey("service.name"); + public static final String OTEL_SDK_DISABLED = "otel.sdk.disabled"; private static final String MAX_CARDINALITY_LIMIT_KEY = "otel.experimental.metrics.cardinality.limit"; + public static final int MAX_CARDINALITY_LIMIT = 10000; private final OpenTelemetrySdk openTelemetrySdk; @@ -52,17 +55,20 @@ public OpenTelemetryService( String serviceName, @Singular Map extraProperties, @VisibleForTesting @Singular List extraMetricReaders) { - Objects.requireNonNull(clusterName); + checkArgument(StringUtils.isNotEmpty(clusterName), "Cluster name cannot be empty"); AutoConfiguredOpenTelemetrySdkBuilder builder = AutoConfiguredOpenTelemetrySdk.builder(); - builder.addPropertiesSupplier( - () -> Collections.singletonMap(MAX_CARDINALITY_LIMIT_KEY, Integer.toString(MAX_CARDINALITY_LIMIT + 1))); + + Map overrideProperties = new HashMap<>(); + overrideProperties.put(OTEL_SDK_DISABLED, "true"); + overrideProperties.put(MAX_CARDINALITY_LIMIT_KEY, Integer.toString(MAX_CARDINALITY_LIMIT + 1)); + builder.addPropertiesSupplier(() -> overrideProperties); builder.addPropertiesSupplier(() -> extraProperties); + builder.addResourceCustomizer( (resource, __) -> { - ResourceBuilder resourceBuilder = Resource.builder(); - resourceBuilder.put(CLUSTER_NAME_ATTRIBUTE, clusterName); + ResourceBuilder resourceBuilder = Resource.builder().put(CLUSTER_NAME_ATTRIBUTE, clusterName); if (serviceName != null) { - resourceBuilder.put(SERVICE_NAME_ATTRIBUTE, serviceName); + resourceBuilder = resourceBuilder.put(SERVICE_NAME_ATTRIBUTE, serviceName); } return resource.merge(resourceBuilder.build()); }); diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java index d201d7a64648b..d0840b22bd584 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java @@ -48,6 +48,7 @@ public void setup() throws Exception { openTelemetryService = OpenTelemetryService.builder(). clusterName("openTelemetryServiceTestCluster"). extraMetricReader(reader). + extraProperty(OpenTelemetryService.OTEL_SDK_DISABLED, "false"). build(); meter = openTelemetryService.getMeter("openTelemetryServiceTestInstrument"); } @@ -58,7 +59,7 @@ public void teardown() throws Exception { reader.close(); } - @Test(expectedExceptions = NullPointerException.class) + @Test(expectedExceptions = IllegalArgumentException.class) public void testIsClusterNameRequired() throws Exception { @Cleanup OpenTelemetryService ots = OpenTelemetryService.builder().build(); @@ -73,6 +74,7 @@ public void testIsClusterNameSet() throws Exception { OpenTelemetryService ots = OpenTelemetryService.builder(). clusterName("testCluster"). extraMetricReader(reader). + extraProperty(OpenTelemetryService.OTEL_SDK_DISABLED, "false"). build(); Predicate predicate = MetricDataMatcher.builder(). @@ -93,6 +95,7 @@ public void testIsServiceNameSet() throws Exception { clusterName("testCluster"). serviceName("testServiceName"). extraMetricReader(reader). + extraProperty(OpenTelemetryService.OTEL_SDK_DISABLED, "false"). build(); Predicate predicate = MetricDataMatcher.builder(). From d204f999e10b83026a3f2888de9a543e7f321d02 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 22 Jan 2024 16:19:47 -0800 Subject: [PATCH 042/133] Add TODO regarding prometheus exporter issue --- .../apache/pulsar/tests/integration/metrics/MetricsTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java index bce80131f5cda..b29dc74e4f893 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java @@ -84,6 +84,8 @@ public void testOpenTelemetryMetrics() throws Exception { Awaitility.waitAtMost(180, TimeUnit.SECONDS).pollInterval(1, TimeUnit.SECONDS).until(() -> { var metricName = "queueSize_ratio"; // Sent automatically by the OpenTelemetry SDK. var metrics = openTelemetryCollectorContainer.getMetricsClient().getMetrics(); + // TODO: Validate cluster name is present once + // https://github.com/open-telemetry/opentelemetry-java/issues/6108 is solved. var brokerMetrics = metrics.findByNameAndLabels(metricName, Pair.of("job", brokerOtelServiceName)); var proxyMetrics = metrics.findByNameAndLabels(metricName, Pair.of("job", proxyOtelServiceName)); var functionWorkerMetrics = From f35685f55fc530f1bfc63435c5df0fe4569997ad Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 22 Jan 2024 18:42:17 -0800 Subject: [PATCH 043/133] Fix function worker integration testing --- .../integration/metrics/MetricsTest.java | 39 +++++++++++++++---- 1 file changed, 31 insertions(+), 8 deletions(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java index b29dc74e4f893..0fb87e7706ae6 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java @@ -24,7 +24,13 @@ import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.naming.TopicDomain; +import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.tests.integration.containers.OpenTelemetryCollectorContainer; +import org.apache.pulsar.tests.integration.functions.PulsarFunctionsTest; +import org.apache.pulsar.tests.integration.functions.utils.CommandGenerator; +import org.apache.pulsar.tests.integration.topologies.FunctionRuntimeType; import org.apache.pulsar.tests.integration.topologies.PulsarCluster; import org.apache.pulsar.tests.integration.topologies.PulsarClusterSpec; import org.apache.pulsar.tests.integration.topologies.PulsarTestBase; @@ -58,20 +64,18 @@ public void testOpenTelemetryMetrics() throws Exception { ); var functionWorkerServiceNameSuffix = PulsarTestBase.randomName(); + var functionWorkerOtelServiceName = "function-worker-" + functionWorkerServiceNameSuffix; var functionWorkerCollectorProps = Map.of( "OTEL_SDK_DISABLED", "false", "OTEL_METRICS_EXPORTER", "otlp", "OTEL_METRIC_EXPORT_INTERVAL", "1000", "OTEL_EXPORTER_OTLP_ENDPOINT", openTelemetryCollectorContainer.getOtlpEndpoint(), - "OTEL_SERVICE_NAME", functionWorkerServiceNameSuffix + "OTEL_SERVICE_NAME", functionWorkerOtelServiceName ); var spec = PulsarClusterSpec.builder() .clusterName(clusterName) - .numBookies(1) - .numBrokers(1) .brokerEnvs(brokerCollectorProps) - .numProxies(1) .proxyEnvs(proxyCollectorProps) .externalService("otel-collector", openTelemetryCollectorContainer) .functionWorkerEnv(functionWorkerServiceNameSuffix, functionWorkerCollectorProps) @@ -79,9 +83,13 @@ public void testOpenTelemetryMetrics() throws Exception { @Cleanup("stop") var pulsarCluster = PulsarCluster.forSpec(spec); pulsarCluster.start(); - // pulsarCluster.setupFunctionWorkers(functionWorkerServiceNameSuffix, FunctionRuntimeType.PROCESS, 1); + pulsarCluster.setupFunctionWorkers(functionWorkerServiceNameSuffix, FunctionRuntimeType.PROCESS, 1); - Awaitility.waitAtMost(180, TimeUnit.SECONDS).pollInterval(1, TimeUnit.SECONDS).until(() -> { + var serviceUrl = pulsarCluster.getPlainTextServiceUrl(); + var functionWorkerCommand = getFunctionWorkerCommand(serviceUrl, functionWorkerServiceNameSuffix); + pulsarCluster.getAnyWorker().execCmdAsync(functionWorkerCommand.split(" ")); + + Awaitility.waitAtMost(1800, TimeUnit.SECONDS).pollInterval(1, TimeUnit.SECONDS).until(() -> { var metricName = "queueSize_ratio"; // Sent automatically by the OpenTelemetry SDK. var metrics = openTelemetryCollectorContainer.getMetricsClient().getMetrics(); // TODO: Validate cluster name is present once @@ -89,8 +97,23 @@ public void testOpenTelemetryMetrics() throws Exception { var brokerMetrics = metrics.findByNameAndLabels(metricName, Pair.of("job", brokerOtelServiceName)); var proxyMetrics = metrics.findByNameAndLabels(metricName, Pair.of("job", proxyOtelServiceName)); var functionWorkerMetrics = - metrics.findByNameAndLabels(metricName, Pair.of("job", functionWorkerServiceNameSuffix)); - return !brokerMetrics.isEmpty() && !proxyMetrics.isEmpty(); + metrics.findByNameAndLabels(metricName, Pair.of("job", functionWorkerOtelServiceName)); + return !brokerMetrics.isEmpty() && !proxyMetrics.isEmpty() && !functionWorkerMetrics.isEmpty(); }); } + + private String getFunctionWorkerCommand(String serviceUrl, String suffix) { + var namespace = NamespaceName.get("public", "default"); + var sourceTopicName = TopicName.get(TopicDomain.persistent.toString(), namespace, "metricTestSource-" + suffix); + var sinkTopicName = TopicName.get(TopicDomain.persistent.toString(), namespace, "metricTestSink-" + suffix); + + var commandGenerator = new CommandGenerator(); + commandGenerator.setAdminUrl(serviceUrl); + commandGenerator.setSourceTopic(sourceTopicName.toString()); + commandGenerator.setSinkTopic(sinkTopicName.toString()); + commandGenerator.setFunctionName("metricsTestLocalRunTest-" + suffix); + commandGenerator.setRuntime(CommandGenerator.Runtime.JAVA); + commandGenerator.setFunctionClassName(PulsarFunctionsTest.EXCLAMATION_JAVA_CLASS); + return commandGenerator.generateLocalRunCommand(null); + } } From 3efb8667a0872a0a70a04f12e33afb72de14389b Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 22 Jan 2024 20:11:45 -0800 Subject: [PATCH 044/133] Cosmetic fix --- .../integration/metrics/MetricsTest.java | 46 ++++++++----------- 1 file changed, 20 insertions(+), 26 deletions(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java index 0fb87e7706ae6..5132b70096bdd 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java @@ -40,38 +40,21 @@ @Slf4j public class MetricsTest { - @Test - public void testOpenTelemetryMetrics() throws Exception { + @Test(timeOut = 300_000) + public void testOpenTelemetryMetricsPresent() throws Exception { var clusterName = "testOpenTelemetryMetrics-" + UUID.randomUUID(); var openTelemetryCollectorContainer = new OpenTelemetryCollectorContainer(clusterName); var brokerOtelServiceName = clusterName + "-broker"; - var brokerCollectorProps = Map.of( - "OTEL_SDK_DISABLED", "false", - "OTEL_METRICS_EXPORTER", "otlp", - "OTEL_METRIC_EXPORT_INTERVAL", "1000", - "OTEL_EXPORTER_OTLP_ENDPOINT", openTelemetryCollectorContainer.getOtlpEndpoint(), - "OTEL_SERVICE_NAME", brokerOtelServiceName - ); + var brokerCollectorProps = getCollectorProps(openTelemetryCollectorContainer, brokerOtelServiceName); var proxyOtelServiceName = clusterName + "-proxy"; - var proxyCollectorProps = Map.of( - "OTEL_SDK_DISABLED", "false", - "OTEL_METRICS_EXPORTER", "otlp", - "OTEL_METRIC_EXPORT_INTERVAL", "1000", - "OTEL_EXPORTER_OTLP_ENDPOINT", openTelemetryCollectorContainer.getOtlpEndpoint(), - "OTEL_SERVICE_NAME", proxyOtelServiceName - ); + var proxyCollectorProps = getCollectorProps(openTelemetryCollectorContainer, proxyOtelServiceName); var functionWorkerServiceNameSuffix = PulsarTestBase.randomName(); var functionWorkerOtelServiceName = "function-worker-" + functionWorkerServiceNameSuffix; - var functionWorkerCollectorProps = Map.of( - "OTEL_SDK_DISABLED", "false", - "OTEL_METRICS_EXPORTER", "otlp", - "OTEL_METRIC_EXPORT_INTERVAL", "1000", - "OTEL_EXPORTER_OTLP_ENDPOINT", openTelemetryCollectorContainer.getOtlpEndpoint(), - "OTEL_SERVICE_NAME", functionWorkerOtelServiceName - ); + var functionWorkerCollectorProps = + getCollectorProps(openTelemetryCollectorContainer, functionWorkerOtelServiceName); var spec = PulsarClusterSpec.builder() .clusterName(clusterName) @@ -83,13 +66,13 @@ public void testOpenTelemetryMetrics() throws Exception { @Cleanup("stop") var pulsarCluster = PulsarCluster.forSpec(spec); pulsarCluster.start(); - pulsarCluster.setupFunctionWorkers(functionWorkerServiceNameSuffix, FunctionRuntimeType.PROCESS, 1); + pulsarCluster.setupFunctionWorkers(functionWorkerServiceNameSuffix, FunctionRuntimeType.PROCESS, 1); var serviceUrl = pulsarCluster.getPlainTextServiceUrl(); var functionWorkerCommand = getFunctionWorkerCommand(serviceUrl, functionWorkerServiceNameSuffix); pulsarCluster.getAnyWorker().execCmdAsync(functionWorkerCommand.split(" ")); - Awaitility.waitAtMost(1800, TimeUnit.SECONDS).pollInterval(1, TimeUnit.SECONDS).until(() -> { + Awaitility.waitAtMost(90, TimeUnit.SECONDS).pollInterval(1, TimeUnit.SECONDS).until(() -> { var metricName = "queueSize_ratio"; // Sent automatically by the OpenTelemetry SDK. var metrics = openTelemetryCollectorContainer.getMetricsClient().getMetrics(); // TODO: Validate cluster name is present once @@ -102,7 +85,18 @@ public void testOpenTelemetryMetrics() throws Exception { }); } - private String getFunctionWorkerCommand(String serviceUrl, String suffix) { + private static Map getCollectorProps( + OpenTelemetryCollectorContainer openTelemetryCollectorContainer, String functionWorkerOtelServiceName) { + return Map.of( + "OTEL_SDK_DISABLED", "false", + "OTEL_METRICS_EXPORTER", "otlp", + "OTEL_METRIC_EXPORT_INTERVAL", "1000", + "OTEL_EXPORTER_OTLP_ENDPOINT", openTelemetryCollectorContainer.getOtlpEndpoint(), + "OTEL_SERVICE_NAME", functionWorkerOtelServiceName + ); + } + + private static String getFunctionWorkerCommand(String serviceUrl, String suffix) { var namespace = NamespaceName.get("public", "default"); var sourceTopicName = TopicName.get(TopicDomain.persistent.toString(), namespace, "metricTestSource-" + suffix); var sinkTopicName = TopicName.get(TopicDomain.persistent.toString(), namespace, "metricTestSink-" + suffix); From 86dd4fab1c07a68bd9365aa6e55f3bf8006f7ca4 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 22 Jan 2024 20:35:01 -0800 Subject: [PATCH 045/133] Remove extra JAR imports --- pom.xml | 10 +++++++-- pulsar-common/pom.xml | 51 ------------------------------------------- 2 files changed, 8 insertions(+), 53 deletions(-) diff --git a/pom.xml b/pom.xml index 42efa175094bb..2a59f88b4989a 100644 --- a/pom.xml +++ b/pom.xml @@ -249,8 +249,6 @@ flexible messaging model and an intuitive client API. 1.5.2-3 2.0.6 1.34.1 - 1.23.1-alpha - 1.32.0 1.18.3 @@ -1436,6 +1434,14 @@ flexible messaging model and an intuitive client API. ${restassured.version} test + + + io.opentelemetry + opentelemetry-bom + ${opentelemetry.version} + pom + import + diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index abddfacca30ce..88fc67cccf66d 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -34,26 +34,6 @@ Pulsar Common Common libraries needed by client/broker/tools - - - - io.opentelemetry - opentelemetry-bom - ${opentelemetry.version} - pom - import - - - - io.opentelemetry - opentelemetry-bom-alpha - ${opentelemetry.version}-alpha - pom - import - - - - ${project.groupId} @@ -226,22 +206,6 @@ io.opentelemetry opentelemetry-api - - io.opentelemetry - opentelemetry-api-events - - - io.opentelemetry - opentelemetry-sdk - - - io.opentelemetry - opentelemetry-sdk-metrics - - - io.opentelemetry - opentelemetry-exporter-logging - io.opentelemetry opentelemetry-exporter-otlp @@ -250,21 +214,6 @@ io.opentelemetry opentelemetry-sdk-extension-autoconfigure - - io.opentelemetry - opentelemetry-sdk-extension-autoconfigure-spi - - - - io.opentelemetry.semconv - opentelemetry-semconv - ${opentelemetry.semconv.version} - - - io.opentelemetry.javaagent - opentelemetry-javaagent - ${opentelemetry.javaagent.version} - From 3dccf2e3bcda64941c200f7f2086d6127cc73ab5 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 22 Jan 2024 20:39:11 -0800 Subject: [PATCH 046/133] Typo fix --- pulsar-common/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index 88fc67cccf66d..2c2017554f1a3 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -198,7 +198,7 @@ com.google.protobuf - protobuf-java + protobuf-java From e93c49607a4d6399f095f40f662973dc71bed6f9 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 22 Jan 2024 20:47:11 -0800 Subject: [PATCH 047/133] pom cleanup --- pulsar-common/pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index 2c2017554f1a3..b5abb5cbbab72 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -256,7 +256,6 @@ io.opentelemetry opentelemetry-sdk-testing - ${opentelemetry.version} test From a6b73b576ae9f202ebcbc0883b6a83e76d5103ab Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 22 Jan 2024 20:47:57 -0800 Subject: [PATCH 048/133] Remove prometheus container --- .../containers/PrometheusContainer.java | 55 ------------------- .../test/resources/containers/prometheus.yaml | 25 --------- 2 files changed, 80 deletions(-) delete mode 100644 tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PrometheusContainer.java delete mode 100644 tests/integration/src/test/resources/containers/prometheus.yaml diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PrometheusContainer.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PrometheusContainer.java deleted file mode 100644 index e96378f171e3d..0000000000000 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PrometheusContainer.java +++ /dev/null @@ -1,55 +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.tests.integration.containers; - -import org.testcontainers.containers.wait.strategy.HostPortWaitStrategy; -import org.testcontainers.utility.MountableFile; - -public class PrometheusContainer extends ChaosContainer { - - private static final String IMAGE_NAME = "prom/prometheus:latest"; - private static final String NAME = "prometheus"; - private static final int PROMETHEUS_PORT = 9090; - - public PrometheusContainer(String clusterName) { - super(clusterName, IMAGE_NAME); - } - - @Override - protected void configure() { - super.configure(); - - this.withNetworkAliases(NAME) - .withCopyToContainer( - MountableFile.forClasspathResource("containers/prometheus.yaml"), - "/etc/prometheus/prometheus.yml") - .withExposedPorts(PROMETHEUS_PORT) - .withCreateContainerCmdModifier(createContainerCmd -> { - createContainerCmd.withHostName(NAME); - createContainerCmd.withName(getContainerName()); - }) - .waitingFor(new HostPortWaitStrategy()); - tailContainerLog(); - } - - @Override - public String getContainerName() { - return clusterName + "-" + NAME; - } -} diff --git a/tests/integration/src/test/resources/containers/prometheus.yaml b/tests/integration/src/test/resources/containers/prometheus.yaml deleted file mode 100644 index 6407df7915c78..0000000000000 --- a/tests/integration/src/test/resources/containers/prometheus.yaml +++ /dev/null @@ -1,25 +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. -# - -scrape_configs: - - job_name: 'otel-collector' - scrape_interval: 2s - static_configs: - - targets: ['otel-collector:8889'] - - targets: ['otel-collector:8888'] \ No newline at end of file From 266f81b9bf0d6a7e6fa0049800b643baa4fa0b08 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 22 Jan 2024 21:34:55 -0800 Subject: [PATCH 049/133] Remove serviceName parameter in OpenTelemetryService builder --- .../common/stats/OpenTelemetryService.java | 13 +++--------- .../stats/OpenTelemetryServiceTest.java | 21 ------------------- 2 files changed, 3 insertions(+), 31 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index 45fb8e3ed3a68..5e3b26fa96832 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -29,7 +29,6 @@ import io.opentelemetry.sdk.metrics.internal.SdkMeterProviderUtil; import io.opentelemetry.sdk.metrics.internal.export.CardinalityLimitSelector; import io.opentelemetry.sdk.resources.Resource; -import io.opentelemetry.sdk.resources.ResourceBuilder; import java.io.Closeable; import java.util.HashMap; import java.util.List; @@ -40,7 +39,6 @@ public class OpenTelemetryService implements Closeable { private static final AttributeKey CLUSTER_NAME_ATTRIBUTE = AttributeKey.stringKey("pulsar.cluster"); - private static final AttributeKey SERVICE_NAME_ATTRIBUTE = AttributeKey.stringKey("service.name"); public static final String OTEL_SDK_DISABLED = "otel.sdk.disabled"; private static final String MAX_CARDINALITY_LIMIT_KEY = "otel.experimental.metrics.cardinality.limit"; @@ -52,7 +50,6 @@ public class OpenTelemetryService implements Closeable { @lombok.Builder public OpenTelemetryService( String clusterName, - String serviceName, @Singular Map extraProperties, @VisibleForTesting @Singular List extraMetricReaders) { checkArgument(StringUtils.isNotEmpty(clusterName), "Cluster name cannot be empty"); @@ -65,19 +62,15 @@ public OpenTelemetryService( builder.addPropertiesSupplier(() -> extraProperties); builder.addResourceCustomizer( - (resource, __) -> { - ResourceBuilder resourceBuilder = Resource.builder().put(CLUSTER_NAME_ATTRIBUTE, clusterName); - if (serviceName != null) { - resourceBuilder = resourceBuilder.put(SERVICE_NAME_ATTRIBUTE, serviceName); - } - return resource.merge(resourceBuilder.build()); - }); + (resource, __) -> resource.merge(Resource.builder().put(CLUSTER_NAME_ATTRIBUTE, clusterName).build())); + final CardinalityLimitSelector cardinalityLimitSelector = __ -> MAX_CARDINALITY_LIMIT + 1; extraMetricReaders.forEach(metricReader -> builder.addMeterProviderCustomizer((sdkMeterProviderBuilder, __) -> { SdkMeterProviderUtil.registerMetricReaderWithCardinalitySelector( sdkMeterProviderBuilder, metricReader, cardinalityLimitSelector); return sdkMeterProviderBuilder; })); + openTelemetrySdk = builder.build().getOpenTelemetrySdk(); } diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java index d0840b22bd584..9551e5d65f6b7 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java @@ -85,27 +85,6 @@ public void testIsClusterNameSet() throws Exception { Assert.assertTrue(metricData.stream().anyMatch(predicate)); } - @Test - public void testIsServiceNameSet() throws Exception { - @Cleanup - InMemoryMetricReader reader = InMemoryMetricReader.create(); - - @Cleanup - OpenTelemetryService ots = OpenTelemetryService.builder(). - clusterName("testCluster"). - serviceName("testServiceName"). - extraMetricReader(reader). - extraProperty(OpenTelemetryService.OTEL_SDK_DISABLED, "false"). - build(); - - Predicate predicate = MetricDataMatcher.builder(). - resourceAttribute(Attributes.of(AttributeKey.stringKey("service.name"), "testServiceName")). - build(); - - Collection metricData = reader.collectAllMetrics(); - Assert.assertTrue(metricData.stream().anyMatch(predicate)); - } - @Test public void testIsInstrumentationNameSetOnMeter() throws Exception { Meter meter = openTelemetryService.getMeter("testInstrumentationScope"); From 569d134a399990d2ba2e1b5596db29b9c6676a31 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 22 Jan 2024 21:38:53 -0800 Subject: [PATCH 050/133] Split test testIsClusterNameRequired into two --- .../pulsar/common/stats/OpenTelemetryServiceTest.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java index 9551e5d65f6b7..671e09b7b2661 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java @@ -31,6 +31,7 @@ import java.util.Collection; import java.util.function.Predicate; import lombok.Cleanup; +import org.apache.commons.lang3.StringUtils; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -60,11 +61,17 @@ public void teardown() throws Exception { } @Test(expectedExceptions = IllegalArgumentException.class) - public void testIsClusterNameRequired() throws Exception { + public void testIsClusterNameCannotBeNull() throws Exception { @Cleanup OpenTelemetryService ots = OpenTelemetryService.builder().build(); } + @Test(expectedExceptions = IllegalArgumentException.class) + public void testIsClusterNameCannotBeEmpty() throws Exception { + @Cleanup + OpenTelemetryService ots = OpenTelemetryService.builder().clusterName(StringUtils.EMPTY).build(); + } + @Test public void testIsClusterNameSet() throws Exception { @Cleanup From 20e48d31520b1ea838288ef4e60423439a188b81 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 23 Jan 2024 12:02:35 -0800 Subject: [PATCH 051/133] Add test todo --- .../apache/pulsar/tests/integration/metrics/MetricsTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java index 5132b70096bdd..deb883676d42b 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java @@ -40,6 +40,8 @@ @Slf4j public class MetricsTest { + // Test with the included Prometheus exporter as well + // https://github.com/open-telemetry/opentelemetry-java/blob/main/sdk-extensions/autoconfigure/README.md#prometheus-exporter @Test(timeOut = 300_000) public void testOpenTelemetryMetricsPresent() throws Exception { var clusterName = "testOpenTelemetryMetrics-" + UUID.randomUUID(); From a91822ad19c2258d19931699e4636cf077d2d595 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 23 Jan 2024 13:46:26 -0800 Subject: [PATCH 052/133] Add prometheus exporter JARs --- pom.xml | 8 ++ pulsar-common/pom.xml | 4 + .../integration/metrics/MetricsTest.java | 94 +++++++++++++++++-- .../integration/topologies/PulsarCluster.java | 7 ++ .../topologies/PulsarClusterSpec.java | 12 +++ 5 files changed, 117 insertions(+), 8 deletions(-) diff --git a/pom.xml b/pom.xml index 2a59f88b4989a..38c2484d06b3a 100644 --- a/pom.xml +++ b/pom.xml @@ -249,6 +249,7 @@ flexible messaging model and an intuitive client API. 1.5.2-3 2.0.6 1.34.1 + 1.34.1-alpha 1.18.3 @@ -1442,6 +1443,13 @@ flexible messaging model and an intuitive client API. pom import + + io.opentelemetry + opentelemetry-bom-alpha + ${opentelemetry.alpha.version} + pom + import + diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index b5abb5cbbab72..71ef32cbc3af0 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -210,6 +210,10 @@ io.opentelemetry opentelemetry-exporter-otlp + + io.opentelemetry + opentelemetry-exporter-prometheus + io.opentelemetry opentelemetry-sdk-extension-autoconfigure diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java index deb883676d42b..a70ace9708bbc 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java @@ -18,16 +18,19 @@ */ package org.apache.pulsar.tests.integration.metrics; +import java.util.List; import java.util.Map; import java.util.UUID; import java.util.concurrent.TimeUnit; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.tests.integration.containers.OpenTelemetryCollectorContainer; +import org.apache.pulsar.tests.integration.containers.PulsarContainer; import org.apache.pulsar.tests.integration.functions.PulsarFunctionsTest; import org.apache.pulsar.tests.integration.functions.utils.CommandGenerator; import org.apache.pulsar.tests.integration.topologies.FunctionRuntimeType; @@ -43,20 +46,20 @@ public class MetricsTest { // Test with the included Prometheus exporter as well // https://github.com/open-telemetry/opentelemetry-java/blob/main/sdk-extensions/autoconfigure/README.md#prometheus-exporter @Test(timeOut = 300_000) - public void testOpenTelemetryMetricsPresent() throws Exception { + public void testOpenTelemetryMetricsOtlpExport() throws Exception { var clusterName = "testOpenTelemetryMetrics-" + UUID.randomUUID(); var openTelemetryCollectorContainer = new OpenTelemetryCollectorContainer(clusterName); var brokerOtelServiceName = clusterName + "-broker"; - var brokerCollectorProps = getCollectorProps(openTelemetryCollectorContainer, brokerOtelServiceName); + var brokerCollectorProps = getCollectorProps(brokerOtelServiceName, openTelemetryCollectorContainer); var proxyOtelServiceName = clusterName + "-proxy"; - var proxyCollectorProps = getCollectorProps(openTelemetryCollectorContainer, proxyOtelServiceName); + var proxyCollectorProps = getCollectorProps(proxyOtelServiceName, openTelemetryCollectorContainer); var functionWorkerServiceNameSuffix = PulsarTestBase.randomName(); var functionWorkerOtelServiceName = "function-worker-" + functionWorkerServiceNameSuffix; var functionWorkerCollectorProps = - getCollectorProps(openTelemetryCollectorContainer, functionWorkerOtelServiceName); + getCollectorProps(functionWorkerOtelServiceName, openTelemetryCollectorContainer); var spec = PulsarClusterSpec.builder() .clusterName(clusterName) @@ -74,8 +77,8 @@ public void testOpenTelemetryMetricsPresent() throws Exception { var functionWorkerCommand = getFunctionWorkerCommand(serviceUrl, functionWorkerServiceNameSuffix); pulsarCluster.getAnyWorker().execCmdAsync(functionWorkerCommand.split(" ")); + var metricName = "queueSize_ratio"; // Sent automatically by the OpenTelemetry SDK. Awaitility.waitAtMost(90, TimeUnit.SECONDS).pollInterval(1, TimeUnit.SECONDS).until(() -> { - var metricName = "queueSize_ratio"; // Sent automatically by the OpenTelemetry SDK. var metrics = openTelemetryCollectorContainer.getMetricsClient().getMetrics(); // TODO: Validate cluster name is present once // https://github.com/open-telemetry/opentelemetry-java/issues/6108 is solved. @@ -87,14 +90,89 @@ public void testOpenTelemetryMetricsPresent() throws Exception { }); } + @Test(timeOut = 300_000) + public void testOpenTelemetryMetricsPrometheusExport() throws Exception { + var prometheusExporterPort = 9464; + var clusterName = "testOpenTelemetryMetrics-" + UUID.randomUUID(); + + var brokerOtelServiceName = clusterName + "-broker"; + var brokerCollectorProps = getCollectorProps(brokerOtelServiceName, prometheusExporterPort); + + var proxyOtelServiceName = clusterName + "-proxy"; + var proxyCollectorProps = getCollectorProps(proxyOtelServiceName, prometheusExporterPort); + + var functionWorkerServiceNameSuffix = PulsarTestBase.randomName(); + var functionWorkerOtelServiceName = "function-worker-" + functionWorkerServiceNameSuffix; + var functionWorkerCollectorProps = + getCollectorProps(functionWorkerOtelServiceName, prometheusExporterPort); + + var spec = PulsarClusterSpec.builder() + .clusterName(clusterName) + .brokerEnvs(brokerCollectorProps) + .brokerAdditionalPorts(List.of(prometheusExporterPort)) + .proxyEnvs(proxyCollectorProps) + .proxyAdditionalPorts(List.of(prometheusExporterPort)) + .functionWorkerEnv(functionWorkerServiceNameSuffix, functionWorkerCollectorProps) + .functionWorkerAdditionalPort(functionWorkerServiceNameSuffix, List.of(prometheusExporterPort)) + .build(); + @Cleanup("stop") + var pulsarCluster = PulsarCluster.forSpec(spec); + pulsarCluster.start(); + + pulsarCluster.setupFunctionWorkers(functionWorkerServiceNameSuffix, FunctionRuntimeType.PROCESS, 1); + var serviceUrl = pulsarCluster.getPlainTextServiceUrl(); + var functionWorkerCommand = getFunctionWorkerCommand(serviceUrl, functionWorkerServiceNameSuffix); + var workerContainer = pulsarCluster.getAnyWorker(); + workerContainer.execCmdAsync(functionWorkerCommand.split(" ")); + + var metricName = "target_info"; // Sent automatically by the OpenTelemetry SDK. + Awaitility.waitAtMost(90, TimeUnit.SECONDS).pollInterval(1, TimeUnit.SECONDS).until(() -> { + var prometheusClient = createMetricsClient(pulsarCluster.getAnyBroker(), prometheusExporterPort); + var metrics = prometheusClient.getMetrics(); + var expectedMetrics = metrics.findByNameAndLabels(metricName, + Pair.of("pulsar_cluster", clusterName), Pair.of("service_name", brokerOtelServiceName)); + return !expectedMetrics.isEmpty(); + }); + + Awaitility.waitAtMost(90, TimeUnit.SECONDS).pollInterval(1, TimeUnit.SECONDS).until(() -> { + var prometheusClient = createMetricsClient(pulsarCluster.getProxy(), prometheusExporterPort); + var metrics = prometheusClient.getMetrics(); + var expectedMetrics = metrics.findByNameAndLabels(metricName, + Pair.of("pulsar_cluster", clusterName), Pair.of("service_name", proxyOtelServiceName)); + return !expectedMetrics.isEmpty(); + }); + + Awaitility.waitAtMost(90, TimeUnit.SECONDS).pollInterval(1, TimeUnit.SECONDS).until(() -> { + var prometheusClient = createMetricsClient(workerContainer, prometheusExporterPort); + var metrics = prometheusClient.getMetrics(); + var expectedMetrics = metrics.findByNameAndLabels(metricName, + Pair.of("pulsar_cluster", clusterName), Pair.of("service_name", functionWorkerOtelServiceName)); + return !expectedMetrics.isEmpty(); + }); + } + + private static PrometheusMetricsClient createMetricsClient(PulsarContainer container, int port) { + return new PrometheusMetricsClient(container.getHost(), container.getMappedPort(port)); + } + private static Map getCollectorProps( - OpenTelemetryCollectorContainer openTelemetryCollectorContainer, String functionWorkerOtelServiceName) { + String serviceName, OpenTelemetryCollectorContainer openTelemetryCollectorContainer) { return Map.of( "OTEL_SDK_DISABLED", "false", + "OTEL_SERVICE_NAME", serviceName, "OTEL_METRICS_EXPORTER", "otlp", - "OTEL_METRIC_EXPORT_INTERVAL", "1000", "OTEL_EXPORTER_OTLP_ENDPOINT", openTelemetryCollectorContainer.getOtlpEndpoint(), - "OTEL_SERVICE_NAME", functionWorkerOtelServiceName + "OTEL_METRIC_EXPORT_INTERVAL", "1000" + ); + } + + private static Map getCollectorProps(String serviceName, int prometheusExporterPort) { + return Map.of( + "OTEL_SDK_DISABLED", "false", + "OTEL_SERVICE_NAME", serviceName, + "OTEL_METRICS_EXPORTER", "prometheus", + "OTEL_EXPORTER_PROMETHEUS_PORT", Integer.toString(prometheusExporterPort), + "OTEL_METRIC_EXPORT_INTERVAL", "1000" ); } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarCluster.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarCluster.java index a6603838cca56..795693a844ff7 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarCluster.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarCluster.java @@ -95,6 +95,7 @@ public static PulsarCluster forSpec(PulsarClusterSpec spec, CSContainer csContai private Map> externalServices = Collections.emptyMap(); private Map> externalServiceEnvs; private Map> functionWorkerEnvs; + private Map> functionWorkerAdditionalPorts; private PulsarCluster(PulsarClusterSpec spec, CSContainer csContainer, boolean sharedCsContainer) { @@ -151,6 +152,9 @@ private PulsarCluster(PulsarClusterSpec spec, CSContainer csContainer, boolean s if (spec.proxyMountFiles != null) { spec.proxyMountFiles.forEach(this.proxyContainer::withFileSystemBind); } + if (spec.proxyAdditionalPorts != null) { + spec.proxyAdditionalPorts.forEach(this.proxyContainer::addExposedPort); + } // create bookies bookieContainers.putAll( @@ -329,6 +333,7 @@ public void start() throws Exception { } this.functionWorkerEnvs = spec.functionWorkerEnvs; + this.functionWorkerAdditionalPorts = spec.functionWorkerAdditionalPorts; } public void startService(String networkAlias, @@ -439,6 +444,8 @@ private void startFunctionWorkersWithProcessContainerFactory(String suffix, int // bookkeeper tools .withEnv("zkServers", ZKContainer.NAME) .withEnv(functionWorkerEnvs.getOrDefault(suffix, Collections.emptyMap())) + .withExposedPorts(functionWorkerAdditionalPorts.getOrDefault(suffix, Collections.emptyList()) + .toArray(new Integer[0])) )); this.startWorkers(); } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java index b0b56c8c6e3d8..7222bb4991159 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java @@ -87,6 +87,13 @@ public class PulsarClusterSpec { @Singular Map> functionWorkerEnvs; + /** + * Additional ports to expose on function workers. + */ + @Singular + Map> functionWorkerAdditionalPorts; + + /** * Allow to query the last message */ @@ -176,6 +183,11 @@ public class PulsarClusterSpec { */ List bookieAdditionalPorts; + /** + * Additional ports to expose on proxy containers. + */ + List proxyAdditionalPorts; + /** * Enable TLS for connection. */ From 6de2f4678d155cc337fa1786bb11d8bbbff9682b Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 25 Jan 2024 15:52:57 -0800 Subject: [PATCH 053/133] Allow cluster name to be overriden by env variables --- .../pulsar/common/stats/OpenTelemetryService.java | 7 ++++++- .../tests/integration/metrics/MetricsTest.java | 15 +++++++++++---- 2 files changed, 17 insertions(+), 5 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index 5e3b26fa96832..945a19b3bd0d9 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -62,7 +62,12 @@ public OpenTelemetryService( builder.addPropertiesSupplier(() -> extraProperties); builder.addResourceCustomizer( - (resource, __) -> resource.merge(Resource.builder().put(CLUSTER_NAME_ATTRIBUTE, clusterName).build())); + (resource, __) -> { + if (resource.getAttribute(CLUSTER_NAME_ATTRIBUTE) != null) { + return resource; // Do not override the attribute if already set. + } + return resource.merge(Resource.builder().put(CLUSTER_NAME_ATTRIBUTE, clusterName).build()); + }); final CardinalityLimitSelector cardinalityLimitSelector = __ -> MAX_CARDINALITY_LIMIT + 1; extraMetricReaders.forEach(metricReader -> builder.addMeterProviderCustomizer((sdkMeterProviderBuilder, __) -> { diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java index a70ace9708bbc..7be45e2b54fc6 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java @@ -18,6 +18,8 @@ */ package org.apache.pulsar.tests.integration.metrics; +import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.UUID; @@ -103,8 +105,9 @@ public void testOpenTelemetryMetricsPrometheusExport() throws Exception { var functionWorkerServiceNameSuffix = PulsarTestBase.randomName(); var functionWorkerOtelServiceName = "function-worker-" + functionWorkerServiceNameSuffix; - var functionWorkerCollectorProps = - getCollectorProps(functionWorkerOtelServiceName, prometheusExporterPort); + // Locally run Pulsar Functions override cluster name definitions to "local". + var functionWorkerCollectorProps = getCollectorProps(functionWorkerOtelServiceName, prometheusExporterPort, + Pair.of("OTEL_RESOURCE_ATTRIBUTES", "pulsar.cluster=" + clusterName)); var spec = PulsarClusterSpec.builder() .clusterName(clusterName) @@ -166,14 +169,18 @@ private static Map getCollectorProps( ); } - private static Map getCollectorProps(String serviceName, int prometheusExporterPort) { - return Map.of( + private static Map getCollectorProps(String serviceName, int prometheusExporterPort, + Pair ... extraProps) { + var defaultProps = Map.of( "OTEL_SDK_DISABLED", "false", "OTEL_SERVICE_NAME", serviceName, "OTEL_METRICS_EXPORTER", "prometheus", "OTEL_EXPORTER_PROMETHEUS_PORT", Integer.toString(prometheusExporterPort), "OTEL_METRIC_EXPORT_INTERVAL", "1000" ); + var props = new HashMap<>(defaultProps); + Arrays.stream(extraProps).forEach(p -> props.put(p.getKey(), p.getValue())); + return props; } private static String getFunctionWorkerCommand(String serviceUrl, String suffix) { From c1401af101680ef6a92d9cf7777e7c19502e9011 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 29 Jan 2024 15:23:51 -0800 Subject: [PATCH 054/133] Test fixes --- .../integration/metrics/MetricsTest.java | 50 +++++++++++++------ 1 file changed, 36 insertions(+), 14 deletions(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java index 7be45e2b54fc6..1a13de59c38fd 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.tests.integration.metrics; +import java.net.URL; import java.util.Arrays; import java.util.HashMap; import java.util.List; @@ -28,10 +29,12 @@ import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; +import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.tests.integration.containers.OpenTelemetryCollectorContainer; +import org.apache.pulsar.tests.integration.containers.ProxyContainer; import org.apache.pulsar.tests.integration.containers.PulsarContainer; import org.apache.pulsar.tests.integration.functions.PulsarFunctionsTest; import org.apache.pulsar.tests.integration.functions.utils.CommandGenerator; @@ -74,9 +77,12 @@ public void testOpenTelemetryMetricsOtlpExport() throws Exception { var pulsarCluster = PulsarCluster.forSpec(spec); pulsarCluster.start(); + var useLocalRunner = false; + pulsarCluster.setupFunctionWorkers(functionWorkerServiceNameSuffix, FunctionRuntimeType.PROCESS, 1); var serviceUrl = pulsarCluster.getPlainTextServiceUrl(); - var functionWorkerCommand = getFunctionWorkerCommand(serviceUrl, functionWorkerServiceNameSuffix); + var functionWorkerCommand = + getFunctionWorkerCommand(pulsarCluster, functionWorkerServiceNameSuffix, useLocalRunner); pulsarCluster.getAnyWorker().execCmdAsync(functionWorkerCommand.split(" ")); var metricName = "queueSize_ratio"; // Sent automatically by the OpenTelemetry SDK. @@ -92,7 +98,7 @@ public void testOpenTelemetryMetricsOtlpExport() throws Exception { }); } - @Test(timeOut = 300_000) + @Test(timeOut = 900_000) public void testOpenTelemetryMetricsPrometheusExport() throws Exception { var prometheusExporterPort = 9464; var clusterName = "testOpenTelemetryMetrics-" + UUID.randomUUID(); @@ -106,8 +112,7 @@ public void testOpenTelemetryMetricsPrometheusExport() throws Exception { var functionWorkerServiceNameSuffix = PulsarTestBase.randomName(); var functionWorkerOtelServiceName = "function-worker-" + functionWorkerServiceNameSuffix; // Locally run Pulsar Functions override cluster name definitions to "local". - var functionWorkerCollectorProps = getCollectorProps(functionWorkerOtelServiceName, prometheusExporterPort, - Pair.of("OTEL_RESOURCE_ATTRIBUTES", "pulsar.cluster=" + clusterName)); + var functionWorkerCollectorProps = getCollectorProps(functionWorkerOtelServiceName, prometheusExporterPort); var spec = PulsarClusterSpec.builder() .clusterName(clusterName) @@ -122,14 +127,16 @@ public void testOpenTelemetryMetricsPrometheusExport() throws Exception { var pulsarCluster = PulsarCluster.forSpec(spec); pulsarCluster.start(); + var useLocalRunner = false; + pulsarCluster.setupFunctionWorkers(functionWorkerServiceNameSuffix, FunctionRuntimeType.PROCESS, 1); - var serviceUrl = pulsarCluster.getPlainTextServiceUrl(); - var functionWorkerCommand = getFunctionWorkerCommand(serviceUrl, functionWorkerServiceNameSuffix); + var functionWorkerCommand = + getFunctionWorkerCommand(pulsarCluster, functionWorkerServiceNameSuffix, useLocalRunner); var workerContainer = pulsarCluster.getAnyWorker(); - workerContainer.execCmdAsync(functionWorkerCommand.split(" ")); + workerContainer.execCmdAsync("sh", "-c", functionWorkerCommand); var metricName = "target_info"; // Sent automatically by the OpenTelemetry SDK. - Awaitility.waitAtMost(90, TimeUnit.SECONDS).pollInterval(1, TimeUnit.SECONDS).until(() -> { + Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { var prometheusClient = createMetricsClient(pulsarCluster.getAnyBroker(), prometheusExporterPort); var metrics = prometheusClient.getMetrics(); var expectedMetrics = metrics.findByNameAndLabels(metricName, @@ -137,7 +144,7 @@ public void testOpenTelemetryMetricsPrometheusExport() throws Exception { return !expectedMetrics.isEmpty(); }); - Awaitility.waitAtMost(90, TimeUnit.SECONDS).pollInterval(1, TimeUnit.SECONDS).until(() -> { + Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { var prometheusClient = createMetricsClient(pulsarCluster.getProxy(), prometheusExporterPort); var metrics = prometheusClient.getMetrics(); var expectedMetrics = metrics.findByNameAndLabels(metricName, @@ -145,7 +152,7 @@ public void testOpenTelemetryMetricsPrometheusExport() throws Exception { return !expectedMetrics.isEmpty(); }); - Awaitility.waitAtMost(90, TimeUnit.SECONDS).pollInterval(1, TimeUnit.SECONDS).until(() -> { + Awaitility.waitAtMost(900, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { var prometheusClient = createMetricsClient(workerContainer, prometheusExporterPort); var metrics = prometheusClient.getMetrics(); var expectedMetrics = metrics.findByNameAndLabels(metricName, @@ -183,18 +190,33 @@ private static Map getCollectorProps(String serviceName, int pro return props; } - private static String getFunctionWorkerCommand(String serviceUrl, String suffix) { + private static String getFunctionWorkerCommand(PulsarCluster pulsarCluster, String suffix, boolean useLocalRunner) + throws Exception { var namespace = NamespaceName.get("public", "default"); var sourceTopicName = TopicName.get(TopicDomain.persistent.toString(), namespace, "metricTestSource-" + suffix); var sinkTopicName = TopicName.get(TopicDomain.persistent.toString(), namespace, "metricTestSink-" + suffix); + try (PulsarAdmin admin = PulsarAdmin.builder().serviceHttpUrl(pulsarCluster.getHttpServiceUrl()).build()) { + admin.topics().createNonPartitionedTopic(sourceTopicName.toString()); + admin.topics().createNonPartitionedTopic(sinkTopicName.toString()); + } + + var serviceUrl = useLocalRunner ? + pulsarCluster.getPlainTextServiceUrl() : + new URL("http", ProxyContainer.NAME, ProxyContainer.BROKER_HTTP_PORT, "").toString(); + var commandGenerator = new CommandGenerator(); - commandGenerator.setAdminUrl(serviceUrl); + if (useLocalRunner) { + commandGenerator.setAdminUrl(serviceUrl); + } commandGenerator.setSourceTopic(sourceTopicName.toString()); commandGenerator.setSinkTopic(sinkTopicName.toString()); - commandGenerator.setFunctionName("metricsTestLocalRunTest-" + suffix); commandGenerator.setRuntime(CommandGenerator.Runtime.JAVA); + commandGenerator.setFunctionName("metricsTestLocalRunTest-" + suffix); commandGenerator.setFunctionClassName(PulsarFunctionsTest.EXCLAMATION_JAVA_CLASS); - return commandGenerator.generateLocalRunCommand(null); + if (useLocalRunner) { + return commandGenerator.generateLocalRunCommand(null); + } + return commandGenerator.generateCreateFunctionCommand(); } } From efe5fcce7fb0697f551ae86503980f1d97b95ffb Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 29 Jan 2024 15:49:03 -0800 Subject: [PATCH 055/133] Cosmetic test fixes --- .../integration/metrics/MetricsTest.java | 32 ++++--------------- 1 file changed, 6 insertions(+), 26 deletions(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java index 1a13de59c38fd..fcb9f3b0e5552 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar.tests.integration.metrics; -import java.net.URL; import java.util.Arrays; import java.util.HashMap; import java.util.List; @@ -34,7 +33,6 @@ import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.tests.integration.containers.OpenTelemetryCollectorContainer; -import org.apache.pulsar.tests.integration.containers.ProxyContainer; import org.apache.pulsar.tests.integration.containers.PulsarContainer; import org.apache.pulsar.tests.integration.functions.PulsarFunctionsTest; import org.apache.pulsar.tests.integration.functions.utils.CommandGenerator; @@ -77,13 +75,9 @@ public void testOpenTelemetryMetricsOtlpExport() throws Exception { var pulsarCluster = PulsarCluster.forSpec(spec); pulsarCluster.start(); - var useLocalRunner = false; - pulsarCluster.setupFunctionWorkers(functionWorkerServiceNameSuffix, FunctionRuntimeType.PROCESS, 1); - var serviceUrl = pulsarCluster.getPlainTextServiceUrl(); - var functionWorkerCommand = - getFunctionWorkerCommand(pulsarCluster, functionWorkerServiceNameSuffix, useLocalRunner); - pulsarCluster.getAnyWorker().execCmdAsync(functionWorkerCommand.split(" ")); + var functionWorkerCommand = getFunctionWorkerCommand(pulsarCluster, functionWorkerServiceNameSuffix); + pulsarCluster.getAnyWorker().execCmdAsync("sh", "-c", functionWorkerCommand); var metricName = "queueSize_ratio"; // Sent automatically by the OpenTelemetry SDK. Awaitility.waitAtMost(90, TimeUnit.SECONDS).pollInterval(1, TimeUnit.SECONDS).until(() -> { @@ -98,7 +92,7 @@ public void testOpenTelemetryMetricsOtlpExport() throws Exception { }); } - @Test(timeOut = 900_000) + @Test(timeOut = 300_000) public void testOpenTelemetryMetricsPrometheusExport() throws Exception { var prometheusExporterPort = 9464; var clusterName = "testOpenTelemetryMetrics-" + UUID.randomUUID(); @@ -111,7 +105,6 @@ public void testOpenTelemetryMetricsPrometheusExport() throws Exception { var functionWorkerServiceNameSuffix = PulsarTestBase.randomName(); var functionWorkerOtelServiceName = "function-worker-" + functionWorkerServiceNameSuffix; - // Locally run Pulsar Functions override cluster name definitions to "local". var functionWorkerCollectorProps = getCollectorProps(functionWorkerOtelServiceName, prometheusExporterPort); var spec = PulsarClusterSpec.builder() @@ -127,11 +120,8 @@ public void testOpenTelemetryMetricsPrometheusExport() throws Exception { var pulsarCluster = PulsarCluster.forSpec(spec); pulsarCluster.start(); - var useLocalRunner = false; - pulsarCluster.setupFunctionWorkers(functionWorkerServiceNameSuffix, FunctionRuntimeType.PROCESS, 1); - var functionWorkerCommand = - getFunctionWorkerCommand(pulsarCluster, functionWorkerServiceNameSuffix, useLocalRunner); + var functionWorkerCommand = getFunctionWorkerCommand(pulsarCluster, functionWorkerServiceNameSuffix); var workerContainer = pulsarCluster.getAnyWorker(); workerContainer.execCmdAsync("sh", "-c", functionWorkerCommand); @@ -152,7 +142,7 @@ public void testOpenTelemetryMetricsPrometheusExport() throws Exception { return !expectedMetrics.isEmpty(); }); - Awaitility.waitAtMost(900, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { + Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { var prometheusClient = createMetricsClient(workerContainer, prometheusExporterPort); var metrics = prometheusClient.getMetrics(); var expectedMetrics = metrics.findByNameAndLabels(metricName, @@ -190,7 +180,7 @@ private static Map getCollectorProps(String serviceName, int pro return props; } - private static String getFunctionWorkerCommand(PulsarCluster pulsarCluster, String suffix, boolean useLocalRunner) + private static String getFunctionWorkerCommand(PulsarCluster pulsarCluster, String suffix) throws Exception { var namespace = NamespaceName.get("public", "default"); var sourceTopicName = TopicName.get(TopicDomain.persistent.toString(), namespace, "metricTestSource-" + suffix); @@ -201,22 +191,12 @@ private static String getFunctionWorkerCommand(PulsarCluster pulsarCluster, Stri admin.topics().createNonPartitionedTopic(sinkTopicName.toString()); } - var serviceUrl = useLocalRunner ? - pulsarCluster.getPlainTextServiceUrl() : - new URL("http", ProxyContainer.NAME, ProxyContainer.BROKER_HTTP_PORT, "").toString(); - var commandGenerator = new CommandGenerator(); - if (useLocalRunner) { - commandGenerator.setAdminUrl(serviceUrl); - } commandGenerator.setSourceTopic(sourceTopicName.toString()); commandGenerator.setSinkTopic(sinkTopicName.toString()); commandGenerator.setRuntime(CommandGenerator.Runtime.JAVA); commandGenerator.setFunctionName("metricsTestLocalRunTest-" + suffix); commandGenerator.setFunctionClassName(PulsarFunctionsTest.EXCLAMATION_JAVA_CLASS); - if (useLocalRunner) { - return commandGenerator.generateLocalRunCommand(null); - } return commandGenerator.generateCreateFunctionCommand(); } } From 2dfd6ec38e4802f7f510877210451c9668b4850b Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 29 Jan 2024 16:39:56 -0800 Subject: [PATCH 056/133] Cosmetic test fixes --- .../OpenTelemetryCollectorContainer.java | 7 +- .../integration/metrics/MetricsTest.java | 129 +++++++++--------- 2 files changed, 62 insertions(+), 74 deletions(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/OpenTelemetryCollectorContainer.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/OpenTelemetryCollectorContainer.java index 44f9f5d17299e..3df12d031b24f 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/OpenTelemetryCollectorContainer.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/OpenTelemetryCollectorContainer.java @@ -20,7 +20,6 @@ import java.time.Duration; import org.apache.http.HttpStatus; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.testcontainers.containers.wait.strategy.HttpWaitStrategy; import org.testcontainers.utility.MountableFile; @@ -29,7 +28,7 @@ public class OpenTelemetryCollectorContainer extends ChaosContainer { - var metrics = openTelemetryCollectorContainer.getMetricsClient().getMetrics(); - // TODO: Validate cluster name is present once - // https://github.com/open-telemetry/opentelemetry-java/issues/6108 is solved. - var brokerMetrics = metrics.findByNameAndLabels(metricName, Pair.of("job", brokerOtelServiceName)); - var proxyMetrics = metrics.findByNameAndLabels(metricName, Pair.of("job", proxyOtelServiceName)); - var functionWorkerMetrics = - metrics.findByNameAndLabels(metricName, Pair.of("job", functionWorkerOtelServiceName)); - return !brokerMetrics.isEmpty() && !proxyMetrics.isEmpty() && !functionWorkerMetrics.isEmpty(); - }); + Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> + hasMetrics(openTelemetryCollectorContainer, OpenTelemetryCollectorContainer.PROMETHEUS_EXPORTER_PORT, + metricName, Pair.of("job", brokerOtelServiceName))); + Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> + hasMetrics(openTelemetryCollectorContainer, OpenTelemetryCollectorContainer.PROMETHEUS_EXPORTER_PORT, + metricName, Pair.of("job", proxyOtelServiceName))); + Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> + hasMetrics(openTelemetryCollectorContainer, OpenTelemetryCollectorContainer.PROMETHEUS_EXPORTER_PORT, + metricName, Pair.of("job", functionWorkerOtelServiceName))); } @Test(timeOut = 300_000) @@ -97,15 +100,20 @@ public void testOpenTelemetryMetricsPrometheusExport() throws Exception { var prometheusExporterPort = 9464; var clusterName = "testOpenTelemetryMetrics-" + UUID.randomUUID(); + var exporter = "prometheus"; + var prometheusExporterPortProp = + Pair.of("OTEL_EXPORTER_PROMETHEUS_PORT", Integer.toString(prometheusExporterPort)); + var brokerOtelServiceName = clusterName + "-broker"; - var brokerCollectorProps = getCollectorProps(brokerOtelServiceName, prometheusExporterPort); + var brokerCollectorProps = getCollectorProps(brokerOtelServiceName, exporter, prometheusExporterPortProp); var proxyOtelServiceName = clusterName + "-proxy"; - var proxyCollectorProps = getCollectorProps(proxyOtelServiceName, prometheusExporterPort); + var proxyCollectorProps = getCollectorProps(proxyOtelServiceName, exporter, prometheusExporterPortProp); var functionWorkerServiceNameSuffix = PulsarTestBase.randomName(); var functionWorkerOtelServiceName = "function-worker-" + functionWorkerServiceNameSuffix; - var functionWorkerCollectorProps = getCollectorProps(functionWorkerOtelServiceName, prometheusExporterPort); + var functionWorkerCollectorProps = + getCollectorProps(functionWorkerOtelServiceName, exporter, prometheusExporterPortProp); var spec = PulsarClusterSpec.builder() .clusterName(clusterName) @@ -120,68 +128,49 @@ public void testOpenTelemetryMetricsPrometheusExport() throws Exception { var pulsarCluster = PulsarCluster.forSpec(spec); pulsarCluster.start(); - pulsarCluster.setupFunctionWorkers(functionWorkerServiceNameSuffix, FunctionRuntimeType.PROCESS, 1); - var functionWorkerCommand = getFunctionWorkerCommand(pulsarCluster, functionWorkerServiceNameSuffix); - var workerContainer = pulsarCluster.getAnyWorker(); - workerContainer.execCmdAsync("sh", "-c", functionWorkerCommand); + var workerContainer = setupFunctionWorker(pulsarCluster, functionWorkerServiceNameSuffix); var metricName = "target_info"; // Sent automatically by the OpenTelemetry SDK. - Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { - var prometheusClient = createMetricsClient(pulsarCluster.getAnyBroker(), prometheusExporterPort); - var metrics = prometheusClient.getMetrics(); - var expectedMetrics = metrics.findByNameAndLabels(metricName, - Pair.of("pulsar_cluster", clusterName), Pair.of("service_name", brokerOtelServiceName)); - return !expectedMetrics.isEmpty(); - }); - - Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { - var prometheusClient = createMetricsClient(pulsarCluster.getProxy(), prometheusExporterPort); - var metrics = prometheusClient.getMetrics(); - var expectedMetrics = metrics.findByNameAndLabels(metricName, - Pair.of("pulsar_cluster", clusterName), Pair.of("service_name", proxyOtelServiceName)); - return !expectedMetrics.isEmpty(); - }); - - Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { - var prometheusClient = createMetricsClient(workerContainer, prometheusExporterPort); - var metrics = prometheusClient.getMetrics(); - var expectedMetrics = metrics.findByNameAndLabels(metricName, - Pair.of("pulsar_cluster", clusterName), Pair.of("service_name", functionWorkerOtelServiceName)); - return !expectedMetrics.isEmpty(); - }); + Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> + hasMetrics(pulsarCluster.getAnyBroker(), prometheusExporterPort, metricName, + Pair.of("pulsar_cluster", clusterName), + Pair.of("service_name", brokerOtelServiceName))); + Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> + hasMetrics(pulsarCluster.getProxy(), prometheusExporterPort, metricName, + Pair.of("pulsar_cluster", clusterName), + Pair.of("service_name", proxyOtelServiceName))); + Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> + hasMetrics(workerContainer, prometheusExporterPort, metricName, + Pair.of("pulsar_cluster", clusterName), + Pair.of("service_name", functionWorkerOtelServiceName))); } - private static PrometheusMetricsClient createMetricsClient(PulsarContainer container, int port) { - return new PrometheusMetricsClient(container.getHost(), container.getMappedPort(port)); + @SafeVarargs + private static boolean hasMetrics(ChaosContainer container, int port, String metricName, + Pair ... expectedLabels) throws Exception { + var client = new PrometheusMetricsClient(container.getHost(), container.getMappedPort(port)); + var allMetrics = client.getMetrics(); + var actualMetrics = allMetrics.findByNameAndLabels(metricName, expectedLabels); + return !actualMetrics.isEmpty(); } - private static Map getCollectorProps( - String serviceName, OpenTelemetryCollectorContainer openTelemetryCollectorContainer) { - return Map.of( - "OTEL_SDK_DISABLED", "false", - "OTEL_SERVICE_NAME", serviceName, - "OTEL_METRICS_EXPORTER", "otlp", - "OTEL_EXPORTER_OTLP_ENDPOINT", openTelemetryCollectorContainer.getOtlpEndpoint(), - "OTEL_METRIC_EXPORT_INTERVAL", "1000" - ); - } - - private static Map getCollectorProps(String serviceName, int prometheusExporterPort, + @SafeVarargs + private static Map getCollectorProps(String serviceName, String exporter, Pair ... extraProps) { var defaultProps = Map.of( "OTEL_SDK_DISABLED", "false", + "OTEL_METRIC_EXPORT_INTERVAL", "1000", "OTEL_SERVICE_NAME", serviceName, - "OTEL_METRICS_EXPORTER", "prometheus", - "OTEL_EXPORTER_PROMETHEUS_PORT", Integer.toString(prometheusExporterPort), - "OTEL_METRIC_EXPORT_INTERVAL", "1000" + "OTEL_METRICS_EXPORTER", exporter ); var props = new HashMap<>(defaultProps); Arrays.stream(extraProps).forEach(p -> props.put(p.getKey(), p.getValue())); return props; } - private static String getFunctionWorkerCommand(PulsarCluster pulsarCluster, String suffix) - throws Exception { + private static WorkerContainer setupFunctionWorker(PulsarCluster pulsarCluster, String suffix) throws Exception { + pulsarCluster.setupFunctionWorkers(suffix, FunctionRuntimeType.PROCESS, 1); + var namespace = NamespaceName.get("public", "default"); var sourceTopicName = TopicName.get(TopicDomain.persistent.toString(), namespace, "metricTestSource-" + suffix); var sinkTopicName = TopicName.get(TopicDomain.persistent.toString(), namespace, "metricTestSink-" + suffix); @@ -197,6 +186,10 @@ private static String getFunctionWorkerCommand(PulsarCluster pulsarCluster, Stri commandGenerator.setRuntime(CommandGenerator.Runtime.JAVA); commandGenerator.setFunctionName("metricsTestLocalRunTest-" + suffix); commandGenerator.setFunctionClassName(PulsarFunctionsTest.EXCLAMATION_JAVA_CLASS); - return commandGenerator.generateCreateFunctionCommand(); + var functionWorkerCommand = commandGenerator.generateCreateFunctionCommand(); + + var workerContainer = pulsarCluster.getAnyWorker(); + workerContainer.execCmdAsync("sh", "-c", functionWorkerCommand); + return workerContainer; } } From 063b6414b63a72b6169b824e462152e542a6586c Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 29 Jan 2024 22:27:37 -0800 Subject: [PATCH 057/133] Cosmetic test fixes --- .../pulsar/tests/integration/metrics/MetricsTest.java | 6 +++--- .../tests/integration/topologies/PulsarClusterSpec.java | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java index 3e9589d88e669..ddf8afe8015c6 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java @@ -49,7 +49,7 @@ public class MetricsTest { // Test with the included Prometheus exporter as well // https://github.com/open-telemetry/opentelemetry-java/blob/main/sdk-extensions/autoconfigure/README.md#prometheus-exporter - @Test(timeOut = 300_000) + @Test(timeOut = 360_000) public void testOpenTelemetryMetricsOtlpExport() throws Exception { var clusterName = "testOpenTelemetryMetrics-" + UUID.randomUUID(); var openTelemetryCollectorContainer = new OpenTelemetryCollectorContainer(clusterName); @@ -95,7 +95,7 @@ public void testOpenTelemetryMetricsOtlpExport() throws Exception { metricName, Pair.of("job", functionWorkerOtelServiceName))); } - @Test(timeOut = 300_000) + @Test(timeOut = 360_000) public void testOpenTelemetryMetricsPrometheusExport() throws Exception { var prometheusExporterPort = 9464; var clusterName = "testOpenTelemetryMetrics-" + UUID.randomUUID(); @@ -122,7 +122,7 @@ public void testOpenTelemetryMetricsPrometheusExport() throws Exception { .proxyEnvs(proxyCollectorProps) .proxyAdditionalPorts(List.of(prometheusExporterPort)) .functionWorkerEnv(functionWorkerServiceNameSuffix, functionWorkerCollectorProps) - .functionWorkerAdditionalPort(functionWorkerServiceNameSuffix, List.of(prometheusExporterPort)) + .functionWorkerAdditionalPorts(functionWorkerServiceNameSuffix, List.of(prometheusExporterPort)) .build(); @Cleanup("stop") var pulsarCluster = PulsarCluster.forSpec(spec); diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java index 7222bb4991159..657ba08e46a0d 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java @@ -90,7 +90,7 @@ public class PulsarClusterSpec { /** * Additional ports to expose on function workers. */ - @Singular + @Singular("functionWorkerAdditionalPorts") Map> functionWorkerAdditionalPorts; From 5bf7fdb279f188957d09594a7f66482a68a9e0eb Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 29 Jan 2024 22:28:48 -0800 Subject: [PATCH 058/133] Abstract extra reader test setup details in OpenTelemetryService --- .../common/stats/OpenTelemetryService.java | 38 ++++++++----------- .../stats/OpenTelemetryServiceTest.java | 20 +++++++++- 2 files changed, 34 insertions(+), 24 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index 945a19b3bd0d9..6f2ddc7dfa1da 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -25,14 +25,11 @@ import io.opentelemetry.sdk.OpenTelemetrySdk; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdk; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; -import io.opentelemetry.sdk.metrics.export.MetricReader; -import io.opentelemetry.sdk.metrics.internal.SdkMeterProviderUtil; -import io.opentelemetry.sdk.metrics.internal.export.CardinalityLimitSelector; import io.opentelemetry.sdk.resources.Resource; import java.io.Closeable; import java.util.HashMap; -import java.util.List; import java.util.Map; +import lombok.Builder; import lombok.Singular; import org.apache.commons.lang3.StringUtils; @@ -47,36 +44,33 @@ public class OpenTelemetryService implements Closeable { private final OpenTelemetrySdk openTelemetrySdk; - @lombok.Builder - public OpenTelemetryService( - String clusterName, - @Singular Map extraProperties, - @VisibleForTesting @Singular List extraMetricReaders) { + @Builder + public OpenTelemetryService(String clusterName, + @Singular Map extraProperties, + // Allows customizing the SDK builder; for testing purposes only. + @VisibleForTesting AutoConfiguredOpenTelemetrySdkBuilder sdkBuilder) { checkArgument(StringUtils.isNotEmpty(clusterName), "Cluster name cannot be empty"); - AutoConfiguredOpenTelemetrySdkBuilder builder = AutoConfiguredOpenTelemetrySdk.builder(); + if (sdkBuilder == null) { + sdkBuilder = AutoConfiguredOpenTelemetrySdk.builder(); + } Map overrideProperties = new HashMap<>(); overrideProperties.put(OTEL_SDK_DISABLED, "true"); + // Cardinality limit property is exclusive, so we need to add 1. overrideProperties.put(MAX_CARDINALITY_LIMIT_KEY, Integer.toString(MAX_CARDINALITY_LIMIT + 1)); - builder.addPropertiesSupplier(() -> overrideProperties); - builder.addPropertiesSupplier(() -> extraProperties); + sdkBuilder.addPropertiesSupplier(() -> overrideProperties); + sdkBuilder.addPropertiesSupplier(() -> extraProperties); - builder.addResourceCustomizer( + sdkBuilder.addResourceCustomizer( (resource, __) -> { if (resource.getAttribute(CLUSTER_NAME_ATTRIBUTE) != null) { - return resource; // Do not override the attribute if already set. + // Do not override if already set (via system properties or environment variables). + return resource; } return resource.merge(Resource.builder().put(CLUSTER_NAME_ATTRIBUTE, clusterName).build()); }); - final CardinalityLimitSelector cardinalityLimitSelector = __ -> MAX_CARDINALITY_LIMIT + 1; - extraMetricReaders.forEach(metricReader -> builder.addMeterProviderCustomizer((sdkMeterProviderBuilder, __) -> { - SdkMeterProviderUtil.registerMetricReaderWithCardinalitySelector( - sdkMeterProviderBuilder, metricReader, cardinalityLimitSelector); - return sdkMeterProviderBuilder; - })); - - openTelemetrySdk = builder.build().getOpenTelemetrySdk(); + openTelemetrySdk = sdkBuilder.build().getOpenTelemetrySdk(); } public Meter getMeter(String instrumentationScopeName) { diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java index 671e09b7b2661..84e7d42f198fc 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java @@ -23,9 +23,13 @@ import io.opentelemetry.api.metrics.DoubleCounter; import io.opentelemetry.api.metrics.LongCounter; import io.opentelemetry.api.metrics.Meter; +import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdk; +import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; import io.opentelemetry.sdk.common.InstrumentationScopeInfo; import io.opentelemetry.sdk.metrics.data.MetricData; import io.opentelemetry.sdk.metrics.data.MetricDataType; +import io.opentelemetry.sdk.metrics.export.MetricReader; +import io.opentelemetry.sdk.metrics.internal.SdkMeterProviderUtil; import io.opentelemetry.sdk.metrics.internal.state.MetricStorage; import io.opentelemetry.sdk.testing.exporter.InMemoryMetricReader; import java.util.Collection; @@ -47,8 +51,8 @@ public class OpenTelemetryServiceTest { public void setup() throws Exception { reader = InMemoryMetricReader.create(); openTelemetryService = OpenTelemetryService.builder(). + sdkBuilder(getSdkBuilder(reader)). clusterName("openTelemetryServiceTestCluster"). - extraMetricReader(reader). extraProperty(OpenTelemetryService.OTEL_SDK_DISABLED, "false"). build(); meter = openTelemetryService.getMeter("openTelemetryServiceTestInstrument"); @@ -60,6 +64,18 @@ public void teardown() throws Exception { reader.close(); } + // Overrides the default sdkBuilder to include the InMemoryMetricReader for testing purposes. + private static AutoConfiguredOpenTelemetrySdkBuilder getSdkBuilder(MetricReader extraReader) { + return AutoConfiguredOpenTelemetrySdk.builder(). + addMeterProviderCustomizer((sdkMeterProviderBuilder, configProperties) -> { + SdkMeterProviderUtil.registerMetricReaderWithCardinalitySelector( + sdkMeterProviderBuilder, extraReader, + // Override the max cardinality limit for this extra reader. + instrumentType -> OpenTelemetryService.MAX_CARDINALITY_LIMIT + 1); + return sdkMeterProviderBuilder; + }); + } + @Test(expectedExceptions = IllegalArgumentException.class) public void testIsClusterNameCannotBeNull() throws Exception { @Cleanup @@ -79,8 +95,8 @@ public void testIsClusterNameSet() throws Exception { @Cleanup OpenTelemetryService ots = OpenTelemetryService.builder(). + sdkBuilder(getSdkBuilder(reader)). clusterName("testCluster"). - extraMetricReader(reader). extraProperty(OpenTelemetryService.OTEL_SDK_DISABLED, "false"). build(); From 3c7be400024a1cd494f3332ca141ee669104977f Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 29 Jan 2024 23:37:04 -0800 Subject: [PATCH 059/133] Update LICENSE.bin.txt --- .../server/src/assemble/LICENSE.bin.txt | 32 ++++++++++++++++++- 1 file changed, 31 insertions(+), 1 deletion(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index f509287f8f030..ede39275d7c15 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -333,6 +333,13 @@ The Apache Software License, Version 2.0 - io.prometheus-simpleclient_tracer_common-0.16.0.jar - io.prometheus-simpleclient_tracer_otel-0.16.0.jar - io.prometheus-simpleclient_tracer_otel_agent-0.16.0.jar + * Prometheus exporter + - io.prometheus-prometheus-metrics-config-1.1.0.jar + - io.prometheus-prometheus-metrics-exporter-common-1.1.0.jar + - io.prometheus-prometheus-metrics-exporter-httpserver-1.1.0.jar + - io.prometheus-prometheus-metrics-exposition-formats-1.1.0.jar + - io.prometheus-prometheus-metrics-model-1.1.0.jar + - io.prometheus-prometheus-metrics-shaded-protobuf-1.1.0.jar * Jakarta Bean Validation API - jakarta.validation-jakarta.validation-api-2.0.2.jar - javax.validation-validation-api-1.1.0.Final.jar @@ -500,6 +507,28 @@ The Apache Software License, Version 2.0 - io.reactivex.rxjava3-rxjava-3.0.1.jar * RoaringBitmap - org.roaringbitmap-RoaringBitmap-0.9.44.jar + - org.roaringbitmap-shims-0.9.44.jar + * OpenTelemetry + - io.opentelemetry-opentelemetry-api-1.34.1.jar + - io.opentelemetry-opentelemetry-api-events-1.34.1-alpha.jar + - io.opentelemetry-opentelemetry-context-1.34.1.jar + - io.opentelemetry-opentelemetry-exporter-common-1.34.1.jar + - io.opentelemetry-opentelemetry-exporter-otlp-1.34.1.jar + - io.opentelemetry-opentelemetry-exporter-otlp-common-1.34.1.jar + - io.opentelemetry-opentelemetry-exporter-prometheus-1.34.1-alpha.jar + - io.opentelemetry-opentelemetry-exporter-sender-okhttp-1.34.1.jar + - io.opentelemetry-opentelemetry-extension-incubator-1.34.1-alpha.jar + - io.opentelemetry-opentelemetry-sdk-1.34.1.jar + - io.opentelemetry-opentelemetry-sdk-common-1.34.1.jar + - io.opentelemetry-opentelemetry-sdk-extension-autoconfigure-1.34.1.jar + - io.opentelemetry-opentelemetry-sdk-extension-autoconfigure-spi-1.34.1.jar + - io.opentelemetry-opentelemetry-sdk-logs-1.34.1.jar + - io.opentelemetry-opentelemetry-sdk-metrics-1.34.1.jar + - io.opentelemetry-opentelemetry-sdk-trace-1.34.1.jar + * Android Annotations + - com.google.android-annotations-4.1.1.4.jar + * Java Native Access + - net.java.dev.jna-jna-5.12.1.jar BSD 3-clause "New" or "Revised" License * Google auth library @@ -525,6 +554,8 @@ MIT License * Auth0, Inc. - com.auth0-java-jwt-4.3.0.jar - com.auth0-jwks-rsa-0.22.0.jar + * Mojo + - org.codehaus.mojo-animal-sniffer-annotations-1.21.jar Protocol Buffers License * Protocol Buffers - com.google.protobuf-protobuf-java-3.19.6.jar -- ../licenses/LICENSE-protobuf.txt @@ -532,7 +563,6 @@ Protocol Buffers License CDDL-1.1 -- ../licenses/LICENSE-CDDL-1.1.txt * Java Annotations API - - javax.annotation-javax.annotation-api-1.3.2.jar - com.sun.activation-javax.activation-1.2.0.jar - javax.xml.bind-jaxb-api-2.3.1.jar * Java Servlet API -- javax.servlet-javax.servlet-api-3.1.0.jar From c0aafa71b9c137023cb891897ff34aabe1927a41 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 30 Jan 2024 09:37:06 -0800 Subject: [PATCH 060/133] Fix ProxyConfiguration missing cluster name field in tests --- .../broker/authentication/ProxySaslAuthenticationTest.java | 1 + .../proxy/extensions/SimpleProxyExtensionTestBase.java | 1 + .../proxy/server/AdminProxyHandlerKeystoreTLSTest.java | 1 + .../pulsar/proxy/server/AuthedAdminProxyHandlerTest.java | 1 + .../pulsar/proxy/server/ProxyAdditionalServletTest.java | 1 + .../proxy/server/ProxyAuthenticatedProducerConsumerTest.java | 1 + .../apache/pulsar/proxy/server/ProxyAuthenticationTest.java | 1 + .../pulsar/proxy/server/ProxyConnectionThrottlingTest.java | 1 + .../pulsar/proxy/server/ProxyEnableHAProxyProtocolTest.java | 1 + .../apache/pulsar/proxy/server/ProxyForwardAuthDataTest.java | 1 + .../pulsar/proxy/server/ProxyKeyStoreTlsTransportTest.java | 1 + .../pulsar/proxy/server/ProxyKeyStoreTlsWithAuthTest.java | 1 + .../pulsar/proxy/server/ProxyKeyStoreTlsWithoutAuthTest.java | 1 + .../pulsar/proxy/server/ProxyLookupThrottlingTest.java | 1 + .../org/apache/pulsar/proxy/server/ProxyMutualTlsTest.java | 1 + .../java/org/apache/pulsar/proxy/server/ProxyParserTest.java | 1 + .../org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java | 1 + .../pulsar/proxy/server/ProxyRolesEnforcementTest.java | 1 + .../apache/pulsar/proxy/server/ProxyServiceStarterTest.java | 1 + .../pulsar/proxy/server/ProxyServiceTlsStarterTest.java | 1 + .../java/org/apache/pulsar/proxy/server/ProxyStatsTest.java | 1 + .../apache/pulsar/proxy/server/ProxyStuckConnectionTest.java | 1 + .../test/java/org/apache/pulsar/proxy/server/ProxyTest.java | 1 + .../java/org/apache/pulsar/proxy/server/ProxyTlsTest.java | 1 + .../org/apache/pulsar/proxy/server/ProxyTlsWithAuthTest.java | 1 + .../pulsar/proxy/server/ProxyWithAuthorizationNegTest.java | 4 +++- .../pulsar/proxy/server/ProxyWithAuthorizationTest.java | 4 +++- .../proxy/server/ProxyWithExtensibleLoadManagerTest.java | 1 + .../pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java | 1 + .../proxy/server/ProxyWithoutServiceDiscoveryTest.java | 5 ++++- .../proxy/server/SuperUserAuthedAdminProxyHandlerTest.java | 1 + .../pulsar/proxy/server/UnauthedAdminProxyHandlerTest.java | 1 + 32 files changed, 39 insertions(+), 3 deletions(-) diff --git a/pulsar-broker-auth-sasl/src/test/java/org/apache/pulsar/broker/authentication/ProxySaslAuthenticationTest.java b/pulsar-broker-auth-sasl/src/test/java/org/apache/pulsar/broker/authentication/ProxySaslAuthenticationTest.java index f6ad76a083bc3..a27384c989000 100644 --- a/pulsar-broker-auth-sasl/src/test/java/org/apache/pulsar/broker/authentication/ProxySaslAuthenticationTest.java +++ b/pulsar-broker-auth-sasl/src/test/java/org/apache/pulsar/broker/authentication/ProxySaslAuthenticationTest.java @@ -242,6 +242,7 @@ void testAuthentication() throws Exception { proxyConfig.setBrokerServiceURL(pulsar.getBrokerServiceUrl()); proxyConfig.setSaslJaasClientAllowedIds(".*" + localHostname + ".*"); proxyConfig.setSaslJaasServerSectionName("PulsarProxy"); + proxyConfig.setClusterName(configClusterName); // proxy connect to broker proxyConfig.setBrokerClientAuthenticationPlugin(AuthenticationSasl.class.getName()); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/extensions/SimpleProxyExtensionTestBase.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/extensions/SimpleProxyExtensionTestBase.java index fde7c938d0a62..f9ace716ecd06 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/extensions/SimpleProxyExtensionTestBase.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/extensions/SimpleProxyExtensionTestBase.java @@ -140,6 +140,7 @@ protected void setup() throws Exception { proxyConfig.setBrokerProxyAllowedTargetPorts("*"); proxyConfig.setMetadataStoreUrl(DUMMY_VALUE); proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); + proxyConfig.setClusterName(configClusterName); proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)))); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerKeystoreTLSTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerKeystoreTLSTest.java index bc2029861f415..00454e5e14027 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerKeystoreTLSTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerKeystoreTLSTest.java @@ -101,6 +101,7 @@ protected void setup() throws Exception { proxyConfig.setBrokerClientAuthenticationPlugin(AuthenticationKeyStoreTls.class.getName()); proxyConfig.setBrokerClientAuthenticationParameters(String.format("keyStoreType:%s,keyStorePath:%s,keyStorePassword:%s", KEYSTORE_TYPE, BROKER_KEYSTORE_FILE_PATH, BROKER_KEYSTORE_PW)); + proxyConfig.setClusterName(DUMMY_VALUE); resource = new PulsarResources(registerCloseable(new ZKMetadataStore(mockZooKeeper)), registerCloseable(new ZKMetadataStore(mockZooKeeperGlobal))); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AuthedAdminProxyHandlerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AuthedAdminProxyHandlerTest.java index d83de9652cfde..ef58648e35a25 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AuthedAdminProxyHandlerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AuthedAdminProxyHandlerTest.java @@ -85,6 +85,7 @@ protected void setup() throws Exception { proxyConfig.setWebServicePortTls(Optional.of(0)); proxyConfig.setTlsEnabledWithBroker(true); proxyConfig.setHttpMaxRequestHeaderSize(20000); + proxyConfig.setClusterName(configClusterName); // enable tls and auth&auth at proxy proxyConfig.setTlsCertificateFilePath(PROXY_CERT_FILE_PATH); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAdditionalServletTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAdditionalServletTest.java index 9f8efa1ec7935..f61a73bbf9177 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAdditionalServletTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAdditionalServletTest.java @@ -78,6 +78,7 @@ protected void setup() throws Exception { proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); // enable full parsing feature proxyConfig.setProxyLogLevel(Optional.of(2)); + proxyConfig.setClusterName(configClusterName); // this is for nar package test // addServletNar(); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticatedProducerConsumerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticatedProducerConsumerTest.java index 1c93cb20c70df..4083c984d9874 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticatedProducerConsumerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticatedProducerConsumerTest.java @@ -137,6 +137,7 @@ protected void setup() throws Exception { proxyConfig.setMetadataStoreUrl(DUMMY_VALUE); proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); + proxyConfig.setClusterName(configClusterName); proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)))); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticationTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticationTest.java index fec0673ff9b56..046e6cfb5cd83 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticationTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticationTest.java @@ -222,6 +222,7 @@ void testAuthentication() throws Exception { proxyConfig.setBrokerProxyAllowedTargetPorts("*"); proxyConfig.setWebServicePort(Optional.of(0)); proxyConfig.setBrokerServiceURL(pulsar.getBrokerServiceUrl()); + proxyConfig.setClusterName(configClusterName); proxyConfig.setBrokerClientAuthenticationPlugin(BasicAuthentication.class.getName()); proxyConfig.setBrokerClientAuthenticationParameters(proxyAuthParams); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyConnectionThrottlingTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyConnectionThrottlingTest.java index a070d1e84d339..78ab9bd0d9581 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyConnectionThrottlingTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyConnectionThrottlingTest.java @@ -59,6 +59,7 @@ protected void setup() throws Exception { proxyConfig.setMaxConcurrentLookupRequests(NUM_CONCURRENT_LOOKUP); proxyConfig.setMaxConcurrentInboundConnections(NUM_CONCURRENT_INBOUND_CONNECTION); proxyConfig.setMaxConcurrentInboundConnectionsPerIp(NUM_CONCURRENT_INBOUND_CONNECTION); + proxyConfig.setClusterName(configClusterName); proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)))); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeper))).when(proxyService).createLocalMetadataStore(); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyEnableHAProxyProtocolTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyEnableHAProxyProtocolTest.java index 5704ba55fed86..413774daf2cd1 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyEnableHAProxyProtocolTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyEnableHAProxyProtocolTest.java @@ -60,6 +60,7 @@ protected void setup() throws Exception { proxyConfig.setMetadataStoreUrl(DUMMY_VALUE); proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); proxyConfig.setHaProxyProtocolEnabled(true); + proxyConfig.setClusterName(configClusterName); proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)))); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyForwardAuthDataTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyForwardAuthDataTest.java index 477fe597f2661..6b3188746b618 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyForwardAuthDataTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyForwardAuthDataTest.java @@ -109,6 +109,7 @@ public void testForwardAuthData() throws Exception { proxyConfig.setBrokerServiceURL(pulsar.getBrokerServiceUrl()); proxyConfig.setBrokerClientAuthenticationPlugin(BasicAuthentication.class.getName()); proxyConfig.setBrokerClientAuthenticationParameters(proxyAuthParams); + proxyConfig.setClusterName(configClusterName); Set providers = new HashSet<>(); providers.add(BasicAuthenticationProvider.class.getName()); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTransportTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTransportTest.java index 5ee03395b80c8..5671c527f68f9 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTransportTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTransportTest.java @@ -77,6 +77,7 @@ protected void setup() throws Exception { proxyConfig.setMetadataStoreUrl(DUMMY_VALUE); proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); + proxyConfig.setClusterName(configClusterName); proxyConfig.setTlsRequireTrustedClientCertOnConnect(false); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsWithAuthTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsWithAuthTest.java index 1f21281a6f6ab..99fb8c03a819f 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsWithAuthTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsWithAuthTest.java @@ -77,6 +77,7 @@ protected void setup() throws Exception { proxyConfig.setMetadataStoreUrl(DUMMY_VALUE); proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); + proxyConfig.setClusterName(configClusterName); // config for authentication and authorization. proxyConfig.setTlsRequireTrustedClientCertOnConnect(true); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsWithoutAuthTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsWithoutAuthTest.java index d7935755ce040..1dcebda7935d7 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsWithoutAuthTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsWithoutAuthTest.java @@ -74,6 +74,7 @@ protected void setup() throws Exception { proxyConfig.setMetadataStoreUrl(DUMMY_VALUE); proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); + proxyConfig.setClusterName(configClusterName); proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)))); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyLookupThrottlingTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyLookupThrottlingTest.java index 167c3b196465a..a9017404d0e9f 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyLookupThrottlingTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyLookupThrottlingTest.java @@ -65,6 +65,7 @@ protected void setup() throws Exception { proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); proxyConfig.setMaxConcurrentLookupRequests(NUM_CONCURRENT_LOOKUP); proxyConfig.setMaxConcurrentInboundConnections(NUM_CONCURRENT_INBOUND_CONNECTION); + proxyConfig.setClusterName(configClusterName); AuthenticationService authenticationService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyMutualTlsTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyMutualTlsTest.java index 08066f2e5bf53..fae44c00ada42 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyMutualTlsTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyMutualTlsTest.java @@ -66,6 +66,7 @@ protected void setup() throws Exception { proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); proxyConfig.setTlsRequireTrustedClientCertOnConnect(true); proxyConfig.setTlsAllowInsecureConnection(false); + proxyConfig.setClusterName(configClusterName); proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)))); 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 0d93185f5e899..3f58250e6d68a 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 @@ -71,6 +71,7 @@ protected void setup() throws Exception { proxyConfig.setBrokerProxyAllowedTargetPorts("*"); proxyConfig.setMetadataStoreUrl(DUMMY_VALUE); proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); + proxyConfig.setClusterName(configClusterName); //enable full parsing feature proxyConfig.setProxyLogLevel(Optional.ofNullable(2)); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java index 6beed27cb6622..b3decf00d7fd6 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java @@ -116,6 +116,7 @@ protected void setup() throws Exception { proxyConfig.setServicePort(Optional.of(0)); proxyConfig.setBrokerProxyAllowedTargetPorts("*"); proxyConfig.setWebServicePort(Optional.of(0)); + proxyConfig.setClusterName(configClusterName); proxyConfig.setBrokerClientAuthenticationPlugin(AuthenticationToken.class.getName()); proxyConfig.setBrokerClientAuthenticationParameters( diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRolesEnforcementTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRolesEnforcementTest.java index 137ea82951519..a6dc758b17e7c 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRolesEnforcementTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRolesEnforcementTest.java @@ -209,6 +209,7 @@ public void testIncorrectRoles() throws Exception { proxyConfig.setBrokerProxyAllowedTargetPorts("*"); proxyConfig.setWebServicePort(Optional.of(0)); proxyConfig.setBrokerServiceURL(pulsar.getBrokerServiceUrl()); + proxyConfig.setClusterName(configClusterName); proxyConfig.setBrokerClientAuthenticationPlugin(BasicAuthentication.class.getName()); proxyConfig.setBrokerClientAuthenticationParameters(proxyAuthParams); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java index 925e8192e145a..7fd6e3acb4e6e 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java @@ -61,6 +61,7 @@ protected void setup() throws Exception { serviceStarter.getConfig().setServicePort(Optional.of(0)); serviceStarter.getConfig().setWebSocketServiceEnabled(true); serviceStarter.getConfig().setBrokerProxyAllowedTargetPorts("*"); + serviceStarter.getConfig().setClusterName(DUMMY_VALUE); serviceStarter.start(); serviceUrl = serviceStarter.getProxyService().getServiceUrl(); } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceTlsStarterTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceTlsStarterTest.java index b21162577a25e..6ea9f6161255a 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceTlsStarterTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceTlsStarterTest.java @@ -68,6 +68,7 @@ protected void setup() throws Exception { serviceStarter.getConfig().setTlsCertificateFilePath(PROXY_CERT_FILE_PATH); serviceStarter.getConfig().setTlsKeyFilePath(PROXY_KEY_FILE_PATH); serviceStarter.getConfig().setBrokerProxyAllowedTargetPorts("*"); + serviceStarter.getConfig().setClusterName(DUMMY_VALUE); serviceStarter.start(); serviceUrl = serviceStarter.getProxyService().getServiceUrlTls(); webPort = serviceStarter.getServer().getListenPortHTTP().get(); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStatsTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStatsTest.java index 155fbf616b0d5..2866c6c26907c 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStatsTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStatsTest.java @@ -72,6 +72,7 @@ protected void setup() throws Exception { proxyConfig.setWebServicePort(Optional.of(0)); proxyConfig.setMetadataStoreUrl(DUMMY_VALUE); proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); + proxyConfig.setClusterName(configClusterName); // enable full parsing feature proxyConfig.setProxyLogLevel(Optional.of(2)); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStuckConnectionTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStuckConnectionTest.java index 79ea7c5d6a31c..6e66008c15aef 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStuckConnectionTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStuckConnectionTest.java @@ -79,6 +79,7 @@ protected void setup() throws Exception { proxyConfig.setServicePort(Optional.ofNullable(0)); proxyConfig.setBrokerProxyAllowedTargetPorts("*"); proxyConfig.setBrokerServiceURL(pulsar.getBrokerServiceUrl()); + proxyConfig.setClusterName(configClusterName); startProxyService(); // use the same port for subsequent restarts 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 ac08052aaf153..9bc12dcc6fcb2 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 @@ -106,6 +106,7 @@ protected void initializeProxyConfig() { proxyConfig.setBrokerProxyAllowedTargetPorts("*"); proxyConfig.setMetadataStoreUrl(DUMMY_VALUE); proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); + proxyConfig.setClusterName(configClusterName); } @Override diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsTest.java index a1b27abece4d1..4e300d39741c3 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsTest.java @@ -61,6 +61,7 @@ protected void setup() throws Exception { proxyConfig.setTlsKeyFilePath(PROXY_KEY_FILE_PATH); proxyConfig.setMetadataStoreUrl(DUMMY_VALUE); proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); + proxyConfig.setClusterName(configClusterName); proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)))); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsWithAuthTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsWithAuthTest.java index ec5cace8a06df..16f610d6d0a3a 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsWithAuthTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsWithAuthTest.java @@ -73,6 +73,7 @@ protected void setup() throws Exception { " \"issuerUrl\":\"" + server.getIssuer() + "\"," + " \"audience\": \"an-audience\"," + " \"privateKey\":\"file://" + tempFile.getAbsolutePath() + "\"}"); + proxyConfig.setClusterName(configClusterName); proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)))); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationNegTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationNegTest.java index e0dcefe2714be..cf9ad5831ec0a 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationNegTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationNegTest.java @@ -57,6 +57,7 @@ public class ProxyWithAuthorizationNegTest extends ProducerConsumerBase { private static final Logger log = LoggerFactory.getLogger(ProxyWithAuthorizationNegTest.class); + private static final String CLUSTER_NAME = "proxy-authorization-neg"; private final String TLS_PROXY_TRUST_CERT_FILE_PATH = "./src/test/resources/authentication/tls/ProxyWithAuthorizationTest/proxy-cacert.pem"; private final String TLS_PROXY_CERT_FILE_PATH = "./src/test/resources/authentication/tls/ProxyWithAuthorizationTest/proxy-cert.pem"; @@ -104,7 +105,7 @@ protected void setup() throws Exception { providers.add(AuthenticationProviderTls.class.getName()); conf.setAuthenticationProviders(providers); - conf.setClusterName("proxy-authorization-neg"); + conf.setClusterName(CLUSTER_NAME); conf.setNumExecutorThreadPoolSize(5); super.init(); @@ -121,6 +122,7 @@ protected void setup() throws Exception { proxyConfig.setWebServicePort(Optional.of(0)); proxyConfig.setWebServicePortTls(Optional.of(0)); proxyConfig.setTlsEnabledWithBroker(true); + proxyConfig.setClusterName(CLUSTER_NAME); // enable tls and auth&auth at proxy proxyConfig.setTlsCertificateFilePath(TLS_PROXY_CERT_FILE_PATH); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java index 4e4c3c550cfd6..05c62aa90239b 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java @@ -64,6 +64,7 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase { private static final Logger log = LoggerFactory.getLogger(ProxyWithAuthorizationTest.class); + public static final String CLUSTER_NAME = "proxy-authorization"; private final SecretKey SECRET_KEY = AuthTokenUtils.createSecretKey(SignatureAlgorithm.HS256); private final String CLIENT_TOKEN = AuthTokenUtils.createToken(SECRET_KEY, "Client", Optional.empty()); @@ -189,7 +190,7 @@ protected void doInitConf() throws Exception { properties.setProperty("tokenSecretKey", AuthTokenUtils.encodeKeyBase64(SECRET_KEY)); conf.setProperties(properties); - conf.setClusterName("proxy-authorization"); + conf.setClusterName(CLUSTER_NAME); conf.setNumExecutorThreadPoolSize(5); } @@ -206,6 +207,7 @@ protected void setup() throws Exception { proxyConfig.setBrokerServiceURLTLS(pulsar.getBrokerServiceUrlTls()); proxyConfig.setBrokerWebServiceURLTLS(pulsar.getWebServiceAddressTls()); proxyConfig.setAdvertisedAddress(null); + proxyConfig.setClusterName(CLUSTER_NAME); proxyConfig.setBrokerProxyAllowedTargetPorts("*"); proxyConfig.setServicePortTls(Optional.of(0)); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java index f997532b2734c..d3c05fec721b0 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java @@ -110,6 +110,7 @@ private ProxyConfiguration initializeProxyConfig() { proxyConfig.setBrokerProxyAllowedTargetPorts("*"); proxyConfig.setMetadataStoreUrl(DUMMY_VALUE); proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); + proxyConfig.setClusterName(configClusterName); return proxyConfig; } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java index 14be7dadc4147..80ae9a0a41079 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java @@ -119,6 +119,7 @@ protected void setup() throws Exception { proxyConfig.setServicePort(Optional.of(0)); proxyConfig.setBrokerProxyAllowedTargetPorts("*"); proxyConfig.setWebServicePort(Optional.of(0)); + proxyConfig.setClusterName(configClusterName); // enable auth&auth and use JWT at proxy proxyConfig.setBrokerClientAuthenticationPlugin(AuthenticationToken.class.getName()); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithoutServiceDiscoveryTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithoutServiceDiscoveryTest.java index e09194bb21dfc..9d9490e74b5ad 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithoutServiceDiscoveryTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithoutServiceDiscoveryTest.java @@ -54,9 +54,11 @@ public class ProxyWithoutServiceDiscoveryTest extends ProducerConsumerBase { private static final Logger log = LoggerFactory.getLogger(ProxyWithoutServiceDiscoveryTest.class); + private static final String CLUSTER_NAME = "without-service-discovery"; private ProxyService proxyService; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + @BeforeMethod @Override protected void setup() throws Exception { @@ -89,7 +91,7 @@ protected void setup() throws Exception { providers.add(AuthenticationProviderTls.class.getName()); conf.setAuthenticationProviders(providers); - conf.setClusterName("without-service-discovery"); + conf.setClusterName(CLUSTER_NAME); conf.setNumExecutorThreadPoolSize(5); super.init(); @@ -106,6 +108,7 @@ protected void setup() throws Exception { proxyConfig.setWebServicePort(Optional.of(0)); proxyConfig.setWebServicePortTls(Optional.of(0)); proxyConfig.setTlsEnabledWithBroker(true); + proxyConfig.setClusterName(CLUSTER_NAME); // enable tls and auth&auth at proxy proxyConfig.setTlsCertificateFilePath(PROXY_CERT_FILE_PATH); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/SuperUserAuthedAdminProxyHandlerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/SuperUserAuthedAdminProxyHandlerTest.java index a44e2a85efa61..57522186c8f16 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/SuperUserAuthedAdminProxyHandlerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/SuperUserAuthedAdminProxyHandlerTest.java @@ -80,6 +80,7 @@ protected void setup() throws Exception { proxyConfig.setWebServicePort(Optional.of(0)); proxyConfig.setWebServicePortTls(Optional.of(0)); proxyConfig.setTlsEnabledWithBroker(true); + proxyConfig.setClusterName(configClusterName); // enable tls and auth&auth at proxy proxyConfig.setTlsCertificateFilePath(BROKER_CERT_FILE_PATH); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/UnauthedAdminProxyHandlerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/UnauthedAdminProxyHandlerTest.java index d239815ae81e8..fe8b1f45385e4 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/UnauthedAdminProxyHandlerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/UnauthedAdminProxyHandlerTest.java @@ -75,6 +75,7 @@ protected void setup() throws Exception { proxyConfig.setStatusFilePath(STATUS_FILE_PATH); proxyConfig.setMetadataStoreUrl(DUMMY_VALUE); proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); + proxyConfig.setClusterName(configClusterName); webServer = new WebServer(proxyConfig, new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig))); From 7356ecc5167d75644495db169f015c24e55786b9 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 30 Jan 2024 09:44:20 -0800 Subject: [PATCH 061/133] Fix cluster name discrepancies in some proxy tests --- .../apache/pulsar/proxy/server/ProxyAuthenticationTest.java | 5 +++-- .../apache/pulsar/proxy/server/ProxyForwardAuthDataTest.java | 5 +++-- .../org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java | 5 +++-- .../pulsar/proxy/server/ProxyRolesEnforcementTest.java | 5 +++-- .../pulsar/proxy/server/ProxyWithAuthorizationTest.java | 2 +- .../pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java | 5 +++-- 6 files changed, 16 insertions(+), 11 deletions(-) diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticationTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticationTest.java index 046e6cfb5cd83..662b8305c0e26 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticationTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticationTest.java @@ -58,6 +58,7 @@ public class ProxyAuthenticationTest extends ProducerConsumerBase { private static final Logger log = LoggerFactory.getLogger(ProxyAuthenticationTest.class); + private static final String CLUSTER_NAME = "test"; public static class BasicAuthenticationData implements AuthenticationDataProvider { private final String authParam; @@ -178,7 +179,7 @@ protected void setup() throws Exception { providers.add(BasicAuthenticationProvider.class.getName()); conf.setAuthenticationProviders(providers); - conf.setClusterName("test"); + conf.setClusterName(CLUSTER_NAME); Set proxyRoles = new HashSet<>(); proxyRoles.add("proxy"); conf.setProxyRoles(proxyRoles); @@ -222,7 +223,7 @@ void testAuthentication() throws Exception { proxyConfig.setBrokerProxyAllowedTargetPorts("*"); proxyConfig.setWebServicePort(Optional.of(0)); proxyConfig.setBrokerServiceURL(pulsar.getBrokerServiceUrl()); - proxyConfig.setClusterName(configClusterName); + proxyConfig.setClusterName(CLUSTER_NAME); proxyConfig.setBrokerClientAuthenticationPlugin(BasicAuthentication.class.getName()); proxyConfig.setBrokerClientAuthenticationParameters(proxyAuthParams); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyForwardAuthDataTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyForwardAuthDataTest.java index 6b3188746b618..5e969ca26e4fd 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyForwardAuthDataTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyForwardAuthDataTest.java @@ -46,6 +46,7 @@ public class ProxyForwardAuthDataTest extends ProducerConsumerBase { private static final Logger log = LoggerFactory.getLogger(ProxyForwardAuthDataTest.class); + private static final String CLUSTER_NAME = "test"; @BeforeMethod @Override @@ -64,7 +65,7 @@ protected void setup() throws Exception { providers.add(BasicAuthenticationProvider.class.getName()); conf.setAuthenticationProviders(providers); - conf.setClusterName("test"); + conf.setClusterName(CLUSTER_NAME); Set proxyRoles = new HashSet(); proxyRoles.add("proxy"); conf.setProxyRoles(proxyRoles); @@ -109,7 +110,7 @@ public void testForwardAuthData() throws Exception { proxyConfig.setBrokerServiceURL(pulsar.getBrokerServiceUrl()); proxyConfig.setBrokerClientAuthenticationPlugin(BasicAuthentication.class.getName()); proxyConfig.setBrokerClientAuthenticationParameters(proxyAuthParams); - proxyConfig.setClusterName(configClusterName); + proxyConfig.setClusterName(CLUSTER_NAME); Set providers = new HashSet<>(); providers.add(BasicAuthenticationProvider.class.getName()); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java index b3decf00d7fd6..d06cf4201ff6f 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java @@ -52,6 +52,7 @@ @Slf4j public class ProxyRefreshAuthTest extends ProducerConsumerBase { + private static final String CLUSTER_NAME = "proxy-authorization"; private final SecretKey SECRET_KEY = AuthTokenUtils.createSecretKey(SignatureAlgorithm.HS256); private ProxyService proxyService; @@ -84,7 +85,7 @@ protected void doInitConf() throws Exception { properties.setProperty("tokenAllowedClockSkewSeconds", "2"); conf.setProperties(properties); - conf.setClusterName("proxy-authorization"); + conf.setClusterName(CLUSTER_NAME); conf.setNumExecutorThreadPoolSize(5); conf.setAuthenticationRefreshCheckSeconds(1); @@ -116,7 +117,7 @@ protected void setup() throws Exception { proxyConfig.setServicePort(Optional.of(0)); proxyConfig.setBrokerProxyAllowedTargetPorts("*"); proxyConfig.setWebServicePort(Optional.of(0)); - proxyConfig.setClusterName(configClusterName); + proxyConfig.setClusterName(CLUSTER_NAME); proxyConfig.setBrokerClientAuthenticationPlugin(AuthenticationToken.class.getName()); proxyConfig.setBrokerClientAuthenticationParameters( diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRolesEnforcementTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRolesEnforcementTest.java index a6dc758b17e7c..a1ffc13ee9350 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRolesEnforcementTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRolesEnforcementTest.java @@ -49,6 +49,7 @@ public class ProxyRolesEnforcementTest extends ProducerConsumerBase { private static final Logger log = LoggerFactory.getLogger(ProxyRolesEnforcementTest.class); + private static final String CLUSTER_NAME = "test"; public static class BasicAuthenticationData implements AuthenticationDataProvider { private final String authParam; @@ -154,7 +155,7 @@ protected void setup() throws Exception { providers.add(BasicAuthenticationProvider.class.getName()); conf.setAuthenticationProviders(providers); - conf.setClusterName("test"); + conf.setClusterName(CLUSTER_NAME); Set proxyRoles = new HashSet<>(); proxyRoles.add("proxy"); conf.setProxyRoles(proxyRoles); @@ -209,7 +210,7 @@ public void testIncorrectRoles() throws Exception { proxyConfig.setBrokerProxyAllowedTargetPorts("*"); proxyConfig.setWebServicePort(Optional.of(0)); proxyConfig.setBrokerServiceURL(pulsar.getBrokerServiceUrl()); - proxyConfig.setClusterName(configClusterName); + proxyConfig.setClusterName(CLUSTER_NAME); proxyConfig.setBrokerClientAuthenticationPlugin(BasicAuthentication.class.getName()); proxyConfig.setBrokerClientAuthenticationParameters(proxyAuthParams); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java index 05c62aa90239b..bc2a7729f946e 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java @@ -64,7 +64,7 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase { private static final Logger log = LoggerFactory.getLogger(ProxyWithAuthorizationTest.class); - public static final String CLUSTER_NAME = "proxy-authorization"; + private static final String CLUSTER_NAME = "proxy-authorization"; private final SecretKey SECRET_KEY = AuthTokenUtils.createSecretKey(SignatureAlgorithm.HS256); private final String CLIENT_TOKEN = AuthTokenUtils.createToken(SECRET_KEY, "Client", Optional.empty()); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java index 80ae9a0a41079..5fb3e04682421 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java @@ -67,6 +67,7 @@ public class ProxyWithJwtAuthorizationTest extends ProducerConsumerBase { private static final Logger log = LoggerFactory.getLogger(ProxyWithJwtAuthorizationTest.class); + private static final String CLUSTER_NAME = "proxy-authorization"; private final String ADMIN_ROLE = "admin"; private final String PROXY_ROLE = "proxy"; @@ -104,7 +105,7 @@ protected void setup() throws Exception { providers.add(AuthenticationProviderToken.class.getName()); conf.setAuthenticationProviders(providers); - conf.setClusterName("proxy-authorization"); + conf.setClusterName(CLUSTER_NAME); conf.setNumExecutorThreadPoolSize(5); super.init(); @@ -119,7 +120,7 @@ protected void setup() throws Exception { proxyConfig.setServicePort(Optional.of(0)); proxyConfig.setBrokerProxyAllowedTargetPorts("*"); proxyConfig.setWebServicePort(Optional.of(0)); - proxyConfig.setClusterName(configClusterName); + proxyConfig.setClusterName(CLUSTER_NAME); // enable auth&auth and use JWT at proxy proxyConfig.setBrokerClientAuthenticationPlugin(AuthenticationToken.class.getName()); From 9f5ad0a4a55e32b1fe1f8f692b15e7ab65eb877d Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 30 Jan 2024 10:08:58 -0800 Subject: [PATCH 062/133] Proxy test fixes --- .../proxy/server/ProxyServiceStarterDisableZeroCopyTest.java | 3 ++- .../apache/pulsar/proxy/server/ProxyServiceStarterTest.java | 2 +- .../apache/pulsar/proxy/server/ProxyServiceTlsStarterTest.java | 2 +- .../apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java | 1 + 4 files changed, 5 insertions(+), 3 deletions(-) diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterDisableZeroCopyTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterDisableZeroCopyTest.java index 0c9fa5c7ac322..3e598a57277a2 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterDisableZeroCopyTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterDisableZeroCopyTest.java @@ -21,7 +21,7 @@ import java.util.Optional; import org.testng.annotations.BeforeClass; -public class ProxyServiceStarterDisableZeroCopyTest extends ProxyServiceStarterTest{ +public class ProxyServiceStarterDisableZeroCopyTest extends ProxyServiceStarterTest { @Override @BeforeClass @@ -35,6 +35,7 @@ protected void setup() throws Exception { serviceStarter.getConfig().setWebSocketServiceEnabled(true); serviceStarter.getConfig().setBrokerProxyAllowedTargetPorts("*"); serviceStarter.getConfig().setProxyZeroCopyModeEnabled(false); + serviceStarter.getConfig().setClusterName(configClusterName); serviceStarter.start(); serviceUrl = serviceStarter.getProxyService().getServiceUrl(); } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java index 7fd6e3acb4e6e..ab3783ce08a8a 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java @@ -61,7 +61,7 @@ protected void setup() throws Exception { serviceStarter.getConfig().setServicePort(Optional.of(0)); serviceStarter.getConfig().setWebSocketServiceEnabled(true); serviceStarter.getConfig().setBrokerProxyAllowedTargetPorts("*"); - serviceStarter.getConfig().setClusterName(DUMMY_VALUE); + serviceStarter.getConfig().setClusterName(configClusterName); serviceStarter.start(); serviceUrl = serviceStarter.getProxyService().getServiceUrl(); } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceTlsStarterTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceTlsStarterTest.java index 6ea9f6161255a..61718bbac3ab0 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceTlsStarterTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceTlsStarterTest.java @@ -68,7 +68,7 @@ protected void setup() throws Exception { serviceStarter.getConfig().setTlsCertificateFilePath(PROXY_CERT_FILE_PATH); serviceStarter.getConfig().setTlsKeyFilePath(PROXY_KEY_FILE_PATH); serviceStarter.getConfig().setBrokerProxyAllowedTargetPorts("*"); - serviceStarter.getConfig().setClusterName(DUMMY_VALUE); + serviceStarter.getConfig().setClusterName(configClusterName); serviceStarter.start(); serviceUrl = serviceStarter.getProxyService().getServiceUrlTls(); webPort = serviceStarter.getServer().getListenPortHTTP().get(); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java index bc2a7729f946e..bc96c7ea51041 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java @@ -434,6 +434,7 @@ public void tlsCiphersAndProtocols(Set tlsCiphers, Set tlsProtoc proxyConfig.setBrokerServiceURL(pulsar.getBrokerServiceUrl()); proxyConfig.setBrokerServiceURLTLS(pulsar.getBrokerServiceUrlTls()); proxyConfig.setAdvertisedAddress(null); + proxyConfig.setClusterName(CLUSTER_NAME); proxyConfig.setServicePort(Optional.of(0)); proxyConfig.setBrokerProxyAllowedTargetPorts("*"); From 1ef19a63de70cf096b6a609da02e61e2e87f1b46 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 30 Jan 2024 11:43:25 -0800 Subject: [PATCH 063/133] Fix pulsar function tests missing cluster name --- .../apache/pulsar/functions/instance/ContextImplTest.java | 1 + .../functions/worker/FunctionAssignmentTailerTest.java | 5 +++++ 2 files changed, 6 insertions(+) diff --git a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ContextImplTest.java b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ContextImplTest.java index 6516b9284c9ca..9285e395015d0 100644 --- a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ContextImplTest.java +++ b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ContextImplTest.java @@ -96,6 +96,7 @@ public void setup() throws PulsarClientException { .setUserConfig("") .build(); config.setFunctionDetails(functionDetails); + config.setClusterName("test-cluster"); logger = mock(Logger.class); pulsarAdmin = mock(PulsarAdmin.class); diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionAssignmentTailerTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionAssignmentTailerTest.java index 022ebd6ba48b8..c78c68f8923d8 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionAssignmentTailerTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionAssignmentTailerTest.java @@ -60,6 +60,8 @@ @Slf4j public class FunctionAssignmentTailerTest { + private static final String CLUSTER_NAME = "test-cluster"; + @Test(timeOut = 10000) public void testErrorNotifier() throws Exception { WorkerConfig workerConfig = new WorkerConfig(); @@ -71,6 +73,7 @@ public void testErrorNotifier() throws Exception { workerConfig.setPulsarServiceUrl("pulsar://localhost:6650"); workerConfig.setStateStorageServiceUrl("foo"); workerConfig.setFunctionAssignmentTopicName("assignments"); + workerConfig.setPulsarFunctionsCluster(CLUSTER_NAME); Function.FunctionMetaData function1 = Function.FunctionMetaData.newBuilder().setFunctionDetails( Function.FunctionDetails.newBuilder() @@ -183,6 +186,7 @@ public void testProcessingAssignments() throws Exception { workerConfig.setPulsarServiceUrl("pulsar://localhost:6650"); workerConfig.setStateStorageServiceUrl("foo"); workerConfig.setFunctionAssignmentTopicName("assignments"); + workerConfig.setPulsarFunctionsCluster(CLUSTER_NAME); Function.FunctionMetaData function1 = Function.FunctionMetaData.newBuilder().setFunctionDetails( Function.FunctionDetails.newBuilder() @@ -307,6 +311,7 @@ public void testTriggerReadToTheEndAndExit() throws Exception { workerConfig.setPulsarServiceUrl("pulsar://localhost:6650"); workerConfig.setStateStorageServiceUrl("foo"); workerConfig.setFunctionAssignmentTopicName("assignments"); + workerConfig.setPulsarFunctionsCluster(CLUSTER_NAME); Function.FunctionMetaData function1 = Function.FunctionMetaData.newBuilder().setFunctionDetails( Function.FunctionDetails.newBuilder() From cc7790517f6f0da050d742a6f09ab147c79bce6d Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 30 Jan 2024 11:43:38 -0800 Subject: [PATCH 064/133] Temporary assert cluster name not empty --- .../apache/pulsar/common/stats/OpenTelemetryService.java | 1 + .../pulsar/common/stats/OpenTelemetryServiceTest.java | 8 ++++---- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index 6f2ddc7dfa1da..2676d36891844 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -49,6 +49,7 @@ public OpenTelemetryService(String clusterName, @Singular Map extraProperties, // Allows customizing the SDK builder; for testing purposes only. @VisibleForTesting AutoConfiguredOpenTelemetrySdkBuilder sdkBuilder) { + assert !StringUtils.isEmpty(clusterName); checkArgument(StringUtils.isNotEmpty(clusterName), "Cluster name cannot be empty"); if (sdkBuilder == null) { sdkBuilder = AutoConfiguredOpenTelemetrySdk.builder(); diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java index 84e7d42f198fc..ad7b97731b32d 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java @@ -76,14 +76,14 @@ private static AutoConfiguredOpenTelemetrySdkBuilder getSdkBuilder(MetricReader }); } - @Test(expectedExceptions = IllegalArgumentException.class) - public void testIsClusterNameCannotBeNull() throws Exception { + @Test(expectedExceptions = IllegalArgumentException.class, enabled = false) + public void testClusterNameCannotBeNull() throws Exception { @Cleanup OpenTelemetryService ots = OpenTelemetryService.builder().build(); } - @Test(expectedExceptions = IllegalArgumentException.class) - public void testIsClusterNameCannotBeEmpty() throws Exception { + @Test(expectedExceptions = IllegalArgumentException.class, enabled = false) + public void testClusterNameCannotBeEmpty() throws Exception { @Cleanup OpenTelemetryService ots = OpenTelemetryService.builder().clusterName(StringUtils.EMPTY).build(); } From 0d55bbe3f57a9e8aee10245bee41270895e30565 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 30 Jan 2024 12:15:30 -0800 Subject: [PATCH 065/133] Add integration test group METRICS --- .github/workflows/pulsar-ci.yaml | 4 +++ build/run_integration_group.sh | 4 +++ .../src/test/resources/pulsar-metrics.xml | 28 +++++++++++++++++++ .../integration/src/test/resources/pulsar.xml | 1 + 4 files changed, 37 insertions(+) create mode 100644 tests/integration/src/test/resources/pulsar-metrics.xml diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index 7767beaa9aa1e..618360c333b16 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -589,6 +589,10 @@ jobs: - name: Transaction group: TRANSACTION + - name: Metrics + group: METRICS + no_coverage: true + steps: - name: checkout uses: actions/checkout@v4 diff --git a/build/run_integration_group.sh b/build/run_integration_group.sh index f20a7ad0793e1..2d82fce08878d 100755 --- a/build/run_integration_group.sh +++ b/build/run_integration_group.sh @@ -181,6 +181,10 @@ test_group_transaction() { mvn_run_integration_test "$@" -DintegrationTestSuiteFile=pulsar-transaction.xml -DintegrationTests } +test_group_metrics() { + mvn_run_integration_test "$@" -DintegrationTestSuiteFile=pulsar-metrics.xml -DintegrationTests +} + test_group_tiered_filesystem() { mvn_run_integration_test "$@" -DintegrationTestSuiteFile=tiered-filesystem-storage.xml -DintegrationTests } diff --git a/tests/integration/src/test/resources/pulsar-metrics.xml b/tests/integration/src/test/resources/pulsar-metrics.xml new file mode 100644 index 0000000000000..b7550378d552a --- /dev/null +++ b/tests/integration/src/test/resources/pulsar-metrics.xml @@ -0,0 +1,28 @@ + + + + + + + + + diff --git a/tests/integration/src/test/resources/pulsar.xml b/tests/integration/src/test/resources/pulsar.xml index bdc5f27cc78fb..aa9a59a6cda64 100644 --- a/tests/integration/src/test/resources/pulsar.xml +++ b/tests/integration/src/test/resources/pulsar.xml @@ -37,5 +37,6 @@ + From ddb30924ce5e14b3d20c15fd201c9a17f92c9eb3 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 30 Jan 2024 16:48:53 -0800 Subject: [PATCH 066/133] Create pulsar-otel-metrics-provider artifact --- pom.xml | 2 + pulsar-broker-common/pom.xml | 24 ++++ pulsar-common/pom.xml | 24 ---- pulsar-otel-metrics-provider/pom.xml | 103 ++++++++++++++++++ .../common/stats/OpenTelemetryService.java | 0 .../common/stats/MetricDataMatcher.java | 0 .../stats/OpenTelemetryServiceTest.java | 0 7 files changed, 129 insertions(+), 24 deletions(-) create mode 100644 pulsar-otel-metrics-provider/pom.xml rename {pulsar-common => pulsar-otel-metrics-provider}/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java (100%) rename {pulsar-common => pulsar-otel-metrics-provider}/src/test/java/org/apache/pulsar/common/stats/MetricDataMatcher.java (100%) rename {pulsar-common => pulsar-otel-metrics-provider}/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java (100%) diff --git a/pom.xml b/pom.xml index 0986d422e8cd7..ca15ba0f78d04 100644 --- a/pom.xml +++ b/pom.xml @@ -2270,6 +2270,7 @@ flexible messaging model and an intuitive client API. pulsar-broker-auth-sasl pulsar-client-auth-sasl pulsar-config-validation + pulsar-otel-metrics-provider structured-event-log @@ -2334,6 +2335,7 @@ flexible messaging model and an intuitive client API. pulsar-broker-auth-sasl pulsar-client-auth-sasl pulsar-config-validation + pulsar-otel-metrics-provider pulsar-transaction diff --git a/pulsar-broker-common/pom.xml b/pulsar-broker-common/pom.xml index d73dba288a3c6..3f03cbb0d9e07 100644 --- a/pulsar-broker-common/pom.xml +++ b/pulsar-broker-common/pom.xml @@ -69,6 +69,24 @@ jjwt-jackson + + + io.opentelemetry + opentelemetry-api + + + io.opentelemetry + opentelemetry-exporter-otlp + + + io.opentelemetry + opentelemetry-exporter-prometheus + + + io.opentelemetry + opentelemetry-sdk-extension-autoconfigure + + org.bouncycastle @@ -82,6 +100,12 @@ awaitility test + + + io.opentelemetry + opentelemetry-sdk-testing + test + diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index 71ef32cbc3af0..54696ca9ccb4d 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -201,24 +201,6 @@ protobuf-java - - - io.opentelemetry - opentelemetry-api - - - io.opentelemetry - opentelemetry-exporter-otlp - - - io.opentelemetry - opentelemetry-exporter-prometheus - - - io.opentelemetry - opentelemetry-sdk-extension-autoconfigure - - org.bouncycastle @@ -256,12 +238,6 @@ awaitility test - - - io.opentelemetry - opentelemetry-sdk-testing - test - diff --git a/pulsar-otel-metrics-provider/pom.xml b/pulsar-otel-metrics-provider/pom.xml new file mode 100644 index 0000000000000..521086e716a64 --- /dev/null +++ b/pulsar-otel-metrics-provider/pom.xml @@ -0,0 +1,103 @@ + + + + 4.0.0 + + org.apache.pulsar + pulsar + 3.3.0-SNAPSHOT + + + pulsar-otel-metrics-provider + OpenTelemetry integration provider + + + + + io.opentelemetry + opentelemetry-api + + + io.opentelemetry + opentelemetry-exporter-otlp + + + io.opentelemetry + opentelemetry-exporter-prometheus + + + io.opentelemetry + opentelemetry-sdk-extension-autoconfigure + + + + + org.awaitility + awaitility + test + + + + io.opentelemetry + opentelemetry-sdk-testing + test + + + + + + + org.gaul + modernizer-maven-plugin + + true + 8 + + + + modernizer + verify + + modernizer + + + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + checkstyle + verify + + check + + + + + + + diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java similarity index 100% rename from pulsar-common/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java rename to pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/MetricDataMatcher.java b/pulsar-otel-metrics-provider/src/test/java/org/apache/pulsar/common/stats/MetricDataMatcher.java similarity index 100% rename from pulsar-common/src/test/java/org/apache/pulsar/common/stats/MetricDataMatcher.java rename to pulsar-otel-metrics-provider/src/test/java/org/apache/pulsar/common/stats/MetricDataMatcher.java diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-otel-metrics-provider/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java similarity index 100% rename from pulsar-common/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java rename to pulsar-otel-metrics-provider/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java From 65d3cc94fe5e55ba3e2db892db8d3a0176ce57f3 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 30 Jan 2024 16:49:51 -0800 Subject: [PATCH 067/133] Revert pulsar-broker-common POM changes --- pulsar-broker-common/pom.xml | 24 ------------------------ 1 file changed, 24 deletions(-) diff --git a/pulsar-broker-common/pom.xml b/pulsar-broker-common/pom.xml index 3f03cbb0d9e07..d73dba288a3c6 100644 --- a/pulsar-broker-common/pom.xml +++ b/pulsar-broker-common/pom.xml @@ -69,24 +69,6 @@ jjwt-jackson - - - io.opentelemetry - opentelemetry-api - - - io.opentelemetry - opentelemetry-exporter-otlp - - - io.opentelemetry - opentelemetry-exporter-prometheus - - - io.opentelemetry - opentelemetry-sdk-extension-autoconfigure - - org.bouncycastle @@ -100,12 +82,6 @@ awaitility test - - - io.opentelemetry - opentelemetry-sdk-testing - test - From b6be1bc6234610d6814c67481a207a19dee8739b Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 30 Jan 2024 17:11:13 -0800 Subject: [PATCH 068/133] Update POMs --- pulsar-broker/pom.xml | 6 ++++++ pulsar-functions/instance/pom.xml | 6 ++++++ pulsar-otel-metrics-provider/pom.xml | 10 ++++++++++ pulsar-proxy/pom.xml | 6 ++++++ 4 files changed, 28 insertions(+) diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index 9dd319f791191..bfbf52cc2f12e 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -143,6 +143,12 @@ ${project.version} + + ${project.groupId} + pulsar-otel-metrics-provider + ${project.version} + + ${project.groupId} pulsar-io-batch-discovery-triggerers diff --git a/pulsar-functions/instance/pom.xml b/pulsar-functions/instance/pom.xml index 0929d5ff2101b..7e4c5b18e2d8b 100644 --- a/pulsar-functions/instance/pom.xml +++ b/pulsar-functions/instance/pom.xml @@ -95,6 +95,12 @@ ${project.parent.version} + + ${project.groupId} + pulsar-otel-metrics-provider + ${project.version} + + org.apache.bookkeeper stream-storage-java-client diff --git a/pulsar-otel-metrics-provider/pom.xml b/pulsar-otel-metrics-provider/pom.xml index 521086e716a64..95cd8f8478f6e 100644 --- a/pulsar-otel-metrics-provider/pom.xml +++ b/pulsar-otel-metrics-provider/pom.xml @@ -51,6 +51,16 @@ opentelemetry-sdk-extension-autoconfigure + + com.google.guava + guava + + + + org.apache.commons + commons-lang3 + + org.awaitility diff --git a/pulsar-proxy/pom.xml b/pulsar-proxy/pom.xml index 8fb1313f9ce32..8bdd2a301e0dd 100644 --- a/pulsar-proxy/pom.xml +++ b/pulsar-proxy/pom.xml @@ -49,6 +49,12 @@ ${project.version} + + ${project.groupId} + pulsar-otel-metrics-provider + ${project.version} + + ${project.groupId} pulsar-docs-tools From 2ed10599fef47b58bdd3dc05606e8faabff53bc3 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 30 Jan 2024 17:41:17 -0800 Subject: [PATCH 069/133] Add package-info --- .../pulsar/common/stats/package-info.java | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) create mode 100644 pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/package-info.java diff --git a/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/package-info.java b/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/package-info.java new file mode 100644 index 0000000000000..2404947561c7f --- /dev/null +++ b/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/package-info.java @@ -0,0 +1,19 @@ +/* + * 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.common.stats; \ No newline at end of file From 19f3a4b906042d3eb5e90e7c756e2524d8f1c112 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 31 Jan 2024 13:20:05 -0800 Subject: [PATCH 070/133] Remove animal-sniffer-annotations from server LICENSE --- distribution/server/src/assemble/LICENSE.bin.txt | 2 -- 1 file changed, 2 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 1c3aa826056df..f8e24671042bc 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -550,8 +550,6 @@ MIT License * Auth0, Inc. - com.auth0-java-jwt-4.3.0.jar - com.auth0-jwks-rsa-0.22.0.jar - * Mojo - - org.codehaus.mojo-animal-sniffer-annotations-1.21.jar Protocol Buffers License * Protocol Buffers - com.google.protobuf-protobuf-java-3.19.6.jar -- ../licenses/LICENSE-protobuf.txt From f742149c7d94065c28938b3b9cc00febf9b75afe Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 31 Jan 2024 14:00:12 -0800 Subject: [PATCH 071/133] Import correct OTel library in POM --- pulsar-otel-metrics-provider/pom.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pulsar-otel-metrics-provider/pom.xml b/pulsar-otel-metrics-provider/pom.xml index 95cd8f8478f6e..d0a06209090ea 100644 --- a/pulsar-otel-metrics-provider/pom.xml +++ b/pulsar-otel-metrics-provider/pom.xml @@ -36,15 +36,15 @@ io.opentelemetry - opentelemetry-api + opentelemetry-exporter-otlp io.opentelemetry - opentelemetry-exporter-otlp + opentelemetry-exporter-prometheus io.opentelemetry - opentelemetry-exporter-prometheus + opentelemetry-sdk io.opentelemetry From 075c7579720e74c1571b52f037941b3910811cc5 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 31 Jan 2024 14:01:26 -0800 Subject: [PATCH 072/133] Exclude logs and traces dependencies from distribution --- distribution/server/src/assemble/bin.xml | 3 +++ .../org/apache/pulsar/common/stats/OpenTelemetryService.java | 5 +++++ 2 files changed, 8 insertions(+) diff --git a/distribution/server/src/assemble/bin.xml b/distribution/server/src/assemble/bin.xml index 4dfec015c0e6f..50cca611b0ddf 100644 --- a/distribution/server/src/assemble/bin.xml +++ b/distribution/server/src/assemble/bin.xml @@ -128,6 +128,9 @@ com.google.android:annotations net.java.dev.jna:jna + + io.opentelemetry:opentelemetry-sdk-logs + io.opentelemetry:opentelemetry-sdk-trace diff --git a/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index 2676d36891844..4422411c1e487 100644 --- a/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -38,6 +38,8 @@ public class OpenTelemetryService implements Closeable { private static final AttributeKey CLUSTER_NAME_ATTRIBUTE = AttributeKey.stringKey("pulsar.cluster"); public static final String OTEL_SDK_DISABLED = "otel.sdk.disabled"; + private static final String OTEL_TRACES_EXPORTER = "otel.traces.exporter"; + private static final String OTEL_LOGS_EXPORTER = "otel.logs.exporter"; private static final String MAX_CARDINALITY_LIMIT_KEY = "otel.experimental.metrics.cardinality.limit"; public static final int MAX_CARDINALITY_LIMIT = 10000; @@ -57,6 +59,9 @@ public OpenTelemetryService(String clusterName, Map overrideProperties = new HashMap<>(); overrideProperties.put(OTEL_SDK_DISABLED, "true"); + // The logs and traces exporters are not included in the distribution, so we need to disable them. + overrideProperties.put(OTEL_LOGS_EXPORTER, "none"); + overrideProperties.put(OTEL_TRACES_EXPORTER, "none"); // Cardinality limit property is exclusive, so we need to add 1. overrideProperties.put(MAX_CARDINALITY_LIMIT_KEY, Integer.toString(MAX_CARDINALITY_LIMIT + 1)); sdkBuilder.addPropertiesSupplier(() -> overrideProperties); From 7912180e8110b31d11c06772a5aa7a620ada3a61 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 31 Jan 2024 14:02:17 -0800 Subject: [PATCH 073/133] Minor cleanup --- .../apache/pulsar/common/stats/OpenTelemetryService.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index 4422411c1e487..215d2acad381d 100644 --- a/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -35,8 +35,6 @@ public class OpenTelemetryService implements Closeable { - private static final AttributeKey CLUSTER_NAME_ATTRIBUTE = AttributeKey.stringKey("pulsar.cluster"); - public static final String OTEL_SDK_DISABLED = "otel.sdk.disabled"; private static final String OTEL_TRACES_EXPORTER = "otel.traces.exporter"; private static final String OTEL_LOGS_EXPORTER = "otel.logs.exporter"; @@ -69,11 +67,12 @@ public OpenTelemetryService(String clusterName, sdkBuilder.addResourceCustomizer( (resource, __) -> { - if (resource.getAttribute(CLUSTER_NAME_ATTRIBUTE) != null) { + AttributeKey clusterNameAttribute = AttributeKey.stringKey("pulsar.cluster"); + if (resource.getAttribute(clusterNameAttribute) != null) { // Do not override if already set (via system properties or environment variables). return resource; } - return resource.merge(Resource.builder().put(CLUSTER_NAME_ATTRIBUTE, clusterName).build()); + return resource.merge(Resource.builder().put(clusterNameAttribute, clusterName).build()); }); openTelemetrySdk = sdkBuilder.build().getOpenTelemetrySdk(); From eae661621ad821d1a510fe1121a585579c0f92a7 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 31 Jan 2024 14:02:45 -0800 Subject: [PATCH 074/133] Remove debug assertion --- .../org/apache/pulsar/common/stats/OpenTelemetryService.java | 1 - .../apache/pulsar/common/stats/OpenTelemetryServiceTest.java | 4 ++-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index 215d2acad381d..ce13da95efa42 100644 --- a/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -49,7 +49,6 @@ public OpenTelemetryService(String clusterName, @Singular Map extraProperties, // Allows customizing the SDK builder; for testing purposes only. @VisibleForTesting AutoConfiguredOpenTelemetrySdkBuilder sdkBuilder) { - assert !StringUtils.isEmpty(clusterName); checkArgument(StringUtils.isNotEmpty(clusterName), "Cluster name cannot be empty"); if (sdkBuilder == null) { sdkBuilder = AutoConfiguredOpenTelemetrySdk.builder(); diff --git a/pulsar-otel-metrics-provider/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-otel-metrics-provider/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java index ad7b97731b32d..49ccc96bb5a9e 100644 --- a/pulsar-otel-metrics-provider/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-otel-metrics-provider/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java @@ -76,13 +76,13 @@ private static AutoConfiguredOpenTelemetrySdkBuilder getSdkBuilder(MetricReader }); } - @Test(expectedExceptions = IllegalArgumentException.class, enabled = false) + @Test(expectedExceptions = IllegalArgumentException.class) public void testClusterNameCannotBeNull() throws Exception { @Cleanup OpenTelemetryService ots = OpenTelemetryService.builder().build(); } - @Test(expectedExceptions = IllegalArgumentException.class, enabled = false) + @Test(expectedExceptions = IllegalArgumentException.class) public void testClusterNameCannotBeEmpty() throws Exception { @Cleanup OpenTelemetryService ots = OpenTelemetryService.builder().clusterName(StringUtils.EMPTY).build(); From 38160328078036a1fb8890bc10d782e512c28dde Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 31 Jan 2024 14:15:53 -0800 Subject: [PATCH 075/133] Add docs --- .../apache/pulsar/common/stats/OpenTelemetryService.java | 1 - .../org/apache/pulsar/common/stats/MetricDataMatcher.java | 6 +++++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index ce13da95efa42..4ce343a869217 100644 --- a/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -39,7 +39,6 @@ public class OpenTelemetryService implements Closeable { private static final String OTEL_TRACES_EXPORTER = "otel.traces.exporter"; private static final String OTEL_LOGS_EXPORTER = "otel.logs.exporter"; private static final String MAX_CARDINALITY_LIMIT_KEY = "otel.experimental.metrics.cardinality.limit"; - public static final int MAX_CARDINALITY_LIMIT = 10000; private final OpenTelemetrySdk openTelemetrySdk; diff --git a/pulsar-otel-metrics-provider/src/test/java/org/apache/pulsar/common/stats/MetricDataMatcher.java b/pulsar-otel-metrics-provider/src/test/java/org/apache/pulsar/common/stats/MetricDataMatcher.java index e602b10783d18..788efcdd19917 100644 --- a/pulsar-otel-metrics-provider/src/test/java/org/apache/pulsar/common/stats/MetricDataMatcher.java +++ b/pulsar-otel-metrics-provider/src/test/java/org/apache/pulsar/common/stats/MetricDataMatcher.java @@ -33,6 +33,10 @@ import lombok.Builder; import lombok.Singular; +/** + * Utility class to match OpenTelemetry metric data based on a set of optional criteria. If a criterion is set, the + * input MetricData must match it in order for the predicate to evaluate true. + */ @Builder public class MetricDataMatcher implements Predicate { private final String name; @@ -75,7 +79,7 @@ private boolean matchesDataAttributes(MetricData md) { private boolean matchesAttributes(Attributes actual, Attributes expected) { // Return true iff all attribute pairs in expected are a subset of those in actual. Allows tests to specify - // just the attributes they care about, insted of exhaustively having to list all of them. + // just the attributes they care about, instead of exhaustively having to list all of them. return expected.asMap().entrySet().stream().allMatch(e -> e.getValue().equals(actual.get(e.getKey()))); } From bde20f3eb4798b08a1a192fdfbf340371b04c30b Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 31 Jan 2024 14:16:55 -0800 Subject: [PATCH 076/133] Revert "Exclude logs and traces dependencies from distribution" This reverts commit 075c7579720e74c1571b52f037941b3910811cc5. --- distribution/server/src/assemble/bin.xml | 3 --- .../org/apache/pulsar/common/stats/OpenTelemetryService.java | 5 ----- 2 files changed, 8 deletions(-) diff --git a/distribution/server/src/assemble/bin.xml b/distribution/server/src/assemble/bin.xml index 50cca611b0ddf..4dfec015c0e6f 100644 --- a/distribution/server/src/assemble/bin.xml +++ b/distribution/server/src/assemble/bin.xml @@ -128,9 +128,6 @@ com.google.android:annotations net.java.dev.jna:jna - - io.opentelemetry:opentelemetry-sdk-logs - io.opentelemetry:opentelemetry-sdk-trace diff --git a/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index 4ce343a869217..5d65d283a14a0 100644 --- a/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -36,8 +36,6 @@ public class OpenTelemetryService implements Closeable { public static final String OTEL_SDK_DISABLED = "otel.sdk.disabled"; - private static final String OTEL_TRACES_EXPORTER = "otel.traces.exporter"; - private static final String OTEL_LOGS_EXPORTER = "otel.logs.exporter"; private static final String MAX_CARDINALITY_LIMIT_KEY = "otel.experimental.metrics.cardinality.limit"; public static final int MAX_CARDINALITY_LIMIT = 10000; @@ -55,9 +53,6 @@ public OpenTelemetryService(String clusterName, Map overrideProperties = new HashMap<>(); overrideProperties.put(OTEL_SDK_DISABLED, "true"); - // The logs and traces exporters are not included in the distribution, so we need to disable them. - overrideProperties.put(OTEL_LOGS_EXPORTER, "none"); - overrideProperties.put(OTEL_TRACES_EXPORTER, "none"); // Cardinality limit property is exclusive, so we need to add 1. overrideProperties.put(MAX_CARDINALITY_LIMIT_KEY, Integer.toString(MAX_CARDINALITY_LIMIT + 1)); sdkBuilder.addPropertiesSupplier(() -> overrideProperties); From 7b7ee9ae56e265bc5a943c0748d7ec7096bdb13f Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 31 Jan 2024 14:22:18 -0800 Subject: [PATCH 077/133] Revert pulsar-common/pom.xml changes --- pulsar-common/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index 54696ca9ccb4d..1b54a7aee2d8c 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -198,7 +198,7 @@ com.google.protobuf - protobuf-java + protobuf-java From d74daaacd40606048a8d5d2c233b050a68bf2934 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 31 Jan 2024 14:36:10 -0800 Subject: [PATCH 078/133] Add description to MetricsTest --- .../pulsar/tests/integration/metrics/MetricsTest.java | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java index ddf8afe8015c6..e80d2a2fb6bf6 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java @@ -47,8 +47,10 @@ @Slf4j public class MetricsTest { - // Test with the included Prometheus exporter as well - // https://github.com/open-telemetry/opentelemetry-java/blob/main/sdk-extensions/autoconfigure/README.md#prometheus-exporter + /* + * Validate that the OpenTelemetry metrics can be exported to a remote OpenTelemetry collector. + * https://github.com/open-telemetry/opentelemetry-java/blob/main/sdk-extensions/autoconfigure/README.md#prometheus-exporter + */ @Test(timeOut = 360_000) public void testOpenTelemetryMetricsOtlpExport() throws Exception { var clusterName = "testOpenTelemetryMetrics-" + UUID.randomUUID(); @@ -95,6 +97,11 @@ public void testOpenTelemetryMetricsOtlpExport() throws Exception { metricName, Pair.of("job", functionWorkerOtelServiceName))); } + /* + * Validate that the OpenTelemetry metrics can be exported to a local Prometheus endpoint running in the same + * process space. + * https://github.com/open-telemetry/opentelemetry-java/blob/main/sdk-extensions/autoconfigure/README.md#prometheus-exporter + */ @Test(timeOut = 360_000) public void testOpenTelemetryMetricsPrometheusExport() throws Exception { var prometheusExporterPort = 9464; From f8d4929ce41eb4ec5cc21c84f6ec895a6f74167d Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 31 Jan 2024 14:36:58 -0800 Subject: [PATCH 079/133] Fix cluster name in AdminProxyHandlerKeystoreTLSTest --- .../pulsar/proxy/server/AdminProxyHandlerKeystoreTLSTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerKeystoreTLSTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerKeystoreTLSTest.java index 00454e5e14027..92c644b470dcd 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerKeystoreTLSTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerKeystoreTLSTest.java @@ -101,7 +101,7 @@ protected void setup() throws Exception { proxyConfig.setBrokerClientAuthenticationPlugin(AuthenticationKeyStoreTls.class.getName()); proxyConfig.setBrokerClientAuthenticationParameters(String.format("keyStoreType:%s,keyStorePath:%s,keyStorePassword:%s", KEYSTORE_TYPE, BROKER_KEYSTORE_FILE_PATH, BROKER_KEYSTORE_PW)); - proxyConfig.setClusterName(DUMMY_VALUE); + proxyConfig.setClusterName(configClusterName); resource = new PulsarResources(registerCloseable(new ZKMetadataStore(mockZooKeeper)), registerCloseable(new ZKMetadataStore(mockZooKeeperGlobal))); From d14bbcecb4490b4395e4caf871c89169e6f5216e Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 31 Jan 2024 14:40:03 -0800 Subject: [PATCH 080/133] Close OpenTelemetry service on PulsarService.close --- .../src/main/java/org/apache/pulsar/broker/PulsarService.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index fcb1a80e368bf..2cf0210442be2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -463,6 +463,9 @@ public CompletableFuture closeAsync() { } resetMetricsServlet(); + if (openTelemetry != null) { + openTelemetry.close(); + } if (this.compactionServiceFactory != null) { try { From b8872091c8e00e126b063888dc9c54c444ae3aaa Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 31 Jan 2024 16:24:10 -0800 Subject: [PATCH 081/133] Add OpenTelemetryService description --- .../org/apache/pulsar/common/stats/OpenTelemetryService.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index 5d65d283a14a0..d3defd0a681a2 100644 --- a/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -33,6 +33,10 @@ import lombok.Singular; import org.apache.commons.lang3.StringUtils; +/** + * Provides a common OpenTelemetry service for Pulsar components to use. Responsible for instantiating the OpenTelemetry + * SDK with a set of overrode properties. Once initialized, furnishes access to OpenTelemetry meters. + */ public class OpenTelemetryService implements Closeable { public static final String OTEL_SDK_DISABLED = "otel.sdk.disabled"; From 3e22db8eedd15ffecc3416ba6a1a70e5e8aade1b Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 31 Jan 2024 16:27:36 -0800 Subject: [PATCH 082/133] Cosmetic fixes --- .../pulsar/tests/integration/metrics/MetricsTest.java | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java index e80d2a2fb6bf6..7bb963adedeca 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java @@ -25,7 +25,6 @@ import java.util.UUID; import java.util.concurrent.TimeUnit; import lombok.Cleanup; -import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.apache.pulsar.client.admin.PulsarAdmin; @@ -44,7 +43,6 @@ import org.awaitility.Awaitility; import org.testng.annotations.Test; -@Slf4j public class MetricsTest { /* @@ -99,7 +97,7 @@ public void testOpenTelemetryMetricsOtlpExport() throws Exception { /* * Validate that the OpenTelemetry metrics can be exported to a local Prometheus endpoint running in the same - * process space. + * process space as the broker/proxy/function-worker. * https://github.com/open-telemetry/opentelemetry-java/blob/main/sdk-extensions/autoconfigure/README.md#prometheus-exporter */ @Test(timeOut = 360_000) @@ -152,16 +150,14 @@ public void testOpenTelemetryMetricsPrometheusExport() throws Exception { Pair.of("service_name", functionWorkerOtelServiceName))); } - @SafeVarargs private static boolean hasMetrics(ChaosContainer container, int port, String metricName, - Pair ... expectedLabels) throws Exception { + Pair ... expectedLabels) { var client = new PrometheusMetricsClient(container.getHost(), container.getMappedPort(port)); var allMetrics = client.getMetrics(); var actualMetrics = allMetrics.findByNameAndLabels(metricName, expectedLabels); return !actualMetrics.isEmpty(); } - @SafeVarargs private static Map getCollectorProps(String serviceName, String exporter, Pair ... extraProps) { var defaultProps = Map.of( From 575446dbe77635eaec90d853f299c21a777a7edb Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 1 Feb 2024 13:49:11 -0800 Subject: [PATCH 083/133] Use OpenTelemetry for the Function Worker only, instead of the spawned function processes too --- .../functions/instance/ContextImpl.java | 7 ------- .../functions/worker/PulsarWorkerService.java | 3 +++ .../stats/PulsarWorkerOpenTelemetry.java | 9 +++++---- .../functions/worker/stats/package-info.java | 19 +++++++++++++++++++ 4 files changed, 27 insertions(+), 11 deletions(-) rename pulsar-functions/{instance/src/main/java/org/apache/pulsar/functions/instance => worker/src/main/java/org/apache/pulsar/functions/worker}/stats/PulsarWorkerOpenTelemetry.java (80%) create mode 100644 pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/package-info.java diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java index a7a70a9da7105..dee0f0dee7745 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java @@ -69,7 +69,6 @@ import org.apache.pulsar.functions.instance.stats.ComponentStatsManager; import org.apache.pulsar.functions.instance.stats.FunctionCollectorRegistry; import org.apache.pulsar.functions.instance.stats.FunctionStatsManager; -import org.apache.pulsar.functions.instance.stats.PulsarWorkerOpenTelemetry; import org.apache.pulsar.functions.instance.stats.SinkStatsManager; import org.apache.pulsar.functions.instance.stats.SourceStatsManager; import org.apache.pulsar.functions.proto.Function; @@ -119,7 +118,6 @@ class ContextImpl implements Context, SinkContext, SourceContext, AutoCloseable Map userMetricsLabels = new HashMap<>(); private final String[] metricsLabels; private final Summary userMetricsSummary; - private final PulsarWorkerOpenTelemetry openTelemetry; private final SubscriptionType subscriptionType; @@ -225,7 +223,6 @@ public ContextImpl(InstanceConfig config, Logger logger, PulsarClient client, .quantile(0.99, 0.01) .quantile(0.999, 0.01) .create()); - this.openTelemetry = new PulsarWorkerOpenTelemetry(config); this.componentType = componentType; this.stateManager = stateManager; this.defaultStateStore = (DefaultStateStore) stateManager.getStore( @@ -757,10 +754,6 @@ public void close() { } catch (InterruptedException | ExecutionException e) { logger.warn("Failed to close producers", e); } - - if (openTelemetry != null) { - openTelemetry.close(); - } } @Override diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java index 16cf778e07290..b194fc3edc0d2 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java @@ -67,6 +67,7 @@ import org.apache.pulsar.functions.worker.service.api.Sinks; import org.apache.pulsar.functions.worker.service.api.Sources; import org.apache.pulsar.functions.worker.service.api.Workers; +import org.apache.pulsar.functions.worker.stats.PulsarWorkerOpenTelemetry; import org.apache.pulsar.metadata.api.MetadataStoreException.AlreadyExistsException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -108,6 +109,7 @@ public interface PulsarClientCreator { private PulsarAdmin brokerAdmin; private PulsarAdmin functionAdmin; private MetricsGenerator metricsGenerator; + private PulsarWorkerOpenTelemetry openTelemetry; @VisibleForTesting private URI dlogUri; private LeaderService leaderService; @@ -188,6 +190,7 @@ public void init(WorkerConfig workerConfig, this.statsUpdater = Executors .newSingleThreadScheduledExecutor(new DefaultThreadFactory("worker-stats-updater")); this.metricsGenerator = new MetricsGenerator(this.statsUpdater, workerConfig); + this.openTelemetry = new PulsarWorkerOpenTelemetry(workerConfig); this.workerConfig = workerConfig; this.dlogUri = dlogUri; this.workerStatsManager = new WorkerStatsManager(workerConfig, runAsStandalone); diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/PulsarWorkerOpenTelemetry.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/PulsarWorkerOpenTelemetry.java similarity index 80% rename from pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/PulsarWorkerOpenTelemetry.java rename to pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/PulsarWorkerOpenTelemetry.java index 9f2a28dda8d9d..f1d10ec07d11f 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/PulsarWorkerOpenTelemetry.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/PulsarWorkerOpenTelemetry.java @@ -16,13 +16,13 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.functions.instance.stats; +package org.apache.pulsar.functions.worker.stats; import io.opentelemetry.api.metrics.Meter; import java.io.Closeable; import lombok.Getter; import org.apache.pulsar.common.stats.OpenTelemetryService; -import org.apache.pulsar.functions.instance.InstanceConfig; +import org.apache.pulsar.functions.worker.WorkerConfig; public class PulsarWorkerOpenTelemetry implements Closeable { @@ -31,8 +31,9 @@ public class PulsarWorkerOpenTelemetry implements Closeable { @Getter private final Meter meter; - public PulsarWorkerOpenTelemetry(InstanceConfig instanceConfig) { - openTelemetryService = OpenTelemetryService.builder().clusterName(instanceConfig.getClusterName()).build(); + public PulsarWorkerOpenTelemetry(WorkerConfig workerConfig) { + openTelemetryService = + OpenTelemetryService.builder().clusterName(workerConfig.getPulsarFunctionsCluster()).build(); meter = openTelemetryService.getMeter("pulsar.worker"); } diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/package-info.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/package-info.java new file mode 100644 index 0000000000000..134d6189b49bb --- /dev/null +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/package-info.java @@ -0,0 +1,19 @@ +/* + * 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.functions.worker.stats; \ No newline at end of file From 5a7163afef1c337e384cef290e2cac8cd70d807a Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 1 Feb 2024 13:51:21 -0800 Subject: [PATCH 084/133] Fix integration test to not launch worker processes --- .../integration/metrics/MetricsTest.java | 30 ++----------------- 1 file changed, 3 insertions(+), 27 deletions(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java index 7bb963adedeca..9cfa2ef3496fb 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java @@ -79,7 +79,7 @@ public void testOpenTelemetryMetricsOtlpExport() throws Exception { var pulsarCluster = PulsarCluster.forSpec(spec); pulsarCluster.start(); - setupFunctionWorker(pulsarCluster, functionWorkerServiceNameSuffix); + pulsarCluster.setupFunctionWorkers(functionWorkerServiceNameSuffix, FunctionRuntimeType.PROCESS, 1); // TODO: Validate cluster name is present once // https://github.com/open-telemetry/opentelemetry-java/issues/6108 is solved. @@ -133,7 +133,8 @@ public void testOpenTelemetryMetricsPrometheusExport() throws Exception { var pulsarCluster = PulsarCluster.forSpec(spec); pulsarCluster.start(); - var workerContainer = setupFunctionWorker(pulsarCluster, functionWorkerServiceNameSuffix); + pulsarCluster.setupFunctionWorkers(functionWorkerServiceNameSuffix, FunctionRuntimeType.PROCESS, 1); + var workerContainer = pulsarCluster.getAnyWorker(); var metricName = "target_info"; // Sent automatically by the OpenTelemetry SDK. Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> @@ -170,29 +171,4 @@ private static Map getCollectorProps(String serviceName, String Arrays.stream(extraProps).forEach(p -> props.put(p.getKey(), p.getValue())); return props; } - - private static WorkerContainer setupFunctionWorker(PulsarCluster pulsarCluster, String suffix) throws Exception { - pulsarCluster.setupFunctionWorkers(suffix, FunctionRuntimeType.PROCESS, 1); - - var namespace = NamespaceName.get("public", "default"); - var sourceTopicName = TopicName.get(TopicDomain.persistent.toString(), namespace, "metricTestSource-" + suffix); - var sinkTopicName = TopicName.get(TopicDomain.persistent.toString(), namespace, "metricTestSink-" + suffix); - - try (PulsarAdmin admin = PulsarAdmin.builder().serviceHttpUrl(pulsarCluster.getHttpServiceUrl()).build()) { - admin.topics().createNonPartitionedTopic(sourceTopicName.toString()); - admin.topics().createNonPartitionedTopic(sinkTopicName.toString()); - } - - var commandGenerator = new CommandGenerator(); - commandGenerator.setSourceTopic(sourceTopicName.toString()); - commandGenerator.setSinkTopic(sinkTopicName.toString()); - commandGenerator.setRuntime(CommandGenerator.Runtime.JAVA); - commandGenerator.setFunctionName("metricsTestLocalRunTest-" + suffix); - commandGenerator.setFunctionClassName(PulsarFunctionsTest.EXCLAMATION_JAVA_CLASS); - var functionWorkerCommand = commandGenerator.generateCreateFunctionCommand(); - - var workerContainer = pulsarCluster.getAnyWorker(); - workerContainer.execCmdAsync("sh", "-c", functionWorkerCommand); - return workerContainer; - } } From ae1205e6962f311379a8ec9b1a7bf708aa52755d Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 1 Feb 2024 13:51:55 -0800 Subject: [PATCH 085/133] Cosmetic fixes --- .../pulsar/tests/integration/metrics/MetricsTest.java | 7 ------- 1 file changed, 7 deletions(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java index 9cfa2ef3496fb..658f5c387cb97 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java @@ -27,15 +27,8 @@ import lombok.Cleanup; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; -import org.apache.pulsar.client.admin.PulsarAdmin; -import org.apache.pulsar.common.naming.NamespaceName; -import org.apache.pulsar.common.naming.TopicDomain; -import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.tests.integration.containers.ChaosContainer; import org.apache.pulsar.tests.integration.containers.OpenTelemetryCollectorContainer; -import org.apache.pulsar.tests.integration.containers.WorkerContainer; -import org.apache.pulsar.tests.integration.functions.PulsarFunctionsTest; -import org.apache.pulsar.tests.integration.functions.utils.CommandGenerator; import org.apache.pulsar.tests.integration.topologies.FunctionRuntimeType; import org.apache.pulsar.tests.integration.topologies.PulsarCluster; import org.apache.pulsar.tests.integration.topologies.PulsarClusterSpec; From 1c4b4c05f33c64bb1059cfdc9488c2d760de4bfd Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 1 Feb 2024 14:03:33 -0800 Subject: [PATCH 086/133] Move dependency to pulsar-functions/worker --- pulsar-functions/worker/pom.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/pulsar-functions/worker/pom.xml b/pulsar-functions/worker/pom.xml index cd89bacbf9ede..6c6f09ec02e12 100644 --- a/pulsar-functions/worker/pom.xml +++ b/pulsar-functions/worker/pom.xml @@ -46,6 +46,12 @@ ${project.version} + + ${project.groupId} + pulsar-otel-metrics-provider + ${project.version} + + ${project.groupId} pulsar-client-original From dd8099095387a9515255b71ecce9dfcdb99d5d9f Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 1 Feb 2024 14:04:49 -0800 Subject: [PATCH 087/133] Revert pulsar-functions-instance changes --- pulsar-functions/instance/pom.xml | 6 ------ .../org/apache/pulsar/functions/instance/ContextImpl.java | 2 +- .../apache/pulsar/functions/instance/ContextImplTest.java | 1 - 3 files changed, 1 insertion(+), 8 deletions(-) diff --git a/pulsar-functions/instance/pom.xml b/pulsar-functions/instance/pom.xml index 7e4c5b18e2d8b..0929d5ff2101b 100644 --- a/pulsar-functions/instance/pom.xml +++ b/pulsar-functions/instance/pom.xml @@ -95,12 +95,6 @@ ${project.parent.version} - - ${project.groupId} - pulsar-otel-metrics-provider - ${project.version} - - org.apache.bookkeeper stream-storage-java-client diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java index dee0f0dee7745..6664a00510e56 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java @@ -731,7 +731,7 @@ public void setUnderlyingBuilder(TypedMessageBuilder underlyingBuilder) { @Override public void close() { - List> futures = new LinkedList<>(); + List futures = new LinkedList<>(); if (publishProducers != null) { for (Producer producer : publishProducers.values()) { diff --git a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ContextImplTest.java b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ContextImplTest.java index 9285e395015d0..6516b9284c9ca 100644 --- a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ContextImplTest.java +++ b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ContextImplTest.java @@ -96,7 +96,6 @@ public void setup() throws PulsarClientException { .setUserConfig("") .build(); config.setFunctionDetails(functionDetails); - config.setClusterName("test-cluster"); logger = mock(Logger.class); pulsarAdmin = mock(PulsarAdmin.class); From af61cc12bba5b93a1ca6bfa1400bc406f487994e Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 1 Feb 2024 23:08:15 -0800 Subject: [PATCH 088/133] Add test OpenTelemetryServiceTest#testServiceIsDisabledByDefault --- .../stats/OpenTelemetryServiceTest.java | 52 ++++++++++++++++--- 1 file changed, 45 insertions(+), 7 deletions(-) diff --git a/pulsar-otel-metrics-provider/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-otel-metrics-provider/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java index 49ccc96bb5a9e..feb6b0782e3d2 100644 --- a/pulsar-otel-metrics-provider/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-otel-metrics-provider/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java @@ -18,10 +18,13 @@ */ package org.apache.pulsar.common.stats; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; import io.opentelemetry.api.common.AttributeKey; import io.opentelemetry.api.common.Attributes; import io.opentelemetry.api.metrics.DoubleCounter; import io.opentelemetry.api.metrics.LongCounter; +import io.opentelemetry.api.metrics.LongCounterBuilder; import io.opentelemetry.api.metrics.Meter; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdk; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; @@ -33,10 +36,11 @@ import io.opentelemetry.sdk.metrics.internal.state.MetricStorage; import io.opentelemetry.sdk.testing.exporter.InMemoryMetricReader; import java.util.Collection; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Predicate; import lombok.Cleanup; import org.apache.commons.lang3.StringUtils; -import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -105,7 +109,7 @@ public void testIsClusterNameSet() throws Exception { build(); Collection metricData = reader.collectAllMetrics(); - Assert.assertTrue(metricData.stream().anyMatch(predicate)); + assertTrue(metricData.stream().anyMatch(predicate)); } @Test @@ -117,7 +121,7 @@ public void testIsInstrumentationNameSetOnMeter() throws Exception { instrumentationScopeInfo(InstrumentationScopeInfo.create("testInstrumentationScope")). build(); Collection metricData = reader.collectAllMetrics(); - Assert.assertTrue(metricData.stream().anyMatch(predicate)); + assertTrue(metricData.stream().anyMatch(predicate)); } @Test @@ -134,14 +138,14 @@ public void testMetricCardinality() throws Exception { build(); Collection metricData = reader.collectAllMetrics(); - Assert.assertTrue(metricData.stream().noneMatch(hasOverflowAttribute)); + assertTrue(metricData.stream().noneMatch(hasOverflowAttribute)); for (int i = 0; i < OpenTelemetryService.MAX_CARDINALITY_LIMIT + 1; i++) { longCounter.add(1, Attributes.of(AttributeKey.stringKey("attribute"), "value" + i)); } metricData = reader.collectAllMetrics(); - Assert.assertTrue(metricData.stream().anyMatch(hasOverflowAttribute)); + assertTrue(metricData.stream().anyMatch(hasOverflowAttribute)); } @Test @@ -158,7 +162,7 @@ public void testLongCounter() throws Exception { build(); Collection metricData = reader.collectAllMetrics(); - Assert.assertTrue(metricData.stream().anyMatch(predicate)); + assertTrue(metricData.stream().anyMatch(predicate)); } @Test @@ -174,6 +178,40 @@ public void testDoubleCounter() throws Exception { build(); Collection metricData = reader.collectAllMetrics(); - Assert.assertTrue(metricData.stream().anyMatch(predicate)); + assertTrue(metricData.stream().anyMatch(predicate)); + } + + @Test + public void testServiceIsDisabledByDefault() throws Exception { + @Cleanup + var metricReader = InMemoryMetricReader.create(); + + @Cleanup + var ots = OpenTelemetryService.builder(). + sdkBuilder(getSdkBuilder(metricReader)). + clusterName("openTelemetryServiceTestCluster"). + build(); + var meter = ots.getMeter("openTelemetryServiceTestInstrument"); + + var builders = List.of( + meter.counterBuilder("dummyCounterA"), + meter.counterBuilder("dummyCounterB").setDescription("desc"), + meter.counterBuilder("dummyCounterC").setDescription("desc").setUnit("unit"), + meter.counterBuilder("dummyCounterD").setUnit("unit") + ); + + var callbackCount = new AtomicInteger(); + // Validate that no matter how the counters are being built, they are all backed by the same underlying object. + // This ensures we conserve memory when the SDK is disabled. + assertEquals(builders.stream().map(LongCounterBuilder::build).distinct().count(), 1); + assertEquals(builders.stream().map(LongCounterBuilder::buildObserver).distinct().count(), 1); + assertEquals(builders.stream().map(b -> b.buildWithCallback(__ -> callbackCount.incrementAndGet())) + .distinct().count(), 1); + + // Validate that no metrics are being emitted at all. + assertTrue(metricReader.collectAllMetrics().isEmpty()); + + // Validate that the callback has not being called. + assertEquals(callbackCount.get(), 0); } } From bba153e419cd5ea3a2989c224f70852162c5ccdf Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Fri, 2 Feb 2024 12:20:44 -0800 Subject: [PATCH 089/133] Factor out attribute names for reuse --- .../apache/pulsar/common/stats/OpenTelemetryService.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index d3defd0a681a2..05b964ea3ab59 100644 --- a/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -39,6 +39,9 @@ */ public class OpenTelemetryService implements Closeable { + public static final AttributeKey CLUSTER_ATTRIBUTE = AttributeKey.stringKey("pulsar.cluster"); + public static final AttributeKey TOPIC_ATTRIBUTE = AttributeKey.stringKey("pulsar.topic"); + public static final String OTEL_SDK_DISABLED = "otel.sdk.disabled"; private static final String MAX_CARDINALITY_LIMIT_KEY = "otel.experimental.metrics.cardinality.limit"; public static final int MAX_CARDINALITY_LIMIT = 10000; @@ -64,12 +67,11 @@ public OpenTelemetryService(String clusterName, sdkBuilder.addResourceCustomizer( (resource, __) -> { - AttributeKey clusterNameAttribute = AttributeKey.stringKey("pulsar.cluster"); - if (resource.getAttribute(clusterNameAttribute) != null) { + if (resource.getAttribute(CLUSTER_ATTRIBUTE) != null) { // Do not override if already set (via system properties or environment variables). return resource; } - return resource.merge(Resource.builder().put(clusterNameAttribute, clusterName).build()); + return resource.merge(Resource.builder().put(CLUSTER_ATTRIBUTE, clusterName).build()); }); openTelemetrySdk = sdkBuilder.build().getOpenTelemetrySdk(); From 82acbe9cb355b14f26ff29f676e8812281b2fbd6 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Fri, 2 Feb 2024 16:42:40 -0800 Subject: [PATCH 090/133] Add more attribute definitions --- .../pulsar/common/stats/OpenTelemetryService.java | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index 05b964ea3ab59..9fe4fcddf3ec4 100644 --- a/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -40,7 +40,17 @@ public class OpenTelemetryService implements Closeable { public static final AttributeKey CLUSTER_ATTRIBUTE = AttributeKey.stringKey("pulsar.cluster"); + public static final AttributeKey CONSUMER_ID_ATTRIBUTE = AttributeKey.longKey("pulsar.consumer.id"); + public static final AttributeKey CONSUMER_NAME_ATTRIBUTE = AttributeKey.stringKey("pulsar.consumer"); + public static final AttributeKey NAMESPACE_ATTRIBUTE = AttributeKey.stringKey("pulsar.namespace"); + public static final AttributeKey PRODUCER_ID_ATTRIBUTE = AttributeKey.longKey("pulsar.producer.id"); + public static final AttributeKey PRODUCER_NAME_ATTRIBUTE = AttributeKey.stringKey("pulsar.producer"); + public static final AttributeKey SUBSCRIPTION_ATTRIBUTE = AttributeKey.stringKey("pulsar.subscription"); + public static final AttributeKey SUBSCRIPTION_TYPE_ATTRIBUTE = + AttributeKey.stringKey("pulsar.subscription.type"); + public static final AttributeKey TENANT_ATTRIBUTE = AttributeKey.stringKey("pulsar.tenant"); public static final AttributeKey TOPIC_ATTRIBUTE = AttributeKey.stringKey("pulsar.topic"); + public static final AttributeKey TOPIC_DOMAIN_ATTRIBUTE = AttributeKey.stringKey("pulsar.topic.domain"); public static final String OTEL_SDK_DISABLED = "otel.sdk.disabled"; private static final String MAX_CARDINALITY_LIMIT_KEY = "otel.experimental.metrics.cardinality.limit"; From fa444c7fa196088d4cae571986c47660b0e18956 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 5 Feb 2024 11:15:28 -0800 Subject: [PATCH 091/133] Use ServiceConfiguration parameter in PulsarBrokerOpenTelemetry constructor --- .../main/java/org/apache/pulsar/broker/PulsarService.java | 2 +- .../pulsar/broker/stats/PulsarBrokerOpenTelemetry.java | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 2cf0210442be2..3701f354b62b0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -902,7 +902,7 @@ public void start() throws PulsarServerException { } this.metricsGenerator = new MetricsGenerator(this); - this.openTelemetry = new PulsarBrokerOpenTelemetry(this); + this.openTelemetry = new PulsarBrokerOpenTelemetry(config); // Initialize the message protocol handlers. // start the protocol handlers only after the broker is ready, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java index 589c4b24f4845..ba76c2ee9cd95 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java @@ -21,7 +21,7 @@ import io.opentelemetry.api.metrics.Meter; import java.io.Closeable; import lombok.Getter; -import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.common.stats.OpenTelemetryService; public class PulsarBrokerOpenTelemetry implements Closeable { @@ -31,8 +31,8 @@ public class PulsarBrokerOpenTelemetry implements Closeable { @Getter private final Meter meter; - public PulsarBrokerOpenTelemetry(PulsarService pulsar) { - openTelemetryService = OpenTelemetryService.builder().clusterName(pulsar.getConfig().getClusterName()).build(); + public PulsarBrokerOpenTelemetry(ServiceConfiguration config) { + openTelemetryService = OpenTelemetryService.builder().clusterName(config.getClusterName()).build(); meter = openTelemetryService.getMeter("pulsar.broker"); } From d84655baa96bc74fc2c719a63f0ccd1952a91468 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 5 Feb 2024 11:21:24 -0800 Subject: [PATCH 092/133] Fix instrumentation scope names --- .../apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java | 3 ++- .../functions/worker/stats/PulsarWorkerOpenTelemetry.java | 3 ++- .../apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java | 3 ++- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java index ba76c2ee9cd95..730ff1c870c1f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java @@ -26,6 +26,7 @@ public class PulsarBrokerOpenTelemetry implements Closeable { + private static final String INSTRUMENTATION_SCOPE_NAME = "org.apache.pulsar.broker"; private final OpenTelemetryService openTelemetryService; @Getter @@ -33,7 +34,7 @@ public class PulsarBrokerOpenTelemetry implements Closeable { public PulsarBrokerOpenTelemetry(ServiceConfiguration config) { openTelemetryService = OpenTelemetryService.builder().clusterName(config.getClusterName()).build(); - meter = openTelemetryService.getMeter("pulsar.broker"); + meter = openTelemetryService.getMeter(INSTRUMENTATION_SCOPE_NAME); } @Override diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/PulsarWorkerOpenTelemetry.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/PulsarWorkerOpenTelemetry.java index f1d10ec07d11f..cdda2ce10676c 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/PulsarWorkerOpenTelemetry.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/PulsarWorkerOpenTelemetry.java @@ -26,6 +26,7 @@ public class PulsarWorkerOpenTelemetry implements Closeable { + private static final String INSTRUMENTATION_SCOPE_NAME = "org.apache.pulsar.function_worker"; private final OpenTelemetryService openTelemetryService; @Getter @@ -34,7 +35,7 @@ public class PulsarWorkerOpenTelemetry implements Closeable { public PulsarWorkerOpenTelemetry(WorkerConfig workerConfig) { openTelemetryService = OpenTelemetryService.builder().clusterName(workerConfig.getPulsarFunctionsCluster()).build(); - meter = openTelemetryService.getMeter("pulsar.worker"); + meter = openTelemetryService.getMeter(INSTRUMENTATION_SCOPE_NAME); } public void close() { diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java index da31a52bf78df..45902b91fb1e0 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java @@ -26,6 +26,7 @@ public class PulsarProxyOpenTelemetry implements Closeable { + private static final String INSTRUMENTATION_SCOPE_NAME = "org.apache.pulsar.proxy"; private final OpenTelemetryService openTelemetryService; @Getter @@ -34,7 +35,7 @@ public class PulsarProxyOpenTelemetry implements Closeable { public PulsarProxyOpenTelemetry(ProxyService proxyService) { openTelemetryService = OpenTelemetryService.builder().clusterName(proxyService.getConfiguration().getClusterName()).build(); - meter = openTelemetryService.getMeter("pulsar.proxy"); + meter = openTelemetryService.getMeter(INSTRUMENTATION_SCOPE_NAME); } @Override From 5d7f0ab893a5ccfec4a74e75ab85e5d427caebe4 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 5 Feb 2024 11:30:16 -0800 Subject: [PATCH 093/133] Fix description in OpenTelemetryService --- .../org/apache/pulsar/common/stats/OpenTelemetryService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index 9fe4fcddf3ec4..a740aed27d227 100644 --- a/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -35,7 +35,7 @@ /** * Provides a common OpenTelemetry service for Pulsar components to use. Responsible for instantiating the OpenTelemetry - * SDK with a set of overrode properties. Once initialized, furnishes access to OpenTelemetry meters. + * SDK with a set of override properties. Once initialized, furnishes access to OpenTelemetry. */ public class OpenTelemetryService implements Closeable { From 893aae3e16a60d0e7e72a4c61bd5004abbc6cc9f Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 5 Feb 2024 11:37:02 -0800 Subject: [PATCH 094/133] Clarify comment regarding OTel cardinality limit --- .../org/apache/pulsar/common/stats/OpenTelemetryService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index a740aed27d227..2e76f5bba96c6 100644 --- a/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -70,7 +70,7 @@ public OpenTelemetryService(String clusterName, Map overrideProperties = new HashMap<>(); overrideProperties.put(OTEL_SDK_DISABLED, "true"); - // Cardinality limit property is exclusive, so we need to add 1. + // Cardinality limit includes the overflow attribute set, so we need to add 1. overrideProperties.put(MAX_CARDINALITY_LIMIT_KEY, Integer.toString(MAX_CARDINALITY_LIMIT + 1)); sdkBuilder.addPropertiesSupplier(() -> overrideProperties); sdkBuilder.addPropertiesSupplier(() -> extraProperties); From ce7676e839adba2054085188224519cc0e61239d Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 5 Feb 2024 11:37:46 -0800 Subject: [PATCH 095/133] Remove redundant testDoubleCounter --- .../common/stats/OpenTelemetryServiceTest.java | 17 ----------------- 1 file changed, 17 deletions(-) diff --git a/pulsar-otel-metrics-provider/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-otel-metrics-provider/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java index feb6b0782e3d2..b3ffe01799e51 100644 --- a/pulsar-otel-metrics-provider/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-otel-metrics-provider/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java @@ -22,7 +22,6 @@ import static org.testng.Assert.assertTrue; import io.opentelemetry.api.common.AttributeKey; import io.opentelemetry.api.common.Attributes; -import io.opentelemetry.api.metrics.DoubleCounter; import io.opentelemetry.api.metrics.LongCounter; import io.opentelemetry.api.metrics.LongCounterBuilder; import io.opentelemetry.api.metrics.Meter; @@ -165,22 +164,6 @@ public void testLongCounter() throws Exception { assertTrue(metricData.stream().anyMatch(predicate)); } - @Test - public void testDoubleCounter() throws Exception { - DoubleCounter doubleCounter = meter.counterBuilder("dummyDoubleCounter").ofDoubles().build(); - doubleCounter.add(3.14, Attributes.of(AttributeKey.stringKey("dummyAttr"), "dummyValue")); - doubleCounter.add(2.71, Attributes.of(AttributeKey.stringKey("dummyAttr"), "dummyValue")); - - Predicate predicate = MetricDataMatcher.builder(). - name("dummyDoubleCounter"). - dataAttribute(Attributes.of(AttributeKey.stringKey("dummyAttr"), "dummyValue")). - doubleValue(5.85). - build(); - - Collection metricData = reader.collectAllMetrics(); - assertTrue(metricData.stream().anyMatch(predicate)); - } - @Test public void testServiceIsDisabledByDefault() throws Exception { @Cleanup From e9e90783b89cb5d6629f89f4a3e48c03ce45b19e Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 5 Feb 2024 11:38:56 -0800 Subject: [PATCH 096/133] Use ProxyConfiguration in PulsarProxyOpenTelemetry constructor --- .../java/org/apache/pulsar/proxy/server/ProxyService.java | 2 +- .../apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java index 925ca503c77e9..61b00871cecdb 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java @@ -287,7 +287,7 @@ public void start() throws Exception { } createMetricsServlet(); - openTelemetry = new PulsarProxyOpenTelemetry(this); + openTelemetry = new PulsarProxyOpenTelemetry(proxyConfig); // Initialize the message protocol handlers. // start the protocol handlers only after the broker is ready, diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java index 45902b91fb1e0..f07f8407a4b4b 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java @@ -22,6 +22,7 @@ import java.io.Closeable; import lombok.Getter; import org.apache.pulsar.common.stats.OpenTelemetryService; +import org.apache.pulsar.proxy.server.ProxyConfiguration; import org.apache.pulsar.proxy.server.ProxyService; public class PulsarProxyOpenTelemetry implements Closeable { @@ -32,9 +33,8 @@ public class PulsarProxyOpenTelemetry implements Closeable { @Getter private final Meter meter; - public PulsarProxyOpenTelemetry(ProxyService proxyService) { - openTelemetryService = - OpenTelemetryService.builder().clusterName(proxyService.getConfiguration().getClusterName()).build(); + public PulsarProxyOpenTelemetry(ProxyConfiguration config) { + openTelemetryService = OpenTelemetryService.builder().clusterName(config.getClusterName()).build(); meter = openTelemetryService.getMeter(INSTRUMENTATION_SCOPE_NAME); } From d6e5bf85a286b70abd32648ae54ce06f170cb5e6 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 5 Feb 2024 12:19:51 -0800 Subject: [PATCH 097/133] Cleanup creation of OTel Collector container in tests --- .../containers/OpenTelemetryCollectorContainer.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/OpenTelemetryCollectorContainer.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/OpenTelemetryCollectorContainer.java index 3df12d031b24f..2b115ca6b95bf 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/OpenTelemetryCollectorContainer.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/OpenTelemetryCollectorContainer.java @@ -45,10 +45,6 @@ protected void configure() { "/etc/otel-collector-config.yaml") .withCommand("--config=/etc/otel-collector-config.yaml") .withExposedPorts(OTLP_RECEIVER_PORT, PROMETHEUS_EXPORTER_PORT, ZPAGES_PORT) - .withCreateContainerCmdModifier(createContainerCmd -> { - createContainerCmd.withHostName(NAME); - createContainerCmd.withName(getContainerName()); - }) .waitingFor(new HttpWaitStrategy() .forPath("/debug/servicez") .forPort(ZPAGES_PORT) From b15e31a92dadc69758a945b886637dc97e0e36ad Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 5 Feb 2024 12:22:30 -0800 Subject: [PATCH 098/133] Rename integration test to OpenTelemetrySanityTest --- .../metrics/{MetricsTest.java => OpenTelemetrySanityTest.java} | 2 +- tests/integration/src/test/resources/pulsar-metrics.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) rename tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/{MetricsTest.java => OpenTelemetrySanityTest.java} (99%) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java similarity index 99% rename from tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java rename to tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java index 658f5c387cb97..18ab60e8827bb 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/MetricsTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java @@ -36,7 +36,7 @@ import org.awaitility.Awaitility; import org.testng.annotations.Test; -public class MetricsTest { +public class OpenTelemetrySanityTest { /* * Validate that the OpenTelemetry metrics can be exported to a remote OpenTelemetry collector. diff --git a/tests/integration/src/test/resources/pulsar-metrics.xml b/tests/integration/src/test/resources/pulsar-metrics.xml index b7550378d552a..1c87f2bdf0d06 100644 --- a/tests/integration/src/test/resources/pulsar-metrics.xml +++ b/tests/integration/src/test/resources/pulsar-metrics.xml @@ -22,7 +22,7 @@ - + From d9a16142557547c7bfad75306021241625ff8a3b Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 5 Feb 2024 13:01:43 -0800 Subject: [PATCH 099/133] Generalize function worker container specs across all workers --- .../metrics/OpenTelemetrySanityTest.java | 7 +++--- .../integration/topologies/PulsarCluster.java | 14 +++++------ .../topologies/PulsarClusterSpec.java | 25 +++++++++---------- 3 files changed, 21 insertions(+), 25 deletions(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java index 18ab60e8827bb..5c4b47936bb3c 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java @@ -66,14 +66,13 @@ public void testOpenTelemetryMetricsOtlpExport() throws Exception { .brokerEnvs(brokerCollectorProps) .proxyEnvs(proxyCollectorProps) .externalService("otel-collector", openTelemetryCollectorContainer) - .functionWorkerEnv(functionWorkerServiceNameSuffix, functionWorkerCollectorProps) + .functionWorkerEnvs(functionWorkerCollectorProps) .build(); @Cleanup("stop") var pulsarCluster = PulsarCluster.forSpec(spec); pulsarCluster.start(); pulsarCluster.setupFunctionWorkers(functionWorkerServiceNameSuffix, FunctionRuntimeType.PROCESS, 1); - // TODO: Validate cluster name is present once // https://github.com/open-telemetry/opentelemetry-java/issues/6108 is solved. var metricName = "queueSize_ratio"; // Sent automatically by the OpenTelemetry SDK. @@ -119,8 +118,8 @@ public void testOpenTelemetryMetricsPrometheusExport() throws Exception { .brokerAdditionalPorts(List.of(prometheusExporterPort)) .proxyEnvs(proxyCollectorProps) .proxyAdditionalPorts(List.of(prometheusExporterPort)) - .functionWorkerEnv(functionWorkerServiceNameSuffix, functionWorkerCollectorProps) - .functionWorkerAdditionalPorts(functionWorkerServiceNameSuffix, List.of(prometheusExporterPort)) + .functionWorkerEnvs(functionWorkerCollectorProps) + .functionWorkerAdditionalPorts(List.of(prometheusExporterPort)) .build(); @Cleanup("stop") var pulsarCluster = PulsarCluster.forSpec(spec); diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarCluster.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarCluster.java index 795693a844ff7..dead7b2f072f8 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarCluster.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarCluster.java @@ -94,8 +94,8 @@ public static PulsarCluster forSpec(PulsarClusterSpec spec, CSContainer csContai private final ProxyContainer proxyContainer; private Map> externalServices = Collections.emptyMap(); private Map> externalServiceEnvs; - private Map> functionWorkerEnvs; - private Map> functionWorkerAdditionalPorts; + private final Map functionWorkerEnvs; + private final List functionWorkerAdditionalPorts; private PulsarCluster(PulsarClusterSpec spec, CSContainer csContainer, boolean sharedCsContainer) { @@ -235,6 +235,8 @@ private PulsarCluster(PulsarClusterSpec spec, CSContainer csContainer, boolean s workerContainers.values().forEach(c -> c.withClasspathResourceMapping(key, value, BindMode.READ_WRITE)); }); + functionWorkerEnvs = spec.functionWorkerEnvs; + functionWorkerAdditionalPorts = spec.functionWorkerAdditionalPorts; } public String getPlainTextServiceUrl() { @@ -331,9 +333,6 @@ public void start() throws Exception { log.info("Successfully started external service {}.", service.getKey()); }); } - - this.functionWorkerEnvs = spec.functionWorkerEnvs; - this.functionWorkerAdditionalPorts = spec.functionWorkerAdditionalPorts; } public void startService(String networkAlias, @@ -443,9 +442,8 @@ private void startFunctionWorkersWithProcessContainerFactory(String suffix, int .withEnv("zookeeperServers", ZKContainer.NAME) // bookkeeper tools .withEnv("zkServers", ZKContainer.NAME) - .withEnv(functionWorkerEnvs.getOrDefault(suffix, Collections.emptyMap())) - .withExposedPorts(functionWorkerAdditionalPorts.getOrDefault(suffix, Collections.emptyList()) - .toArray(new Integer[0])) + .withEnv(functionWorkerEnvs) + .withExposedPorts(functionWorkerAdditionalPorts.toArray(new Integer[0])) )); this.startWorkers(); } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java index 657ba08e46a0d..4b0710835ba83 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java @@ -81,19 +81,6 @@ public class PulsarClusterSpec { @Default int numFunctionWorkers = 0; - /** - * Specify envs for function workers. - */ - @Singular - Map> functionWorkerEnvs; - - /** - * Additional ports to expose on function workers. - */ - @Singular("functionWorkerAdditionalPorts") - Map> functionWorkerAdditionalPorts; - - /** * Allow to query the last message */ @@ -160,6 +147,12 @@ public class PulsarClusterSpec { */ Map bookkeeperEnvs; + /** + * Specify envs for function workers. + */ + @Singular + Map functionWorkerEnvs; + /** * Specify mount files. */ @@ -188,6 +181,12 @@ public class PulsarClusterSpec { */ List proxyAdditionalPorts; + /** + * Additional ports to expose on function workers. + */ + @Singular + List functionWorkerAdditionalPorts; + /** * Enable TLS for connection. */ From 7657b21b566072bf1fb702d9a79c29b36c976967 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 5 Feb 2024 13:33:07 -0800 Subject: [PATCH 100/133] Refactor OpenTelemetrySanityTest --- .../metrics/OpenTelemetrySanityTest.java | 61 +++++++++++-------- 1 file changed, 35 insertions(+), 26 deletions(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java index 5c4b47936bb3c..98d693f3f28cf 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java @@ -76,15 +76,21 @@ public void testOpenTelemetryMetricsOtlpExport() throws Exception { // TODO: Validate cluster name is present once // https://github.com/open-telemetry/opentelemetry-java/issues/6108 is solved. var metricName = "queueSize_ratio"; // Sent automatically by the OpenTelemetry SDK. - Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> - hasMetrics(openTelemetryCollectorContainer, OpenTelemetryCollectorContainer.PROMETHEUS_EXPORTER_PORT, - metricName, Pair.of("job", brokerOtelServiceName))); - Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> - hasMetrics(openTelemetryCollectorContainer, OpenTelemetryCollectorContainer.PROMETHEUS_EXPORTER_PORT, - metricName, Pair.of("job", proxyOtelServiceName))); - Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> - hasMetrics(openTelemetryCollectorContainer, OpenTelemetryCollectorContainer.PROMETHEUS_EXPORTER_PORT, - metricName, Pair.of("job", functionWorkerOtelServiceName))); + Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { + var metrics = getMetricsFromPrometheus( + openTelemetryCollectorContainer, OpenTelemetryCollectorContainer.PROMETHEUS_EXPORTER_PORT); + return !metrics.findByNameAndLabels(metricName, "job", brokerOtelServiceName).isEmpty(); + }); + Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { + var metrics = getMetricsFromPrometheus( + openTelemetryCollectorContainer, OpenTelemetryCollectorContainer.PROMETHEUS_EXPORTER_PORT); + return !metrics.findByNameAndLabels(metricName, "job", proxyOtelServiceName).isEmpty(); + }); + Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { + var metrics = getMetricsFromPrometheus( + openTelemetryCollectorContainer, OpenTelemetryCollectorContainer.PROMETHEUS_EXPORTER_PORT); + return !metrics.findByNameAndLabels(metricName, "job", functionWorkerOtelServiceName).isEmpty(); + }); } /* @@ -129,26 +135,29 @@ public void testOpenTelemetryMetricsPrometheusExport() throws Exception { var workerContainer = pulsarCluster.getAnyWorker(); var metricName = "target_info"; // Sent automatically by the OpenTelemetry SDK. - Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> - hasMetrics(pulsarCluster.getAnyBroker(), prometheusExporterPort, metricName, - Pair.of("pulsar_cluster", clusterName), - Pair.of("service_name", brokerOtelServiceName))); - Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> - hasMetrics(pulsarCluster.getProxy(), prometheusExporterPort, metricName, - Pair.of("pulsar_cluster", clusterName), - Pair.of("service_name", proxyOtelServiceName))); - Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> - hasMetrics(workerContainer, prometheusExporterPort, metricName, - Pair.of("pulsar_cluster", clusterName), - Pair.of("service_name", functionWorkerOtelServiceName))); + Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { + var metrics = getMetricsFromPrometheus(pulsarCluster.getAnyBroker(), prometheusExporterPort); + return !metrics.findByNameAndLabels(metricName, + Pair.of("pulsar_cluster", clusterName), + Pair.of("service_name", brokerOtelServiceName)).isEmpty(); + }); + Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { + var metrics = getMetricsFromPrometheus(pulsarCluster.getProxy(), prometheusExporterPort); + return !metrics.findByNameAndLabels(metricName, + Pair.of("pulsar_cluster", clusterName), + Pair.of("service_name", proxyOtelServiceName)).isEmpty(); + }); + Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { + var metrics = getMetricsFromPrometheus(workerContainer, prometheusExporterPort); + return !metrics.findByNameAndLabels(metricName, + Pair.of("pulsar_cluster", clusterName), + Pair.of("service_name", functionWorkerOtelServiceName)).isEmpty(); + }); } - private static boolean hasMetrics(ChaosContainer container, int port, String metricName, - Pair ... expectedLabels) { + private static PrometheusMetricsClient.Metrics getMetricsFromPrometheus(ChaosContainer container, int port) { var client = new PrometheusMetricsClient(container.getHost(), container.getMappedPort(port)); - var allMetrics = client.getMetrics(); - var actualMetrics = allMetrics.findByNameAndLabels(metricName, expectedLabels); - return !actualMetrics.isEmpty(); + return client.getMetrics(); } private static Map getCollectorProps(String serviceName, String exporter, From 8bb00ddb1ef7175ee6b144045e22a35266708b16 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 5 Feb 2024 15:40:58 -0800 Subject: [PATCH 101/133] Rename artifact to pulsar-otel-integration --- pom.xml | 4 ++-- pulsar-broker/pom.xml | 2 +- pulsar-functions/worker/pom.xml | 2 +- .../pom.xml | 4 ++-- .../org/apache/pulsar/common/stats/OpenTelemetryService.java | 0 .../java/org/apache/pulsar/common/stats/package-info.java | 0 .../org/apache/pulsar/common/stats/MetricDataMatcher.java | 0 .../apache/pulsar/common/stats/OpenTelemetryServiceTest.java | 0 pulsar-proxy/pom.xml | 2 +- 9 files changed, 7 insertions(+), 7 deletions(-) rename {pulsar-otel-metrics-provider => pulsar-otel-integration}/pom.xml (96%) rename {pulsar-otel-metrics-provider => pulsar-otel-integration}/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java (100%) rename {pulsar-otel-metrics-provider => pulsar-otel-integration}/src/main/java/org/apache/pulsar/common/stats/package-info.java (100%) rename {pulsar-otel-metrics-provider => pulsar-otel-integration}/src/test/java/org/apache/pulsar/common/stats/MetricDataMatcher.java (100%) rename {pulsar-otel-metrics-provider => pulsar-otel-integration}/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java (100%) diff --git a/pom.xml b/pom.xml index ca15ba0f78d04..f4fa30b9ac8b4 100644 --- a/pom.xml +++ b/pom.xml @@ -2270,7 +2270,7 @@ flexible messaging model and an intuitive client API. pulsar-broker-auth-sasl pulsar-client-auth-sasl pulsar-config-validation - pulsar-otel-metrics-provider + pulsar-otel-integration structured-event-log @@ -2335,7 +2335,7 @@ flexible messaging model and an intuitive client API. pulsar-broker-auth-sasl pulsar-client-auth-sasl pulsar-config-validation - pulsar-otel-metrics-provider + pulsar-otel-integration pulsar-transaction diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index bfbf52cc2f12e..341e47651e464 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -145,7 +145,7 @@ ${project.groupId} - pulsar-otel-metrics-provider + pulsar-otel-integration ${project.version} diff --git a/pulsar-functions/worker/pom.xml b/pulsar-functions/worker/pom.xml index 6c6f09ec02e12..dd808c1918a3e 100644 --- a/pulsar-functions/worker/pom.xml +++ b/pulsar-functions/worker/pom.xml @@ -48,7 +48,7 @@ ${project.groupId} - pulsar-otel-metrics-provider + pulsar-otel-integration ${project.version} diff --git a/pulsar-otel-metrics-provider/pom.xml b/pulsar-otel-integration/pom.xml similarity index 96% rename from pulsar-otel-metrics-provider/pom.xml rename to pulsar-otel-integration/pom.xml index d0a06209090ea..3b8ea11324775 100644 --- a/pulsar-otel-metrics-provider/pom.xml +++ b/pulsar-otel-integration/pom.xml @@ -29,8 +29,8 @@ 3.3.0-SNAPSHOT - pulsar-otel-metrics-provider - OpenTelemetry integration provider + pulsar-otel-integration + OpenTelemetry Integration diff --git a/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java similarity index 100% rename from pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java rename to pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java diff --git a/pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/package-info.java b/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/package-info.java similarity index 100% rename from pulsar-otel-metrics-provider/src/main/java/org/apache/pulsar/common/stats/package-info.java rename to pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/package-info.java diff --git a/pulsar-otel-metrics-provider/src/test/java/org/apache/pulsar/common/stats/MetricDataMatcher.java b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/MetricDataMatcher.java similarity index 100% rename from pulsar-otel-metrics-provider/src/test/java/org/apache/pulsar/common/stats/MetricDataMatcher.java rename to pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/MetricDataMatcher.java diff --git a/pulsar-otel-metrics-provider/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java similarity index 100% rename from pulsar-otel-metrics-provider/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java rename to pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java diff --git a/pulsar-proxy/pom.xml b/pulsar-proxy/pom.xml index 8bdd2a301e0dd..dbe1a9a5eb8a7 100644 --- a/pulsar-proxy/pom.xml +++ b/pulsar-proxy/pom.xml @@ -51,7 +51,7 @@ ${project.groupId} - pulsar-otel-metrics-provider + pulsar-otel-integration ${project.version} From 542221cb1e99680a1ea9fb3325ca1fd3a9527488 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 5 Feb 2024 16:44:49 -0800 Subject: [PATCH 102/133] Checkstyle fix --- .../pulsar/broker/stats/prometheus/PrometheusMetricsClient.java | 0 .../org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java | 1 - 2 files changed, 1 deletion(-) rename {pulsar-broker => pulsar-broker-common}/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java (100%) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java similarity index 100% rename from pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java rename to pulsar-broker-common/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java index f07f8407a4b4b..eb9ae4ddc61f6 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java @@ -23,7 +23,6 @@ import lombok.Getter; import org.apache.pulsar.common.stats.OpenTelemetryService; import org.apache.pulsar.proxy.server.ProxyConfiguration; -import org.apache.pulsar.proxy.server.ProxyService; public class PulsarProxyOpenTelemetry implements Closeable { From cdc34ed05e964b3bd10bf8a3143138634edc2f45 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 5 Feb 2024 22:48:34 -0800 Subject: [PATCH 103/133] Fix metric cardinality test --- pulsar-broker-common/pom.xml | 18 ++++++++ pulsar-broker/pom.xml | 14 +++--- pulsar-otel-integration/pom.xml | 14 ++++++ .../stats/OpenTelemetryServiceTest.java | 43 +++++++++++-------- tests/integration/pom.xml | 2 +- 5 files changed, 65 insertions(+), 26 deletions(-) diff --git a/pulsar-broker-common/pom.xml b/pulsar-broker-common/pom.xml index d73dba288a3c6..8e942c78d5b40 100644 --- a/pulsar-broker-common/pom.xml +++ b/pulsar-broker-common/pom.xml @@ -82,10 +82,28 @@ awaitility test + + + io.rest-assured + rest-assured + test + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + + org.gaul modernizer-maven-plugin diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index 341e47651e464..2b8502ee6902a 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -209,6 +209,14 @@ test + + ${project.groupId} + pulsar-broker-common + ${project.version} + test-jar + test + + @@ -436,12 +444,6 @@ javax.activation - - io.rest-assured - rest-assured - test - - diff --git a/pulsar-otel-integration/pom.xml b/pulsar-otel-integration/pom.xml index 3b8ea11324775..e8f043ff0d615 100644 --- a/pulsar-otel-integration/pom.xml +++ b/pulsar-otel-integration/pom.xml @@ -62,6 +62,20 @@ + + org.apache.pulsar + pulsar-broker-common + ${project.version} + test-jar + test + + + + io.rest-assured + rest-assured + test + + org.awaitility awaitility diff --git a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java index b3ffe01799e51..bfce19b09591c 100644 --- a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java @@ -36,10 +36,13 @@ import io.opentelemetry.sdk.testing.exporter.InMemoryMetricReader; import java.util.Collection; import java.util.List; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Predicate; import lombok.Cleanup; import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; +import org.awaitility.Awaitility; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -124,27 +127,29 @@ public void testIsInstrumentationNameSetOnMeter() throws Exception { } @Test - public void testMetricCardinality() throws Exception { - LongCounter longCounter = meter.counterBuilder("dummyMetricCardinalityTest").build(); - - for (int i = 0; i < OpenTelemetryService.MAX_CARDINALITY_LIMIT; i++) { - longCounter.add(1, Attributes.of(AttributeKey.stringKey("attribute"), "value" + i)); - } - - Predicate hasOverflowAttribute = MetricDataMatcher.builder(). - name("dummyMetricCardinalityTest"). - dataAttribute(MetricStorage.CARDINALITY_OVERFLOW). - build(); - - Collection metricData = reader.collectAllMetrics(); - assertTrue(metricData.stream().noneMatch(hasOverflowAttribute)); - - for (int i = 0; i < OpenTelemetryService.MAX_CARDINALITY_LIMIT + 1; i++) { - longCounter.add(1, Attributes.of(AttributeKey.stringKey("attribute"), "value" + i)); + public void testMetricCardinalityIsSet() throws Exception { + var prometheusExporterPort = 9464; + @Cleanup + var ots = OpenTelemetryService.builder(). + clusterName("openTelemetryServiceCardinalityTestCluster"). + extraProperty(OpenTelemetryService.OTEL_SDK_DISABLED, "false"). + extraProperty("otel.metrics.exporter", "prometheus"). + extraProperty("otel.exporter.prometheus.port", Integer.toString(prometheusExporterPort)). + extraProperty("otel.metric.export.interval", "100"). + build(); + var meter = ots.getMeter("openTelemetryMetricCardinalityTest"); + var counter = meter.counterBuilder("dummyCounter").build(); + for (int i = 0; i < OpenTelemetryService.MAX_CARDINALITY_LIMIT + 100; i++) { + counter.add(1, Attributes.of(AttributeKey.stringKey("attribute"), "value" + i)); } - metricData = reader.collectAllMetrics(); - assertTrue(metricData.stream().anyMatch(hasOverflowAttribute)); + Awaitility.waitAtMost(30, TimeUnit.SECONDS).ignoreExceptions().until(() -> { + var client = new PrometheusMetricsClient("localhost", prometheusExporterPort); + var allMetrics = client.getMetrics(); + var actualMetrics = allMetrics.findByNameAndLabels("dummyCounter_total"); + var overflowMetric = allMetrics.findByNameAndLabels("dummyCounter_total", "otel_metric_overflow", "true"); + return actualMetrics.size() == OpenTelemetryService.MAX_CARDINALITY_LIMIT + 1 && overflowMetric.size() == 1; + }); } @Test diff --git a/tests/integration/pom.xml b/tests/integration/pom.xml index cf163acdfe28e..3306a3fdbeade 100644 --- a/tests/integration/pom.xml +++ b/tests/integration/pom.xml @@ -57,7 +57,7 @@ org.apache.pulsar - pulsar-broker + pulsar-broker-common ${project.version} test-jar test From 7a99baf6b93161f2cde9a9d48fc00b83a2973603 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 5 Feb 2024 22:57:54 -0800 Subject: [PATCH 104/133] Fix metric cardinality test --- .../org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java index bfce19b09591c..fbeb6a26fe96b 100644 --- a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java @@ -32,7 +32,6 @@ import io.opentelemetry.sdk.metrics.data.MetricDataType; import io.opentelemetry.sdk.metrics.export.MetricReader; import io.opentelemetry.sdk.metrics.internal.SdkMeterProviderUtil; -import io.opentelemetry.sdk.metrics.internal.state.MetricStorage; import io.opentelemetry.sdk.testing.exporter.InMemoryMetricReader; import java.util.Collection; import java.util.List; From ef6f14861736aab582350371b8747d8b31cbb20d Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 6 Feb 2024 11:48:58 -0800 Subject: [PATCH 105/133] Add service name and version attributes --- .../stats/PulsarBrokerOpenTelemetry.java | 8 ++- .../stats/PulsarWorkerOpenTelemetry.java | 9 ++- .../common/stats/OpenTelemetryService.java | 23 +++++-- .../common/stats/MetricDataMatcher.java | 11 ++++ .../stats/OpenTelemetryServiceTest.java | 25 ++++++++ .../proxy/stats/PulsarProxyOpenTelemetry.java | 8 ++- tests/integration/pom.xml | 7 ++- .../metrics/OpenTelemetrySanityTest.java | 63 ++++++++----------- 8 files changed, 107 insertions(+), 47 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java index 730ff1c870c1f..209518685a5c9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java @@ -21,11 +21,13 @@ import io.opentelemetry.api.metrics.Meter; import java.io.Closeable; import lombok.Getter; +import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.common.stats.OpenTelemetryService; public class PulsarBrokerOpenTelemetry implements Closeable { + public static final String SERVICE_NAME = "pulsar-broker"; private static final String INSTRUMENTATION_SCOPE_NAME = "org.apache.pulsar.broker"; private final OpenTelemetryService openTelemetryService; @@ -33,7 +35,11 @@ public class PulsarBrokerOpenTelemetry implements Closeable { private final Meter meter; public PulsarBrokerOpenTelemetry(ServiceConfiguration config) { - openTelemetryService = OpenTelemetryService.builder().clusterName(config.getClusterName()).build(); + openTelemetryService = OpenTelemetryService.builder() + .clusterName(config.getClusterName()) + .serviceName(SERVICE_NAME) + .serviceVersion(PulsarVersion.getVersion()) + .build(); meter = openTelemetryService.getMeter(INSTRUMENTATION_SCOPE_NAME); } diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/PulsarWorkerOpenTelemetry.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/PulsarWorkerOpenTelemetry.java index cdda2ce10676c..a29fb3df92153 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/PulsarWorkerOpenTelemetry.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/PulsarWorkerOpenTelemetry.java @@ -21,11 +21,13 @@ import io.opentelemetry.api.metrics.Meter; import java.io.Closeable; import lombok.Getter; +import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.common.stats.OpenTelemetryService; import org.apache.pulsar.functions.worker.WorkerConfig; public class PulsarWorkerOpenTelemetry implements Closeable { + public static final String SERVICE_NAME = "pulsar-function-worker"; private static final String INSTRUMENTATION_SCOPE_NAME = "org.apache.pulsar.function_worker"; private final OpenTelemetryService openTelemetryService; @@ -33,8 +35,11 @@ public class PulsarWorkerOpenTelemetry implements Closeable { private final Meter meter; public PulsarWorkerOpenTelemetry(WorkerConfig workerConfig) { - openTelemetryService = - OpenTelemetryService.builder().clusterName(workerConfig.getPulsarFunctionsCluster()).build(); + openTelemetryService = OpenTelemetryService.builder() + .clusterName(workerConfig.getPulsarFunctionsCluster()) + .serviceName(SERVICE_NAME) + .serviceVersion(PulsarVersion.getVersion()) + .build(); meter = openTelemetryService.getMeter(INSTRUMENTATION_SCOPE_NAME); } diff --git a/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index 2e76f5bba96c6..2e6685f67065a 100644 --- a/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -29,6 +29,7 @@ import java.io.Closeable; import java.util.HashMap; import java.util.Map; +import java.util.Objects; import lombok.Builder; import lombok.Singular; import org.apache.commons.lang3.StringUtils; @@ -40,6 +41,8 @@ public class OpenTelemetryService implements Closeable { public static final AttributeKey CLUSTER_ATTRIBUTE = AttributeKey.stringKey("pulsar.cluster"); + public static final AttributeKey SERVICE_NAME_ATTRIBUTE = AttributeKey.stringKey("service.name"); + public static final AttributeKey SERVICE_VERSION_ATTRIBUTE = AttributeKey.stringKey("service.version"); public static final AttributeKey CONSUMER_ID_ATTRIBUTE = AttributeKey.longKey("pulsar.consumer.id"); public static final AttributeKey CONSUMER_NAME_ATTRIBUTE = AttributeKey.stringKey("pulsar.consumer"); public static final AttributeKey NAMESPACE_ATTRIBUTE = AttributeKey.stringKey("pulsar.namespace"); @@ -60,6 +63,8 @@ public class OpenTelemetryService implements Closeable { @Builder public OpenTelemetryService(String clusterName, + String serviceName, + String serviceVersion, @Singular Map extraProperties, // Allows customizing the SDK builder; for testing purposes only. @VisibleForTesting AutoConfiguredOpenTelemetrySdkBuilder sdkBuilder) { @@ -77,11 +82,21 @@ public OpenTelemetryService(String clusterName, sdkBuilder.addResourceCustomizer( (resource, __) -> { - if (resource.getAttribute(CLUSTER_ATTRIBUTE) != null) { - // Do not override if already set (via system properties or environment variables). - return resource; + var resourceBuilder = Resource.builder(); + // Do not override attributes if already set (via system properties or environment variables). + if (resource.getAttribute(CLUSTER_ATTRIBUTE) == null) { + resourceBuilder.put(CLUSTER_ATTRIBUTE, clusterName); } - return resource.merge(Resource.builder().put(CLUSTER_ATTRIBUTE, clusterName).build()); + if (StringUtils.isNotEmpty(serviceName) + && Objects.equals(Resource.getDefault().getAttribute(SERVICE_NAME_ATTRIBUTE), + resource.getAttribute(SERVICE_NAME_ATTRIBUTE))) { + resourceBuilder.put(SERVICE_NAME_ATTRIBUTE, serviceName); + } + if (StringUtils.isNotEmpty(serviceVersion) + && resource.getAttribute(SERVICE_VERSION_ATTRIBUTE) == null) { + resourceBuilder.put(SERVICE_VERSION_ATTRIBUTE, serviceVersion); + } + return resource.merge(resourceBuilder.build()); }); openTelemetrySdk = sdkBuilder.build().getOpenTelemetrySdk(); diff --git a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/MetricDataMatcher.java b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/MetricDataMatcher.java index 788efcdd19917..7ec13417161bb 100644 --- a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/MetricDataMatcher.java +++ b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/MetricDataMatcher.java @@ -80,6 +80,17 @@ private boolean matchesDataAttributes(MetricData md) { private boolean matchesAttributes(Attributes actual, Attributes expected) { // Return true iff all attribute pairs in expected are a subset of those in actual. Allows tests to specify // just the attributes they care about, instead of exhaustively having to list all of them. + if (true) { + for (var entry : expected.asMap().entrySet()) { + var key = entry.getKey(); + var value = entry.getValue(); + var actualValue = actual.get(key); + if (!value.equals(actualValue)) { + return false; + } + } + return true; + } return expected.asMap().entrySet().stream().allMatch(e -> e.getValue().equals(actual.get(e.getKey()))); } diff --git a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java index fbeb6a26fe96b..dc5d0d7c37a56 100644 --- a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java @@ -113,6 +113,31 @@ public void testIsClusterNameSet() throws Exception { assertTrue(metricData.stream().anyMatch(predicate)); } + @Test + public void testIsServiceNameAndVersionSet() throws Exception { + @Cleanup + var reader = InMemoryMetricReader.create(); + + @Cleanup + var ots = OpenTelemetryService.builder(). + sdkBuilder(getSdkBuilder(reader)). + clusterName("testServiceNameAndVersion"). + serviceName("openTelemetryServiceTestService"). + serviceVersion("1.0.0"). + extraProperty(OpenTelemetryService.OTEL_SDK_DISABLED, "false"). + build(); + + var predicate = MetricDataMatcher.builder(). + resourceAttribute(Attributes.of( + AttributeKey.stringKey("pulsar.cluster"), "testServiceNameAndVersion", + AttributeKey.stringKey("service.name"), "openTelemetryServiceTestService", + AttributeKey.stringKey("service.version"), "1.0.0")). + build(); + + var metricData = reader.collectAllMetrics(); + assertTrue(metricData.stream().anyMatch(predicate)); + } + @Test public void testIsInstrumentationNameSetOnMeter() throws Exception { Meter meter = openTelemetryService.getMeter("testInstrumentationScope"); diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java index eb9ae4ddc61f6..042eb492cc732 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java @@ -21,11 +21,13 @@ import io.opentelemetry.api.metrics.Meter; import java.io.Closeable; import lombok.Getter; +import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.common.stats.OpenTelemetryService; import org.apache.pulsar.proxy.server.ProxyConfiguration; public class PulsarProxyOpenTelemetry implements Closeable { + public static final String SERVICE_NAME = "pulsar-proxy"; private static final String INSTRUMENTATION_SCOPE_NAME = "org.apache.pulsar.proxy"; private final OpenTelemetryService openTelemetryService; @@ -33,7 +35,11 @@ public class PulsarProxyOpenTelemetry implements Closeable { private final Meter meter; public PulsarProxyOpenTelemetry(ProxyConfiguration config) { - openTelemetryService = OpenTelemetryService.builder().clusterName(config.getClusterName()).build(); + openTelemetryService = OpenTelemetryService.builder() + .clusterName(config.getClusterName()) + .serviceName(SERVICE_NAME) + .serviceVersion(PulsarVersion.getVersion()) + .build(); meter = openTelemetryService.getMeter(INSTRUMENTATION_SCOPE_NAME); } diff --git a/tests/integration/pom.xml b/tests/integration/pom.xml index 3306a3fdbeade..5582931851bae 100644 --- a/tests/integration/pom.xml +++ b/tests/integration/pom.xml @@ -80,6 +80,12 @@ ${project.version} test + + org.apache.pulsar + pulsar-proxy + ${project.version} + test + org.apache.pulsar managed-ledger @@ -176,7 +182,6 @@ test - com.rabbitmq amqp-client diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java index 98d693f3f28cf..c6c2cb76176a5 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java @@ -26,7 +26,11 @@ import java.util.concurrent.TimeUnit; import lombok.Cleanup; import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.PulsarVersion; +import org.apache.pulsar.broker.stats.PulsarBrokerOpenTelemetry; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; +import org.apache.pulsar.functions.worker.stats.PulsarWorkerOpenTelemetry; +import org.apache.pulsar.proxy.stats.PulsarProxyOpenTelemetry; import org.apache.pulsar.tests.integration.containers.ChaosContainer; import org.apache.pulsar.tests.integration.containers.OpenTelemetryCollectorContainer; import org.apache.pulsar.tests.integration.topologies.FunctionRuntimeType; @@ -38,10 +42,7 @@ public class OpenTelemetrySanityTest { - /* - * Validate that the OpenTelemetry metrics can be exported to a remote OpenTelemetry collector. - * https://github.com/open-telemetry/opentelemetry-java/blob/main/sdk-extensions/autoconfigure/README.md#prometheus-exporter - */ + // Validate that the OpenTelemetry metrics can be exported to a remote OpenTelemetry collector. @Test(timeOut = 360_000) public void testOpenTelemetryMetricsOtlpExport() throws Exception { var clusterName = "testOpenTelemetryMetrics-" + UUID.randomUUID(); @@ -51,15 +52,9 @@ public void testOpenTelemetryMetricsOtlpExport() throws Exception { var otlpEndpointProp = Pair.of("OTEL_EXPORTER_OTLP_ENDPOINT", openTelemetryCollectorContainer.getOtlpEndpoint()); - var brokerOtelServiceName = clusterName + "-broker"; - var brokerCollectorProps = getCollectorProps(brokerOtelServiceName, exporter, otlpEndpointProp); - - var proxyOtelServiceName = clusterName + "-proxy"; - var proxyCollectorProps = getCollectorProps(proxyOtelServiceName, exporter, otlpEndpointProp); - - var functionWorkerServiceNameSuffix = PulsarTestBase.randomName(); - var functionWorkerOtelServiceName = "function-worker-" + functionWorkerServiceNameSuffix; - var functionWorkerCollectorProps = getCollectorProps(functionWorkerOtelServiceName, exporter, otlpEndpointProp); + var brokerCollectorProps = getCollectorProps(exporter, otlpEndpointProp); + var proxyCollectorProps = getCollectorProps(exporter, otlpEndpointProp); + var functionWorkerCollectorProps = getCollectorProps(exporter, otlpEndpointProp); var spec = PulsarClusterSpec.builder() .clusterName(clusterName) @@ -71,25 +66,25 @@ public void testOpenTelemetryMetricsOtlpExport() throws Exception { @Cleanup("stop") var pulsarCluster = PulsarCluster.forSpec(spec); pulsarCluster.start(); + pulsarCluster.setupFunctionWorkers(PulsarTestBase.randomName(), FunctionRuntimeType.PROCESS, 1); - pulsarCluster.setupFunctionWorkers(functionWorkerServiceNameSuffix, FunctionRuntimeType.PROCESS, 1); - // TODO: Validate cluster name is present once + // TODO: Validate cluster name and service version are present once // https://github.com/open-telemetry/opentelemetry-java/issues/6108 is solved. var metricName = "queueSize_ratio"; // Sent automatically by the OpenTelemetry SDK. Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { var metrics = getMetricsFromPrometheus( openTelemetryCollectorContainer, OpenTelemetryCollectorContainer.PROMETHEUS_EXPORTER_PORT); - return !metrics.findByNameAndLabels(metricName, "job", brokerOtelServiceName).isEmpty(); + return !metrics.findByNameAndLabels(metricName, "job", PulsarBrokerOpenTelemetry.SERVICE_NAME).isEmpty(); }); Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { var metrics = getMetricsFromPrometheus( openTelemetryCollectorContainer, OpenTelemetryCollectorContainer.PROMETHEUS_EXPORTER_PORT); - return !metrics.findByNameAndLabels(metricName, "job", proxyOtelServiceName).isEmpty(); + return !metrics.findByNameAndLabels(metricName, "job", PulsarProxyOpenTelemetry.SERVICE_NAME).isEmpty(); }); Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { var metrics = getMetricsFromPrometheus( openTelemetryCollectorContainer, OpenTelemetryCollectorContainer.PROMETHEUS_EXPORTER_PORT); - return !metrics.findByNameAndLabels(metricName, "job", functionWorkerOtelServiceName).isEmpty(); + return !metrics.findByNameAndLabels(metricName, "job", PulsarWorkerOpenTelemetry.SERVICE_NAME).isEmpty(); }); } @@ -107,16 +102,9 @@ public void testOpenTelemetryMetricsPrometheusExport() throws Exception { var prometheusExporterPortProp = Pair.of("OTEL_EXPORTER_PROMETHEUS_PORT", Integer.toString(prometheusExporterPort)); - var brokerOtelServiceName = clusterName + "-broker"; - var brokerCollectorProps = getCollectorProps(brokerOtelServiceName, exporter, prometheusExporterPortProp); - - var proxyOtelServiceName = clusterName + "-proxy"; - var proxyCollectorProps = getCollectorProps(proxyOtelServiceName, exporter, prometheusExporterPortProp); - - var functionWorkerServiceNameSuffix = PulsarTestBase.randomName(); - var functionWorkerOtelServiceName = "function-worker-" + functionWorkerServiceNameSuffix; - var functionWorkerCollectorProps = - getCollectorProps(functionWorkerOtelServiceName, exporter, prometheusExporterPortProp); + var brokerCollectorProps = getCollectorProps(exporter, prometheusExporterPortProp); + var proxyCollectorProps = getCollectorProps(exporter, prometheusExporterPortProp); + var functionWorkerCollectorProps = getCollectorProps(exporter, prometheusExporterPortProp); var spec = PulsarClusterSpec.builder() .clusterName(clusterName) @@ -130,28 +118,29 @@ public void testOpenTelemetryMetricsPrometheusExport() throws Exception { @Cleanup("stop") var pulsarCluster = PulsarCluster.forSpec(spec); pulsarCluster.start(); - - pulsarCluster.setupFunctionWorkers(functionWorkerServiceNameSuffix, FunctionRuntimeType.PROCESS, 1); - var workerContainer = pulsarCluster.getAnyWorker(); + pulsarCluster.setupFunctionWorkers(PulsarTestBase.randomName(), FunctionRuntimeType.PROCESS, 1); var metricName = "target_info"; // Sent automatically by the OpenTelemetry SDK. Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { var metrics = getMetricsFromPrometheus(pulsarCluster.getAnyBroker(), prometheusExporterPort); return !metrics.findByNameAndLabels(metricName, Pair.of("pulsar_cluster", clusterName), - Pair.of("service_name", brokerOtelServiceName)).isEmpty(); + Pair.of("service_name", PulsarBrokerOpenTelemetry.SERVICE_NAME), + Pair.of("service_version", PulsarVersion.getVersion())).isEmpty(); }); Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { var metrics = getMetricsFromPrometheus(pulsarCluster.getProxy(), prometheusExporterPort); return !metrics.findByNameAndLabels(metricName, Pair.of("pulsar_cluster", clusterName), - Pair.of("service_name", proxyOtelServiceName)).isEmpty(); + Pair.of("service_name", PulsarProxyOpenTelemetry.SERVICE_NAME), + Pair.of("service_version", PulsarVersion.getVersion())).isEmpty(); }); Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { - var metrics = getMetricsFromPrometheus(workerContainer, prometheusExporterPort); + var metrics = getMetricsFromPrometheus(pulsarCluster.getAnyWorker(), prometheusExporterPort); return !metrics.findByNameAndLabels(metricName, Pair.of("pulsar_cluster", clusterName), - Pair.of("service_name", functionWorkerOtelServiceName)).isEmpty(); + Pair.of("service_name", PulsarWorkerOpenTelemetry.SERVICE_NAME), + Pair.of("service_version", PulsarVersion.getVersion())).isEmpty(); }); } @@ -160,12 +149,10 @@ private static PrometheusMetricsClient.Metrics getMetricsFromPrometheus(ChaosCon return client.getMetrics(); } - private static Map getCollectorProps(String serviceName, String exporter, - Pair ... extraProps) { + private static Map getCollectorProps(String exporter, Pair ... extraProps) { var defaultProps = Map.of( "OTEL_SDK_DISABLED", "false", "OTEL_METRIC_EXPORT_INTERVAL", "1000", - "OTEL_SERVICE_NAME", serviceName, "OTEL_METRICS_EXPORTER", exporter ); var props = new HashMap<>(defaultProps); From fefcc89366083ad3fa7256fe1fa17253b0a0763c Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 6 Feb 2024 12:01:32 -0800 Subject: [PATCH 106/133] Rename OpenTelemetrySanityTest.getOpenTelemetryProps --- .../metrics/OpenTelemetrySanityTest.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java index c6c2cb76176a5..4520050eb1ae0 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java @@ -52,9 +52,9 @@ public void testOpenTelemetryMetricsOtlpExport() throws Exception { var otlpEndpointProp = Pair.of("OTEL_EXPORTER_OTLP_ENDPOINT", openTelemetryCollectorContainer.getOtlpEndpoint()); - var brokerCollectorProps = getCollectorProps(exporter, otlpEndpointProp); - var proxyCollectorProps = getCollectorProps(exporter, otlpEndpointProp); - var functionWorkerCollectorProps = getCollectorProps(exporter, otlpEndpointProp); + var brokerCollectorProps = getOpenTelemetryProps(exporter, otlpEndpointProp); + var proxyCollectorProps = getOpenTelemetryProps(exporter, otlpEndpointProp); + var functionWorkerCollectorProps = getOpenTelemetryProps(exporter, otlpEndpointProp); var spec = PulsarClusterSpec.builder() .clusterName(clusterName) @@ -102,9 +102,9 @@ public void testOpenTelemetryMetricsPrometheusExport() throws Exception { var prometheusExporterPortProp = Pair.of("OTEL_EXPORTER_PROMETHEUS_PORT", Integer.toString(prometheusExporterPort)); - var brokerCollectorProps = getCollectorProps(exporter, prometheusExporterPortProp); - var proxyCollectorProps = getCollectorProps(exporter, prometheusExporterPortProp); - var functionWorkerCollectorProps = getCollectorProps(exporter, prometheusExporterPortProp); + var brokerCollectorProps = getOpenTelemetryProps(exporter, prometheusExporterPortProp); + var proxyCollectorProps = getOpenTelemetryProps(exporter, prometheusExporterPortProp); + var functionWorkerCollectorProps = getOpenTelemetryProps(exporter, prometheusExporterPortProp); var spec = PulsarClusterSpec.builder() .clusterName(clusterName) @@ -149,7 +149,7 @@ private static PrometheusMetricsClient.Metrics getMetricsFromPrometheus(ChaosCon return client.getMetrics(); } - private static Map getCollectorProps(String exporter, Pair ... extraProps) { + private static Map getOpenTelemetryProps(String exporter, Pair ... extraProps) { var defaultProps = Map.of( "OTEL_SDK_DISABLED", "false", "OTEL_METRIC_EXPORT_INTERVAL", "1000", From 5e5551fb6ffabf597f9325e82f140d5ac2165123 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 6 Feb 2024 12:26:20 -0800 Subject: [PATCH 107/133] Remove extra attribute definitions until needed --- .../pulsar/common/stats/OpenTelemetryService.java | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index 2e6685f67065a..d7459f4f3bbc4 100644 --- a/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -43,18 +43,6 @@ public class OpenTelemetryService implements Closeable { public static final AttributeKey CLUSTER_ATTRIBUTE = AttributeKey.stringKey("pulsar.cluster"); public static final AttributeKey SERVICE_NAME_ATTRIBUTE = AttributeKey.stringKey("service.name"); public static final AttributeKey SERVICE_VERSION_ATTRIBUTE = AttributeKey.stringKey("service.version"); - public static final AttributeKey CONSUMER_ID_ATTRIBUTE = AttributeKey.longKey("pulsar.consumer.id"); - public static final AttributeKey CONSUMER_NAME_ATTRIBUTE = AttributeKey.stringKey("pulsar.consumer"); - public static final AttributeKey NAMESPACE_ATTRIBUTE = AttributeKey.stringKey("pulsar.namespace"); - public static final AttributeKey PRODUCER_ID_ATTRIBUTE = AttributeKey.longKey("pulsar.producer.id"); - public static final AttributeKey PRODUCER_NAME_ATTRIBUTE = AttributeKey.stringKey("pulsar.producer"); - public static final AttributeKey SUBSCRIPTION_ATTRIBUTE = AttributeKey.stringKey("pulsar.subscription"); - public static final AttributeKey SUBSCRIPTION_TYPE_ATTRIBUTE = - AttributeKey.stringKey("pulsar.subscription.type"); - public static final AttributeKey TENANT_ATTRIBUTE = AttributeKey.stringKey("pulsar.tenant"); - public static final AttributeKey TOPIC_ATTRIBUTE = AttributeKey.stringKey("pulsar.topic"); - public static final AttributeKey TOPIC_DOMAIN_ATTRIBUTE = AttributeKey.stringKey("pulsar.topic.domain"); - public static final String OTEL_SDK_DISABLED = "otel.sdk.disabled"; private static final String MAX_CARDINALITY_LIMIT_KEY = "otel.experimental.metrics.cardinality.limit"; public static final int MAX_CARDINALITY_LIMIT = 10000; From 4951a206862ba10a8223d92328fc90695e6eeb79 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 6 Feb 2024 17:52:44 -0800 Subject: [PATCH 108/133] Refactor sdkBuilder customization in OpenTelemetryService --- .../common/stats/OpenTelemetryService.java | 11 +++++--- .../stats/OpenTelemetryServiceTest.java | 25 +++++++------------ 2 files changed, 16 insertions(+), 20 deletions(-) diff --git a/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index d7459f4f3bbc4..e5b87d5e47326 100644 --- a/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -25,11 +25,13 @@ import io.opentelemetry.sdk.OpenTelemetrySdk; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdk; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; +import io.opentelemetry.sdk.autoconfigure.spi.AutoConfigurationCustomizer; import io.opentelemetry.sdk.resources.Resource; import java.io.Closeable; import java.util.HashMap; import java.util.Map; import java.util.Objects; +import java.util.function.Consumer; import lombok.Builder; import lombok.Singular; import org.apache.commons.lang3.StringUtils; @@ -53,12 +55,13 @@ public class OpenTelemetryService implements Closeable { public OpenTelemetryService(String clusterName, String serviceName, String serviceVersion, - @Singular Map extraProperties, + @VisibleForTesting @Singular Map extraProperties, // Allows customizing the SDK builder; for testing purposes only. - @VisibleForTesting AutoConfiguredOpenTelemetrySdkBuilder sdkBuilder) { + @VisibleForTesting Consumer autoConfigurationCustomizer) { checkArgument(StringUtils.isNotEmpty(clusterName), "Cluster name cannot be empty"); - if (sdkBuilder == null) { - sdkBuilder = AutoConfiguredOpenTelemetrySdk.builder(); + AutoConfiguredOpenTelemetrySdkBuilder sdkBuilder = AutoConfiguredOpenTelemetrySdk.builder(); + if (autoConfigurationCustomizer != null) { + autoConfigurationCustomizer.accept(sdkBuilder); } Map overrideProperties = new HashMap<>(); diff --git a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java index dc5d0d7c37a56..9b05fbffa4ad2 100644 --- a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java @@ -25,18 +25,17 @@ import io.opentelemetry.api.metrics.LongCounter; import io.opentelemetry.api.metrics.LongCounterBuilder; import io.opentelemetry.api.metrics.Meter; -import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdk; -import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; +import io.opentelemetry.sdk.autoconfigure.spi.AutoConfigurationCustomizer; import io.opentelemetry.sdk.common.InstrumentationScopeInfo; import io.opentelemetry.sdk.metrics.data.MetricData; import io.opentelemetry.sdk.metrics.data.MetricDataType; import io.opentelemetry.sdk.metrics.export.MetricReader; -import io.opentelemetry.sdk.metrics.internal.SdkMeterProviderUtil; import io.opentelemetry.sdk.testing.exporter.InMemoryMetricReader; import java.util.Collection; import java.util.List; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Consumer; import java.util.function.Predicate; import lombok.Cleanup; import org.apache.commons.lang3.StringUtils; @@ -56,7 +55,7 @@ public class OpenTelemetryServiceTest { public void setup() throws Exception { reader = InMemoryMetricReader.create(); openTelemetryService = OpenTelemetryService.builder(). - sdkBuilder(getSdkBuilder(reader)). + autoConfigurationCustomizer(getAutoConfigurationCustomizer(reader)). clusterName("openTelemetryServiceTestCluster"). extraProperty(OpenTelemetryService.OTEL_SDK_DISABLED, "false"). build(); @@ -70,15 +69,9 @@ public void teardown() throws Exception { } // Overrides the default sdkBuilder to include the InMemoryMetricReader for testing purposes. - private static AutoConfiguredOpenTelemetrySdkBuilder getSdkBuilder(MetricReader extraReader) { - return AutoConfiguredOpenTelemetrySdk.builder(). - addMeterProviderCustomizer((sdkMeterProviderBuilder, configProperties) -> { - SdkMeterProviderUtil.registerMetricReaderWithCardinalitySelector( - sdkMeterProviderBuilder, extraReader, - // Override the max cardinality limit for this extra reader. - instrumentType -> OpenTelemetryService.MAX_CARDINALITY_LIMIT + 1); - return sdkMeterProviderBuilder; - }); + private static Consumer getAutoConfigurationCustomizer(MetricReader extraReader) { + return autoConfigurationCustomizer -> autoConfigurationCustomizer.addMeterProviderCustomizer( + (sdkMeterProviderBuilder, __) -> sdkMeterProviderBuilder.registerMetricReader(extraReader)); } @Test(expectedExceptions = IllegalArgumentException.class) @@ -100,7 +93,7 @@ public void testIsClusterNameSet() throws Exception { @Cleanup OpenTelemetryService ots = OpenTelemetryService.builder(). - sdkBuilder(getSdkBuilder(reader)). + autoConfigurationCustomizer(getAutoConfigurationCustomizer(reader)). clusterName("testCluster"). extraProperty(OpenTelemetryService.OTEL_SDK_DISABLED, "false"). build(); @@ -120,7 +113,7 @@ public void testIsServiceNameAndVersionSet() throws Exception { @Cleanup var ots = OpenTelemetryService.builder(). - sdkBuilder(getSdkBuilder(reader)). + autoConfigurationCustomizer(getAutoConfigurationCustomizer(reader)). clusterName("testServiceNameAndVersion"). serviceName("openTelemetryServiceTestService"). serviceVersion("1.0.0"). @@ -200,7 +193,7 @@ public void testServiceIsDisabledByDefault() throws Exception { @Cleanup var ots = OpenTelemetryService.builder(). - sdkBuilder(getSdkBuilder(metricReader)). + autoConfigurationCustomizer(getAutoConfigurationCustomizer(metricReader)). clusterName("openTelemetryServiceTestCluster"). build(); var meter = ots.getMeter("openTelemetryServiceTestInstrument"); From e71cf0e1fc6d437c33f4be4afa5a640f7ee2b379 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 6 Feb 2024 18:29:40 -0800 Subject: [PATCH 109/133] Refactor OpenTelemetryService constructor --- .../common/stats/OpenTelemetryService.java | 11 +++-- .../stats/OpenTelemetryServiceTest.java | 40 ++++++++++++------- 2 files changed, 31 insertions(+), 20 deletions(-) diff --git a/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index e5b87d5e47326..a03ea81badcb4 100644 --- a/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -33,7 +33,6 @@ import java.util.Objects; import java.util.function.Consumer; import lombok.Builder; -import lombok.Singular; import org.apache.commons.lang3.StringUtils; /** @@ -55,21 +54,17 @@ public class OpenTelemetryService implements Closeable { public OpenTelemetryService(String clusterName, String serviceName, String serviceVersion, - @VisibleForTesting @Singular Map extraProperties, // Allows customizing the SDK builder; for testing purposes only. @VisibleForTesting Consumer autoConfigurationCustomizer) { checkArgument(StringUtils.isNotEmpty(clusterName), "Cluster name cannot be empty"); AutoConfiguredOpenTelemetrySdkBuilder sdkBuilder = AutoConfiguredOpenTelemetrySdk.builder(); - if (autoConfigurationCustomizer != null) { - autoConfigurationCustomizer.accept(sdkBuilder); - } Map overrideProperties = new HashMap<>(); overrideProperties.put(OTEL_SDK_DISABLED, "true"); // Cardinality limit includes the overflow attribute set, so we need to add 1. overrideProperties.put(MAX_CARDINALITY_LIMIT_KEY, Integer.toString(MAX_CARDINALITY_LIMIT + 1)); sdkBuilder.addPropertiesSupplier(() -> overrideProperties); - sdkBuilder.addPropertiesSupplier(() -> extraProperties); + // sdkBuilder.addPropertiesSupplier(() -> extraProperties); sdkBuilder.addResourceCustomizer( (resource, __) -> { @@ -90,6 +85,10 @@ public OpenTelemetryService(String clusterName, return resource.merge(resourceBuilder.build()); }); + if (autoConfigurationCustomizer != null) { + autoConfigurationCustomizer.accept(sdkBuilder); + } + openTelemetrySdk = sdkBuilder.build().getOpenTelemetrySdk(); } diff --git a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java index 9b05fbffa4ad2..6ab807cb79dec 100644 --- a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java @@ -31,14 +31,17 @@ import io.opentelemetry.sdk.metrics.data.MetricDataType; import io.opentelemetry.sdk.metrics.export.MetricReader; import io.opentelemetry.sdk.testing.exporter.InMemoryMetricReader; +import java.util.Arrays; import java.util.Collection; import java.util.List; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; import java.util.function.Predicate; +import java.util.stream.Collectors; import lombok.Cleanup; import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.awaitility.Awaitility; import org.testng.annotations.AfterMethod; @@ -55,9 +58,9 @@ public class OpenTelemetryServiceTest { public void setup() throws Exception { reader = InMemoryMetricReader.create(); openTelemetryService = OpenTelemetryService.builder(). - autoConfigurationCustomizer(getAutoConfigurationCustomizer(reader)). + autoConfigurationCustomizer(getAutoConfigurationCustomizer(reader, + Pair.of(OpenTelemetryService.OTEL_SDK_DISABLED, "false"))). clusterName("openTelemetryServiceTestCluster"). - extraProperty(OpenTelemetryService.OTEL_SDK_DISABLED, "false"). build(); meter = openTelemetryService.getMeter("openTelemetryServiceTestInstrument"); } @@ -68,10 +71,18 @@ public void teardown() throws Exception { reader.close(); } - // Overrides the default sdkBuilder to include the InMemoryMetricReader for testing purposes. - private static Consumer getAutoConfigurationCustomizer(MetricReader extraReader) { - return autoConfigurationCustomizer -> autoConfigurationCustomizer.addMeterProviderCustomizer( - (sdkMeterProviderBuilder, __) -> sdkMeterProviderBuilder.registerMetricReader(extraReader)); + // Customizes the SDK builder to include the MetricReader and extra properties for testing purposes. + private static Consumer getAutoConfigurationCustomizer(MetricReader extraReader, + Pair... extraProperties) { + return autoConfigurationCustomizer -> { + if (extraReader != null) { + autoConfigurationCustomizer.addMeterProviderCustomizer( + (sdkMeterProviderBuilder, __) -> sdkMeterProviderBuilder.registerMetricReader(extraReader)); + } + var extraPropertiesMap = + Arrays.stream(extraProperties).collect(Collectors.toMap(Pair::getLeft, Pair::getRight)); + autoConfigurationCustomizer.addPropertiesSupplier(() -> extraPropertiesMap); + }; } @Test(expectedExceptions = IllegalArgumentException.class) @@ -93,9 +104,9 @@ public void testIsClusterNameSet() throws Exception { @Cleanup OpenTelemetryService ots = OpenTelemetryService.builder(). - autoConfigurationCustomizer(getAutoConfigurationCustomizer(reader)). + autoConfigurationCustomizer(getAutoConfigurationCustomizer(reader, + Pair.of(OpenTelemetryService.OTEL_SDK_DISABLED, "false"))). clusterName("testCluster"). - extraProperty(OpenTelemetryService.OTEL_SDK_DISABLED, "false"). build(); Predicate predicate = MetricDataMatcher.builder(). @@ -113,11 +124,11 @@ public void testIsServiceNameAndVersionSet() throws Exception { @Cleanup var ots = OpenTelemetryService.builder(). - autoConfigurationCustomizer(getAutoConfigurationCustomizer(reader)). + autoConfigurationCustomizer(getAutoConfigurationCustomizer(reader, + Pair.of(OpenTelemetryService.OTEL_SDK_DISABLED, "false"))). clusterName("testServiceNameAndVersion"). serviceName("openTelemetryServiceTestService"). serviceVersion("1.0.0"). - extraProperty(OpenTelemetryService.OTEL_SDK_DISABLED, "false"). build(); var predicate = MetricDataMatcher.builder(). @@ -148,11 +159,12 @@ public void testMetricCardinalityIsSet() throws Exception { var prometheusExporterPort = 9464; @Cleanup var ots = OpenTelemetryService.builder(). + autoConfigurationCustomizer(getAutoConfigurationCustomizer(null, + Pair.of(OpenTelemetryService.OTEL_SDK_DISABLED, "false"), + Pair.of("otel.metrics.exporter", "prometheus"), + Pair.of("otel.exporter.prometheus.port", Integer.toString(prometheusExporterPort)), + Pair.of("otel.metric.export.interval", "100"))). clusterName("openTelemetryServiceCardinalityTestCluster"). - extraProperty(OpenTelemetryService.OTEL_SDK_DISABLED, "false"). - extraProperty("otel.metrics.exporter", "prometheus"). - extraProperty("otel.exporter.prometheus.port", Integer.toString(prometheusExporterPort)). - extraProperty("otel.metric.export.interval", "100"). build(); var meter = ots.getMeter("openTelemetryMetricCardinalityTest"); var counter = meter.counterBuilder("dummyCounter").build(); From 1af770f352cd3e9b4e0a9f113bd7a183665edaf9 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 6 Feb 2024 22:30:23 -0800 Subject: [PATCH 110/133] Expose OpenTelemetry object from OpenTelemetryService --- .../pulsar/broker/stats/PulsarBrokerOpenTelemetry.java | 2 +- .../functions/worker/stats/PulsarWorkerOpenTelemetry.java | 2 +- .../apache/pulsar/common/stats/OpenTelemetryService.java | 7 +++---- .../pulsar/common/stats/OpenTelemetryServiceTest.java | 8 ++++---- .../pulsar/proxy/stats/PulsarProxyOpenTelemetry.java | 2 +- 5 files changed, 10 insertions(+), 11 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java index 209518685a5c9..8eebc8de11d85 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java @@ -40,7 +40,7 @@ public PulsarBrokerOpenTelemetry(ServiceConfiguration config) { .serviceName(SERVICE_NAME) .serviceVersion(PulsarVersion.getVersion()) .build(); - meter = openTelemetryService.getMeter(INSTRUMENTATION_SCOPE_NAME); + meter = openTelemetryService.getOpenTelemetry().getMeter(INSTRUMENTATION_SCOPE_NAME); } @Override diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/PulsarWorkerOpenTelemetry.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/PulsarWorkerOpenTelemetry.java index a29fb3df92153..7bb90a83f1fb6 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/PulsarWorkerOpenTelemetry.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/PulsarWorkerOpenTelemetry.java @@ -40,7 +40,7 @@ public PulsarWorkerOpenTelemetry(WorkerConfig workerConfig) { .serviceName(SERVICE_NAME) .serviceVersion(PulsarVersion.getVersion()) .build(); - meter = openTelemetryService.getMeter(INSTRUMENTATION_SCOPE_NAME); + meter = openTelemetryService.getOpenTelemetry().getMeter(INSTRUMENTATION_SCOPE_NAME); } public void close() { diff --git a/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index a03ea81badcb4..04fcbf2ade0d3 100644 --- a/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -20,8 +20,8 @@ import static com.google.common.base.Preconditions.checkArgument; import com.google.common.annotations.VisibleForTesting; +import io.opentelemetry.api.OpenTelemetry; import io.opentelemetry.api.common.AttributeKey; -import io.opentelemetry.api.metrics.Meter; import io.opentelemetry.sdk.OpenTelemetrySdk; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdk; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; @@ -64,7 +64,6 @@ public OpenTelemetryService(String clusterName, // Cardinality limit includes the overflow attribute set, so we need to add 1. overrideProperties.put(MAX_CARDINALITY_LIMIT_KEY, Integer.toString(MAX_CARDINALITY_LIMIT + 1)); sdkBuilder.addPropertiesSupplier(() -> overrideProperties); - // sdkBuilder.addPropertiesSupplier(() -> extraProperties); sdkBuilder.addResourceCustomizer( (resource, __) -> { @@ -92,8 +91,8 @@ public OpenTelemetryService(String clusterName, openTelemetrySdk = sdkBuilder.build().getOpenTelemetrySdk(); } - public Meter getMeter(String instrumentationScopeName) { - return openTelemetrySdk.getMeter(instrumentationScopeName); + public OpenTelemetry getOpenTelemetry() { + return openTelemetrySdk; } @Override diff --git a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java index 6ab807cb79dec..33e5ad383ae5e 100644 --- a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java @@ -62,7 +62,7 @@ public void setup() throws Exception { Pair.of(OpenTelemetryService.OTEL_SDK_DISABLED, "false"))). clusterName("openTelemetryServiceTestCluster"). build(); - meter = openTelemetryService.getMeter("openTelemetryServiceTestInstrument"); + meter = openTelemetryService.getOpenTelemetry().getMeter("openTelemetryServiceTestInstrument"); } @AfterMethod @@ -144,7 +144,7 @@ public void testIsServiceNameAndVersionSet() throws Exception { @Test public void testIsInstrumentationNameSetOnMeter() throws Exception { - Meter meter = openTelemetryService.getMeter("testInstrumentationScope"); + Meter meter = openTelemetryService.getOpenTelemetry().getMeter("testInstrumentationScope"); meter.counterBuilder("dummyCounter").build().add(1); MetricDataMatcher predicate = MetricDataMatcher.builder(). name("dummyCounter"). @@ -166,7 +166,7 @@ public void testMetricCardinalityIsSet() throws Exception { Pair.of("otel.metric.export.interval", "100"))). clusterName("openTelemetryServiceCardinalityTestCluster"). build(); - var meter = ots.getMeter("openTelemetryMetricCardinalityTest"); + var meter = ots.getOpenTelemetry().getMeter("openTelemetryMetricCardinalityTest"); var counter = meter.counterBuilder("dummyCounter").build(); for (int i = 0; i < OpenTelemetryService.MAX_CARDINALITY_LIMIT + 100; i++) { counter.add(1, Attributes.of(AttributeKey.stringKey("attribute"), "value" + i)); @@ -208,7 +208,7 @@ public void testServiceIsDisabledByDefault() throws Exception { autoConfigurationCustomizer(getAutoConfigurationCustomizer(metricReader)). clusterName("openTelemetryServiceTestCluster"). build(); - var meter = ots.getMeter("openTelemetryServiceTestInstrument"); + var meter = ots.getOpenTelemetry().getMeter("openTelemetryServiceTestInstrument"); var builders = List.of( meter.counterBuilder("dummyCounterA"), diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java index 042eb492cc732..4534186279802 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java @@ -40,7 +40,7 @@ public PulsarProxyOpenTelemetry(ProxyConfiguration config) { .serviceName(SERVICE_NAME) .serviceVersion(PulsarVersion.getVersion()) .build(); - meter = openTelemetryService.getMeter(INSTRUMENTATION_SCOPE_NAME); + meter = openTelemetryService.getOpenTelemetry().getMeter(INSTRUMENTATION_SCOPE_NAME); } @Override From 6eb9f16c69a6f2d8478184833c1f7982dd14e14c Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 6 Feb 2024 23:08:06 -0800 Subject: [PATCH 111/133] Fix missing test dependency --- pulsar-broker/pom.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index 2b8502ee6902a..a284695656154 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -472,6 +472,11 @@ ${project.version} + + io.rest-assured + rest-assured + test + From 3abb653f92d11ad8165e1c9e921cd1f3e175ea06 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 7 Feb 2024 08:56:31 -0800 Subject: [PATCH 112/133] Inline fields --- .../apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java | 3 +-- .../functions/worker/stats/PulsarWorkerOpenTelemetry.java | 3 +-- .../apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java | 3 +-- 3 files changed, 3 insertions(+), 6 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java index 8eebc8de11d85..7314b9d78200c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java @@ -28,7 +28,6 @@ public class PulsarBrokerOpenTelemetry implements Closeable { public static final String SERVICE_NAME = "pulsar-broker"; - private static final String INSTRUMENTATION_SCOPE_NAME = "org.apache.pulsar.broker"; private final OpenTelemetryService openTelemetryService; @Getter @@ -40,7 +39,7 @@ public PulsarBrokerOpenTelemetry(ServiceConfiguration config) { .serviceName(SERVICE_NAME) .serviceVersion(PulsarVersion.getVersion()) .build(); - meter = openTelemetryService.getOpenTelemetry().getMeter(INSTRUMENTATION_SCOPE_NAME); + meter = openTelemetryService.getOpenTelemetry().getMeter("org.apache.pulsar.broker"); } @Override diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/PulsarWorkerOpenTelemetry.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/PulsarWorkerOpenTelemetry.java index 7bb90a83f1fb6..3ada834cbe0b0 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/PulsarWorkerOpenTelemetry.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/PulsarWorkerOpenTelemetry.java @@ -28,7 +28,6 @@ public class PulsarWorkerOpenTelemetry implements Closeable { public static final String SERVICE_NAME = "pulsar-function-worker"; - private static final String INSTRUMENTATION_SCOPE_NAME = "org.apache.pulsar.function_worker"; private final OpenTelemetryService openTelemetryService; @Getter @@ -40,7 +39,7 @@ public PulsarWorkerOpenTelemetry(WorkerConfig workerConfig) { .serviceName(SERVICE_NAME) .serviceVersion(PulsarVersion.getVersion()) .build(); - meter = openTelemetryService.getOpenTelemetry().getMeter(INSTRUMENTATION_SCOPE_NAME); + meter = openTelemetryService.getOpenTelemetry().getMeter("org.apache.pulsar.function_worker"); } public void close() { diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java index 4534186279802..43cd51627cb06 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java @@ -28,7 +28,6 @@ public class PulsarProxyOpenTelemetry implements Closeable { public static final String SERVICE_NAME = "pulsar-proxy"; - private static final String INSTRUMENTATION_SCOPE_NAME = "org.apache.pulsar.proxy"; private final OpenTelemetryService openTelemetryService; @Getter @@ -40,7 +39,7 @@ public PulsarProxyOpenTelemetry(ProxyConfiguration config) { .serviceName(SERVICE_NAME) .serviceVersion(PulsarVersion.getVersion()) .build(); - meter = openTelemetryService.getOpenTelemetry().getMeter(INSTRUMENTATION_SCOPE_NAME); + meter = openTelemetryService.getOpenTelemetry().getMeter("org.apache.pulsar.proxy"); } @Override From 2b126e73b7dbcc1f93580b1a380c2affb1dbbaa9 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 7 Feb 2024 09:26:36 -0800 Subject: [PATCH 113/133] Cleanup MetricDataMatcher --- .../apache/pulsar/common/stats/MetricDataMatcher.java | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/MetricDataMatcher.java b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/MetricDataMatcher.java index 7ec13417161bb..788efcdd19917 100644 --- a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/MetricDataMatcher.java +++ b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/MetricDataMatcher.java @@ -80,17 +80,6 @@ private boolean matchesDataAttributes(MetricData md) { private boolean matchesAttributes(Attributes actual, Attributes expected) { // Return true iff all attribute pairs in expected are a subset of those in actual. Allows tests to specify // just the attributes they care about, instead of exhaustively having to list all of them. - if (true) { - for (var entry : expected.asMap().entrySet()) { - var key = entry.getKey(); - var value = entry.getValue(); - var actualValue = actual.get(key); - if (!value.equals(actualValue)) { - return false; - } - } - return true; - } return expected.asMap().entrySet().stream().allMatch(e -> e.getValue().equals(actual.get(e.getKey()))); } From a41b41d3633fcb39f1f922c543b50e40853462d5 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 7 Feb 2024 09:28:17 -0800 Subject: [PATCH 114/133] Close OpenTelemetry in PulsarWorkerService --- .../apache/pulsar/functions/worker/PulsarWorkerService.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java index b194fc3edc0d2..ee38ed31812a9 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java @@ -662,6 +662,10 @@ public void stop() { if (null != stateStoreProvider) { stateStoreProvider.close(); } + + if (null != openTelemetry) { + openTelemetry.close(); + } } } From e1ecd3b56eec7d9177a8ad6167a1e941206b41cc Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 7 Feb 2024 09:31:17 -0800 Subject: [PATCH 115/133] Cleanup property definitions in OpenTelemetryService --- .../pulsar/common/stats/OpenTelemetryService.java | 10 +++++----- .../pulsar/common/stats/OpenTelemetryServiceTest.java | 8 ++++---- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index 04fcbf2ade0d3..5d70098843259 100644 --- a/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -44,9 +44,8 @@ public class OpenTelemetryService implements Closeable { public static final AttributeKey CLUSTER_ATTRIBUTE = AttributeKey.stringKey("pulsar.cluster"); public static final AttributeKey SERVICE_NAME_ATTRIBUTE = AttributeKey.stringKey("service.name"); public static final AttributeKey SERVICE_VERSION_ATTRIBUTE = AttributeKey.stringKey("service.version"); - public static final String OTEL_SDK_DISABLED = "otel.sdk.disabled"; - private static final String MAX_CARDINALITY_LIMIT_KEY = "otel.experimental.metrics.cardinality.limit"; - public static final int MAX_CARDINALITY_LIMIT = 10000; + static final String OTEL_SDK_DISABLED_KEY = "otel.sdk.disabled"; + static final int MAX_CARDINALITY_LIMIT = 10000; private final OpenTelemetrySdk openTelemetrySdk; @@ -60,9 +59,10 @@ public OpenTelemetryService(String clusterName, AutoConfiguredOpenTelemetrySdkBuilder sdkBuilder = AutoConfiguredOpenTelemetrySdk.builder(); Map overrideProperties = new HashMap<>(); - overrideProperties.put(OTEL_SDK_DISABLED, "true"); + overrideProperties.put(OTEL_SDK_DISABLED_KEY, "true"); // Cardinality limit includes the overflow attribute set, so we need to add 1. - overrideProperties.put(MAX_CARDINALITY_LIMIT_KEY, Integer.toString(MAX_CARDINALITY_LIMIT + 1)); + overrideProperties.put( + "otel.experimental.metrics.cardinality.limit", Integer.toString(MAX_CARDINALITY_LIMIT + 1)); sdkBuilder.addPropertiesSupplier(() -> overrideProperties); sdkBuilder.addResourceCustomizer( diff --git a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java index 33e5ad383ae5e..015f46e0a3bf2 100644 --- a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java @@ -59,7 +59,7 @@ public void setup() throws Exception { reader = InMemoryMetricReader.create(); openTelemetryService = OpenTelemetryService.builder(). autoConfigurationCustomizer(getAutoConfigurationCustomizer(reader, - Pair.of(OpenTelemetryService.OTEL_SDK_DISABLED, "false"))). + Pair.of(OpenTelemetryService.OTEL_SDK_DISABLED_KEY, "false"))). clusterName("openTelemetryServiceTestCluster"). build(); meter = openTelemetryService.getOpenTelemetry().getMeter("openTelemetryServiceTestInstrument"); @@ -105,7 +105,7 @@ public void testIsClusterNameSet() throws Exception { @Cleanup OpenTelemetryService ots = OpenTelemetryService.builder(). autoConfigurationCustomizer(getAutoConfigurationCustomizer(reader, - Pair.of(OpenTelemetryService.OTEL_SDK_DISABLED, "false"))). + Pair.of(OpenTelemetryService.OTEL_SDK_DISABLED_KEY, "false"))). clusterName("testCluster"). build(); @@ -125,7 +125,7 @@ public void testIsServiceNameAndVersionSet() throws Exception { @Cleanup var ots = OpenTelemetryService.builder(). autoConfigurationCustomizer(getAutoConfigurationCustomizer(reader, - Pair.of(OpenTelemetryService.OTEL_SDK_DISABLED, "false"))). + Pair.of(OpenTelemetryService.OTEL_SDK_DISABLED_KEY, "false"))). clusterName("testServiceNameAndVersion"). serviceName("openTelemetryServiceTestService"). serviceVersion("1.0.0"). @@ -160,7 +160,7 @@ public void testMetricCardinalityIsSet() throws Exception { @Cleanup var ots = OpenTelemetryService.builder(). autoConfigurationCustomizer(getAutoConfigurationCustomizer(null, - Pair.of(OpenTelemetryService.OTEL_SDK_DISABLED, "false"), + Pair.of(OpenTelemetryService.OTEL_SDK_DISABLED_KEY, "false"), Pair.of("otel.metrics.exporter", "prometheus"), Pair.of("otel.exporter.prometheus.port", Integer.toString(prometheusExporterPort)), Pair.of("otel.metric.export.interval", "100"))). From cd05268ed895f1a0f628e8c951ceb52a08edf819 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 7 Feb 2024 10:11:32 -0800 Subject: [PATCH 116/133] Cleanup OpenTelemetryServiceTest --- .../common/stats/OpenTelemetryService.java | 7 ++-- .../stats/OpenTelemetryServiceTest.java | 38 +++++++++---------- 2 files changed, 20 insertions(+), 25 deletions(-) diff --git a/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index 5d70098843259..bffe7e3f5e7c5 100644 --- a/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -25,7 +25,6 @@ import io.opentelemetry.sdk.OpenTelemetrySdk; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdk; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; -import io.opentelemetry.sdk.autoconfigure.spi.AutoConfigurationCustomizer; import io.opentelemetry.sdk.resources.Resource; import java.io.Closeable; import java.util.HashMap; @@ -54,7 +53,7 @@ public OpenTelemetryService(String clusterName, String serviceName, String serviceVersion, // Allows customizing the SDK builder; for testing purposes only. - @VisibleForTesting Consumer autoConfigurationCustomizer) { + @VisibleForTesting Consumer sdkBuilderConsumer) { checkArgument(StringUtils.isNotEmpty(clusterName), "Cluster name cannot be empty"); AutoConfiguredOpenTelemetrySdkBuilder sdkBuilder = AutoConfiguredOpenTelemetrySdk.builder(); @@ -84,8 +83,8 @@ public OpenTelemetryService(String clusterName, return resource.merge(resourceBuilder.build()); }); - if (autoConfigurationCustomizer != null) { - autoConfigurationCustomizer.accept(sdkBuilder); + if (sdkBuilderConsumer != null) { + sdkBuilderConsumer.accept(sdkBuilder); } openTelemetrySdk = sdkBuilder.build().getOpenTelemetrySdk(); diff --git a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java index 015f46e0a3bf2..ccfc7b2715710 100644 --- a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java @@ -25,23 +25,21 @@ import io.opentelemetry.api.metrics.LongCounter; import io.opentelemetry.api.metrics.LongCounterBuilder; import io.opentelemetry.api.metrics.Meter; -import io.opentelemetry.sdk.autoconfigure.spi.AutoConfigurationCustomizer; +import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; import io.opentelemetry.sdk.common.InstrumentationScopeInfo; import io.opentelemetry.sdk.metrics.data.MetricData; import io.opentelemetry.sdk.metrics.data.MetricDataType; import io.opentelemetry.sdk.metrics.export.MetricReader; import io.opentelemetry.sdk.testing.exporter.InMemoryMetricReader; -import java.util.Arrays; import java.util.Collection; import java.util.List; +import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; import java.util.function.Predicate; -import java.util.stream.Collectors; import lombok.Cleanup; import org.apache.commons.lang3.StringUtils; -import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.awaitility.Awaitility; import org.testng.annotations.AfterMethod; @@ -58,8 +56,8 @@ public class OpenTelemetryServiceTest { public void setup() throws Exception { reader = InMemoryMetricReader.create(); openTelemetryService = OpenTelemetryService.builder(). - autoConfigurationCustomizer(getAutoConfigurationCustomizer(reader, - Pair.of(OpenTelemetryService.OTEL_SDK_DISABLED_KEY, "false"))). + sdkBuilderConsumer(getSdkBuilderConsumer(reader, + Map.of(OpenTelemetryService.OTEL_SDK_DISABLED_KEY, "false"))). clusterName("openTelemetryServiceTestCluster"). build(); meter = openTelemetryService.getOpenTelemetry().getMeter("openTelemetryServiceTestInstrument"); @@ -72,16 +70,14 @@ public void teardown() throws Exception { } // Customizes the SDK builder to include the MetricReader and extra properties for testing purposes. - private static Consumer getAutoConfigurationCustomizer(MetricReader extraReader, - Pair... extraProperties) { + private static Consumer getSdkBuilderConsumer(MetricReader extraReader, + Map extraProperties) { return autoConfigurationCustomizer -> { if (extraReader != null) { autoConfigurationCustomizer.addMeterProviderCustomizer( (sdkMeterProviderBuilder, __) -> sdkMeterProviderBuilder.registerMetricReader(extraReader)); } - var extraPropertiesMap = - Arrays.stream(extraProperties).collect(Collectors.toMap(Pair::getLeft, Pair::getRight)); - autoConfigurationCustomizer.addPropertiesSupplier(() -> extraPropertiesMap); + autoConfigurationCustomizer.addPropertiesSupplier(() -> extraProperties); }; } @@ -104,8 +100,8 @@ public void testIsClusterNameSet() throws Exception { @Cleanup OpenTelemetryService ots = OpenTelemetryService.builder(). - autoConfigurationCustomizer(getAutoConfigurationCustomizer(reader, - Pair.of(OpenTelemetryService.OTEL_SDK_DISABLED_KEY, "false"))). + sdkBuilderConsumer(getSdkBuilderConsumer(reader, + Map.of(OpenTelemetryService.OTEL_SDK_DISABLED_KEY, "false"))). clusterName("testCluster"). build(); @@ -124,8 +120,8 @@ public void testIsServiceNameAndVersionSet() throws Exception { @Cleanup var ots = OpenTelemetryService.builder(). - autoConfigurationCustomizer(getAutoConfigurationCustomizer(reader, - Pair.of(OpenTelemetryService.OTEL_SDK_DISABLED_KEY, "false"))). + sdkBuilderConsumer(getSdkBuilderConsumer(reader, + Map.of(OpenTelemetryService.OTEL_SDK_DISABLED_KEY, "false"))). clusterName("testServiceNameAndVersion"). serviceName("openTelemetryServiceTestService"). serviceVersion("1.0.0"). @@ -159,11 +155,11 @@ public void testMetricCardinalityIsSet() throws Exception { var prometheusExporterPort = 9464; @Cleanup var ots = OpenTelemetryService.builder(). - autoConfigurationCustomizer(getAutoConfigurationCustomizer(null, - Pair.of(OpenTelemetryService.OTEL_SDK_DISABLED_KEY, "false"), - Pair.of("otel.metrics.exporter", "prometheus"), - Pair.of("otel.exporter.prometheus.port", Integer.toString(prometheusExporterPort)), - Pair.of("otel.metric.export.interval", "100"))). + sdkBuilderConsumer(getSdkBuilderConsumer(null, + Map.of(OpenTelemetryService.OTEL_SDK_DISABLED_KEY, "false", + "otel.metrics.exporter", "prometheus", + "otel.exporter.prometheus.port", Integer.toString(prometheusExporterPort), + "otel.metric.export.interval", "100"))). clusterName("openTelemetryServiceCardinalityTestCluster"). build(); var meter = ots.getOpenTelemetry().getMeter("openTelemetryMetricCardinalityTest"); @@ -205,7 +201,7 @@ public void testServiceIsDisabledByDefault() throws Exception { @Cleanup var ots = OpenTelemetryService.builder(). - autoConfigurationCustomizer(getAutoConfigurationCustomizer(metricReader)). + sdkBuilderConsumer(getSdkBuilderConsumer(metricReader, Map.of())). clusterName("openTelemetryServiceTestCluster"). build(); var meter = ots.getOpenTelemetry().getMeter("openTelemetryServiceTestInstrument"); From 8d0a747a99ecb91cd20ff05ad5f9f1b01d21fb88 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 7 Feb 2024 10:14:22 -0800 Subject: [PATCH 117/133] Cleanup OpenTelemetryServiceTest --- .../apache/pulsar/common/stats/OpenTelemetryServiceTest.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java index ccfc7b2715710..ae6bab5607dae 100644 --- a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java @@ -158,8 +158,7 @@ public void testMetricCardinalityIsSet() throws Exception { sdkBuilderConsumer(getSdkBuilderConsumer(null, Map.of(OpenTelemetryService.OTEL_SDK_DISABLED_KEY, "false", "otel.metrics.exporter", "prometheus", - "otel.exporter.prometheus.port", Integer.toString(prometheusExporterPort), - "otel.metric.export.interval", "100"))). + "otel.exporter.prometheus.port", Integer.toString(prometheusExporterPort)))). clusterName("openTelemetryServiceCardinalityTestCluster"). build(); var meter = ots.getOpenTelemetry().getMeter("openTelemetryMetricCardinalityTest"); From eaafe9285066408fe0c71efa9d7d1e3a42e59315 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 7 Feb 2024 10:19:13 -0800 Subject: [PATCH 118/133] Checkstyle fixes --- .../apache/pulsar/common/stats/OpenTelemetryServiceTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java index ae6bab5607dae..3d7b6a51fcec3 100644 --- a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java @@ -71,7 +71,7 @@ public void teardown() throws Exception { // Customizes the SDK builder to include the MetricReader and extra properties for testing purposes. private static Consumer getSdkBuilderConsumer(MetricReader extraReader, - Map extraProperties) { + Map extraProperties) { return autoConfigurationCustomizer -> { if (extraReader != null) { autoConfigurationCustomizer.addMeterProviderCustomizer( From 1286960561d4ddfaf212831947d4a10daeb17af3 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 7 Feb 2024 11:14:55 -0800 Subject: [PATCH 119/133] Use service attributes from semconv --- pom.xml | 6 ++++++ pulsar-otel-integration/pom.xml | 4 ++++ .../common/stats/OpenTelemetryService.java | 19 +++++++++---------- 3 files changed, 19 insertions(+), 10 deletions(-) diff --git a/pom.xml b/pom.xml index f4fa30b9ac8b4..277c6884acb4e 100644 --- a/pom.xml +++ b/pom.xml @@ -250,6 +250,7 @@ flexible messaging model and an intuitive client API. 2.0.6 1.34.1 1.34.1-alpha + 1.23.1-alpha 1.18.3 @@ -1450,6 +1451,11 @@ flexible messaging model and an intuitive client API. pom import + + io.opentelemetry.semconv + opentelemetry-semconv + ${opentelemetry.semconv.version} + diff --git a/pulsar-otel-integration/pom.xml b/pulsar-otel-integration/pom.xml index e8f043ff0d615..320b26e2844d1 100644 --- a/pulsar-otel-integration/pom.xml +++ b/pulsar-otel-integration/pom.xml @@ -50,6 +50,10 @@ io.opentelemetry opentelemetry-sdk-extension-autoconfigure + + io.opentelemetry.semconv + opentelemetry-semconv + com.google.guava diff --git a/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index bffe7e3f5e7c5..db07af05c1037 100644 --- a/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -26,6 +26,7 @@ import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdk; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; import io.opentelemetry.sdk.resources.Resource; +import io.opentelemetry.semconv.ResourceAttributes; import java.io.Closeable; import java.util.HashMap; import java.util.Map; @@ -41,8 +42,6 @@ public class OpenTelemetryService implements Closeable { public static final AttributeKey CLUSTER_ATTRIBUTE = AttributeKey.stringKey("pulsar.cluster"); - public static final AttributeKey SERVICE_NAME_ATTRIBUTE = AttributeKey.stringKey("service.name"); - public static final AttributeKey SERVICE_VERSION_ATTRIBUTE = AttributeKey.stringKey("service.version"); static final String OTEL_SDK_DISABLED_KEY = "otel.sdk.disabled"; static final int MAX_CARDINALITY_LIMIT = 10000; @@ -54,7 +53,7 @@ public OpenTelemetryService(String clusterName, String serviceVersion, // Allows customizing the SDK builder; for testing purposes only. @VisibleForTesting Consumer sdkBuilderConsumer) { - checkArgument(StringUtils.isNotEmpty(clusterName), "Cluster name cannot be empty"); + checkArgument(StringUtils.isNotBlank(clusterName), "Cluster name cannot be empty"); AutoConfiguredOpenTelemetrySdkBuilder sdkBuilder = AutoConfiguredOpenTelemetrySdk.builder(); Map overrideProperties = new HashMap<>(); @@ -71,14 +70,14 @@ public OpenTelemetryService(String clusterName, if (resource.getAttribute(CLUSTER_ATTRIBUTE) == null) { resourceBuilder.put(CLUSTER_ATTRIBUTE, clusterName); } - if (StringUtils.isNotEmpty(serviceName) - && Objects.equals(Resource.getDefault().getAttribute(SERVICE_NAME_ATTRIBUTE), - resource.getAttribute(SERVICE_NAME_ATTRIBUTE))) { - resourceBuilder.put(SERVICE_NAME_ATTRIBUTE, serviceName); + if (StringUtils.isNotBlank(serviceName) + && Objects.equals(Resource.getDefault().getAttribute(ResourceAttributes.SERVICE_NAME), + resource.getAttribute(ResourceAttributes.SERVICE_NAME))) { + resourceBuilder.put(ResourceAttributes.SERVICE_NAME, serviceName); } - if (StringUtils.isNotEmpty(serviceVersion) - && resource.getAttribute(SERVICE_VERSION_ATTRIBUTE) == null) { - resourceBuilder.put(SERVICE_VERSION_ATTRIBUTE, serviceVersion); + if (StringUtils.isNotBlank(serviceVersion) + && resource.getAttribute(ResourceAttributes.SERVICE_VERSION) == null) { + resourceBuilder.put(ResourceAttributes.SERVICE_VERSION, serviceVersion); } return resource.merge(resourceBuilder.build()); }); From c6df6ffb250484392186cb4a6b1b28df62c28faa Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 7 Feb 2024 14:03:43 -0800 Subject: [PATCH 120/133] Factor out attributes to interface OpenTelemetryAttributes --- .../common/stats/OpenTelemetryAttributes.java | 25 +++++++++++++++++++ .../common/stats/OpenTelemetryService.java | 6 ++--- 2 files changed, 27 insertions(+), 4 deletions(-) create mode 100644 pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryAttributes.java diff --git a/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryAttributes.java b/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryAttributes.java new file mode 100644 index 0000000000000..4b71989a87c27 --- /dev/null +++ b/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryAttributes.java @@ -0,0 +1,25 @@ +/* + * 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.common.stats; + +import io.opentelemetry.api.common.AttributeKey; + +public interface OpenTelemetryAttributes { + AttributeKey PULSAR_CLUSTER = AttributeKey.stringKey("pulsar.cluster"); +} diff --git a/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index db07af05c1037..9ef7f3c128987 100644 --- a/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -21,7 +21,6 @@ import static com.google.common.base.Preconditions.checkArgument; import com.google.common.annotations.VisibleForTesting; import io.opentelemetry.api.OpenTelemetry; -import io.opentelemetry.api.common.AttributeKey; import io.opentelemetry.sdk.OpenTelemetrySdk; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdk; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; @@ -41,7 +40,6 @@ */ public class OpenTelemetryService implements Closeable { - public static final AttributeKey CLUSTER_ATTRIBUTE = AttributeKey.stringKey("pulsar.cluster"); static final String OTEL_SDK_DISABLED_KEY = "otel.sdk.disabled"; static final int MAX_CARDINALITY_LIMIT = 10000; @@ -67,8 +65,8 @@ public OpenTelemetryService(String clusterName, (resource, __) -> { var resourceBuilder = Resource.builder(); // Do not override attributes if already set (via system properties or environment variables). - if (resource.getAttribute(CLUSTER_ATTRIBUTE) == null) { - resourceBuilder.put(CLUSTER_ATTRIBUTE, clusterName); + if (resource.getAttribute(OpenTelemetryAttributes.PULSAR_CLUSTER) == null) { + resourceBuilder.put(OpenTelemetryAttributes.PULSAR_CLUSTER, clusterName); } if (StringUtils.isNotBlank(serviceName) && Objects.equals(Resource.getDefault().getAttribute(ResourceAttributes.SERVICE_NAME), From d50d0a25e0f0dd93f05f368edcf44f40a237a684 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 7 Feb 2024 15:08:55 -0800 Subject: [PATCH 121/133] Use OpenTelemetryAssertions test library --- .../stats/OpenTelemetryServiceTest.java | 56 ++++++++----------- 1 file changed, 22 insertions(+), 34 deletions(-) diff --git a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java index 3d7b6a51fcec3..fb14d05b8fb52 100644 --- a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.common.stats; +import static io.opentelemetry.sdk.testing.assertj.OpenTelemetryAssertions.assertThat; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; import io.opentelemetry.api.common.AttributeKey; @@ -27,17 +28,14 @@ import io.opentelemetry.api.metrics.Meter; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; import io.opentelemetry.sdk.common.InstrumentationScopeInfo; -import io.opentelemetry.sdk.metrics.data.MetricData; -import io.opentelemetry.sdk.metrics.data.MetricDataType; import io.opentelemetry.sdk.metrics.export.MetricReader; import io.opentelemetry.sdk.testing.exporter.InMemoryMetricReader; -import java.util.Collection; +import io.opentelemetry.semconv.ResourceAttributes; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; -import java.util.function.Predicate; import lombok.Cleanup; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; @@ -105,12 +103,10 @@ public void testIsClusterNameSet() throws Exception { clusterName("testCluster"). build(); - Predicate predicate = MetricDataMatcher.builder(). - resourceAttribute(Attributes.of(AttributeKey.stringKey("pulsar.cluster"), "testCluster")). - build(); - - Collection metricData = reader.collectAllMetrics(); - assertTrue(metricData.stream().anyMatch(predicate)); + assertThat(reader.collectAllMetrics()) + .allSatisfy(metric -> assertThat(metric) + .hasResourceSatisfying( + resource -> resource.hasAttribute(OpenTelemetryAttributes.PULSAR_CLUSTER, "testCluster"))); } @Test @@ -127,27 +123,21 @@ public void testIsServiceNameAndVersionSet() throws Exception { serviceVersion("1.0.0"). build(); - var predicate = MetricDataMatcher.builder(). - resourceAttribute(Attributes.of( - AttributeKey.stringKey("pulsar.cluster"), "testServiceNameAndVersion", - AttributeKey.stringKey("service.name"), "openTelemetryServiceTestService", - AttributeKey.stringKey("service.version"), "1.0.0")). - build(); - - var metricData = reader.collectAllMetrics(); - assertTrue(metricData.stream().anyMatch(predicate)); + assertThat(reader.collectAllMetrics()) + .allSatisfy(metric -> assertThat(metric) + .hasResourceSatisfying(resource -> resource + .hasAttribute(OpenTelemetryAttributes.PULSAR_CLUSTER, "testServiceNameAndVersion") + .hasAttribute(ResourceAttributes.SERVICE_NAME, "openTelemetryServiceTestService") + .hasAttribute(ResourceAttributes.SERVICE_VERSION, "1.0.0"))); } @Test public void testIsInstrumentationNameSetOnMeter() throws Exception { Meter meter = openTelemetryService.getOpenTelemetry().getMeter("testInstrumentationScope"); meter.counterBuilder("dummyCounter").build().add(1); - MetricDataMatcher predicate = MetricDataMatcher.builder(). - name("dummyCounter"). - instrumentationScopeInfo(InstrumentationScopeInfo.create("testInstrumentationScope")). - build(); - Collection metricData = reader.collectAllMetrics(); - assertTrue(metricData.stream().anyMatch(predicate)); + assertThat(reader.collectAllMetrics()) + .anySatisfy(metricData -> assertThat(metricData) + .hasInstrumentationScope(InstrumentationScopeInfo.create("testInstrumentationScope"))); } @Test @@ -182,15 +172,13 @@ public void testLongCounter() throws Exception { longCounter.add(1, Attributes.of(AttributeKey.stringKey("dummyAttr"), "dummyValue")); longCounter.add(2, Attributes.of(AttributeKey.stringKey("dummyAttr"), "dummyValue")); - Predicate predicate = MetricDataMatcher.builder(). - name("dummyLongCounter"). - dataAttribute(Attributes.of(AttributeKey.stringKey("dummyAttr"), "dummyValue")). - type(MetricDataType.LONG_SUM). - longValue(3L). - build(); - - Collection metricData = reader.collectAllMetrics(); - assertTrue(metricData.stream().anyMatch(predicate)); + assertThat(reader.collectAllMetrics()) + .anySatisfy(metric -> assertThat(metric) + .hasName("dummyLongCounter") + .hasLongSumSatisfying(sum -> sum + .hasPointsSatisfying(point -> point + .hasAttribute(AttributeKey.stringKey("dummyAttr"), "dummyValue") + .hasValue(3)))); } @Test From 23f559517f1e7a25c22c32ce2ef973cb90f8b6b4 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 7 Feb 2024 15:09:39 -0800 Subject: [PATCH 122/133] Remove redundant MetricDataMatcher class --- .../common/stats/MetricDataMatcher.java | 97 ------------------- 1 file changed, 97 deletions(-) delete mode 100644 pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/MetricDataMatcher.java diff --git a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/MetricDataMatcher.java b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/MetricDataMatcher.java deleted file mode 100644 index 788efcdd19917..0000000000000 --- a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/MetricDataMatcher.java +++ /dev/null @@ -1,97 +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.common.stats; - -import io.opentelemetry.api.common.Attributes; -import io.opentelemetry.sdk.common.InstrumentationScopeInfo; -import io.opentelemetry.sdk.metrics.data.DoublePointData; -import io.opentelemetry.sdk.metrics.data.LongPointData; -import io.opentelemetry.sdk.metrics.data.MetricData; -import io.opentelemetry.sdk.metrics.data.MetricDataType; -import io.opentelemetry.sdk.metrics.data.PointData; -import io.opentelemetry.sdk.resources.Resource; -import java.util.Collection; -import java.util.List; -import java.util.function.Predicate; -import java.util.stream.Collectors; -import lombok.Builder; -import lombok.Singular; - -/** - * Utility class to match OpenTelemetry metric data based on a set of optional criteria. If a criterion is set, the - * input MetricData must match it in order for the predicate to evaluate true. - */ -@Builder -public class MetricDataMatcher implements Predicate { - private final String name; - private final MetricDataType type; - private final InstrumentationScopeInfo instrumentationScopeInfo; - private final Resource resource; - @Singular - private final List resourceAttributes; - @Singular - private final List dataAttributes; - @Singular - private final List longValues; - @Singular - private final List doubleValues; - - @Override - public boolean test(MetricData md) { - return (name == null || name.equals(md.getName())) - && (type == null || type.equals(md.getType())) - && (instrumentationScopeInfo == null - || instrumentationScopeInfo.equals(md.getInstrumentationScopeInfo())) - && (resource == null || resource.equals(md.getResource())) - && matchesResourceAttributes(md) - && (dataAttributes == null || matchesDataAttributes(md)) - && matchesLongValues(md) - && matchesDoubleValues(md); - } - - private boolean matchesResourceAttributes(MetricData md) { - Attributes actual = md.getResource().getAttributes(); - return resourceAttributes.stream().allMatch(expected -> matchesAttributes(actual, expected)); - } - - private boolean matchesDataAttributes(MetricData md) { - Collection actuals = - md.getData().getPoints().stream().map(PointData::getAttributes).collect(Collectors.toSet()); - return dataAttributes.stream(). - allMatch(expected -> actuals.stream().anyMatch(actual -> matchesAttributes(actual, expected))); - } - - private boolean matchesAttributes(Attributes actual, Attributes expected) { - // Return true iff all attribute pairs in expected are a subset of those in actual. Allows tests to specify - // just the attributes they care about, instead of exhaustively having to list all of them. - return expected.asMap().entrySet().stream().allMatch(e -> e.getValue().equals(actual.get(e.getKey()))); - } - - private boolean matchesLongValues(MetricData md) { - Collection actualData = - md.getLongSumData().getPoints().stream().map(LongPointData::getValue).collect(Collectors.toSet()); - return actualData.containsAll(longValues); - } - - private boolean matchesDoubleValues(MetricData md) { - Collection actualData = - md.getDoubleSumData().getPoints().stream().map(DoublePointData::getValue).collect(Collectors.toSet()); - return actualData.containsAll(doubleValues); - } -} From 122c72b02d7d0a745b4625d11c4892433783bafe Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 7 Feb 2024 15:24:31 -0800 Subject: [PATCH 123/133] Cleanup --- .../common/stats/OpenTelemetryService.java | 14 ++++---- .../stats/OpenTelemetryServiceTest.java | 32 +++++++++---------- 2 files changed, 22 insertions(+), 24 deletions(-) diff --git a/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java index 9ef7f3c128987..ab0ad6d2cebcb 100644 --- a/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java @@ -27,7 +27,6 @@ import io.opentelemetry.sdk.resources.Resource; import io.opentelemetry.semconv.ResourceAttributes; import java.io.Closeable; -import java.util.HashMap; import java.util.Map; import java.util.Objects; import java.util.function.Consumer; @@ -52,14 +51,13 @@ public OpenTelemetryService(String clusterName, // Allows customizing the SDK builder; for testing purposes only. @VisibleForTesting Consumer sdkBuilderConsumer) { checkArgument(StringUtils.isNotBlank(clusterName), "Cluster name cannot be empty"); - AutoConfiguredOpenTelemetrySdkBuilder sdkBuilder = AutoConfiguredOpenTelemetrySdk.builder(); + var sdkBuilder = AutoConfiguredOpenTelemetrySdk.builder(); - Map overrideProperties = new HashMap<>(); - overrideProperties.put(OTEL_SDK_DISABLED_KEY, "true"); - // Cardinality limit includes the overflow attribute set, so we need to add 1. - overrideProperties.put( - "otel.experimental.metrics.cardinality.limit", Integer.toString(MAX_CARDINALITY_LIMIT + 1)); - sdkBuilder.addPropertiesSupplier(() -> overrideProperties); + sdkBuilder.addPropertiesSupplier(() -> Map.of( + OTEL_SDK_DISABLED_KEY, "true", + // Cardinality limit includes the overflow attribute set, so we need to add 1. + "otel.experimental.metrics.cardinality.limit", Integer.toString(MAX_CARDINALITY_LIMIT + 1) + )); sdkBuilder.addResourceCustomizer( (resource, __) -> { diff --git a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java index fb14d05b8fb52..ab03fd40bd4a0 100644 --- a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java @@ -23,7 +23,6 @@ import static org.testng.Assert.assertTrue; import io.opentelemetry.api.common.AttributeKey; 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.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; @@ -80,24 +79,24 @@ private static Consumer getSdkBuilderCons } @Test(expectedExceptions = IllegalArgumentException.class) - public void testClusterNameCannotBeNull() throws Exception { + public void testClusterNameCannotBeNull() { @Cleanup - OpenTelemetryService ots = OpenTelemetryService.builder().build(); + var ots = OpenTelemetryService.builder().build(); } @Test(expectedExceptions = IllegalArgumentException.class) - public void testClusterNameCannotBeEmpty() throws Exception { + public void testClusterNameCannotBeEmpty() { @Cleanup - OpenTelemetryService ots = OpenTelemetryService.builder().clusterName(StringUtils.EMPTY).build(); + var ots = OpenTelemetryService.builder().clusterName(StringUtils.EMPTY).build(); } @Test public void testIsClusterNameSet() throws Exception { @Cleanup - InMemoryMetricReader reader = InMemoryMetricReader.create(); + var reader = InMemoryMetricReader.create(); @Cleanup - OpenTelemetryService ots = OpenTelemetryService.builder(). + var ots = OpenTelemetryService.builder(). sdkBuilderConsumer(getSdkBuilderConsumer(reader, Map.of(OpenTelemetryService.OTEL_SDK_DISABLED_KEY, "false"))). clusterName("testCluster"). @@ -132,16 +131,16 @@ public void testIsServiceNameAndVersionSet() throws Exception { } @Test - public void testIsInstrumentationNameSetOnMeter() throws Exception { - Meter meter = openTelemetryService.getOpenTelemetry().getMeter("testInstrumentationScope"); + public void testIsInstrumentationNameSetOnMeter() { + var meter = openTelemetryService.getOpenTelemetry().getMeter("testInstrumentationScope"); meter.counterBuilder("dummyCounter").build().add(1); assertThat(reader.collectAllMetrics()) .anySatisfy(metricData -> assertThat(metricData) - .hasInstrumentationScope(InstrumentationScopeInfo.create("testInstrumentationScope"))); + .hasInstrumentationScope(InstrumentationScopeInfo.create("testInstrumentationScope"))); } @Test - public void testMetricCardinalityIsSet() throws Exception { + public void testMetricCardinalityIsSet() { var prometheusExporterPort = 9464; @Cleanup var ots = OpenTelemetryService.builder(). @@ -167,17 +166,18 @@ public void testMetricCardinalityIsSet() throws Exception { } @Test - public void testLongCounter() throws Exception { - LongCounter longCounter = meter.counterBuilder("dummyLongCounter").build(); - longCounter.add(1, Attributes.of(AttributeKey.stringKey("dummyAttr"), "dummyValue")); - longCounter.add(2, Attributes.of(AttributeKey.stringKey("dummyAttr"), "dummyValue")); + public void testLongCounter() { + var longCounter = meter.counterBuilder("dummyLongCounter").build(); + var attributes = Attributes.of(AttributeKey.stringKey("dummyAttr"), "dummyValue"); + longCounter.add(1, attributes); + longCounter.add(2, attributes); assertThat(reader.collectAllMetrics()) .anySatisfy(metric -> assertThat(metric) .hasName("dummyLongCounter") .hasLongSumSatisfying(sum -> sum .hasPointsSatisfying(point -> point - .hasAttribute(AttributeKey.stringKey("dummyAttr"), "dummyValue") + .hasAttributes(attributes) .hasValue(3)))); } From fa95f9fcd90191a5816ae19be3c9a70bb92aac3d Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 7 Feb 2024 15:30:22 -0800 Subject: [PATCH 124/133] Add opentelemetry-semconv to LICENSE --- distribution/server/src/assemble/LICENSE.bin.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index f8e24671042bc..5d8fa137401f6 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -525,6 +525,7 @@ The Apache Software License, Version 2.0 - io.opentelemetry-opentelemetry-sdk-logs-1.34.1.jar - io.opentelemetry-opentelemetry-sdk-metrics-1.34.1.jar - io.opentelemetry-opentelemetry-sdk-trace-1.34.1.jar + - io.opentelemetry.semconv-opentelemetry-semconv-1.23.1-alpha.jar BSD 3-clause "New" or "Revised" License * Google auth library From ac8a2e946c12b5ee24075225d0387c617f82a582 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 8 Feb 2024 01:17:52 -0800 Subject: [PATCH 125/133] Revert pulsar-broker minor pom change --- pulsar-broker/pom.xml | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index a284695656154..a0232627a2c98 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -444,6 +444,12 @@ javax.activation + + io.rest-assured + rest-assured + test + + @@ -472,11 +478,6 @@ ${project.version} - - io.rest-assured - rest-assured - test - From 2c2dd46e34590acf180f618dbb640c8d0f59671e Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 8 Feb 2024 12:16:28 -0800 Subject: [PATCH 126/133] Enable coverage for metrics integration tests --- .github/workflows/pulsar-ci.yaml | 1 - 1 file changed, 1 deletion(-) diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index 618360c333b16..effeab90beb95 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -591,7 +591,6 @@ jobs: - name: Metrics group: METRICS - no_coverage: true steps: - name: checkout From baa4fa9c1b6e52f79f141d71c346daabc6cfe082 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 8 Feb 2024 12:35:23 -0800 Subject: [PATCH 127/133] Rename module to pulsar-opentelemetry --- pom.xml | 4 ++-- pulsar-broker/pom.xml | 2 +- pulsar-functions/worker/pom.xml | 2 +- {pulsar-otel-integration => pulsar-opentelemetry}/pom.xml | 2 +- .../apache/pulsar/common/stats/OpenTelemetryAttributes.java | 0 .../org/apache/pulsar/common/stats/OpenTelemetryService.java | 0 .../java/org/apache/pulsar/common/stats/package-info.java | 0 .../apache/pulsar/common/stats/OpenTelemetryServiceTest.java | 0 pulsar-proxy/pom.xml | 2 +- 9 files changed, 6 insertions(+), 6 deletions(-) rename {pulsar-otel-integration => pulsar-opentelemetry}/pom.xml (98%) rename {pulsar-otel-integration => pulsar-opentelemetry}/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryAttributes.java (100%) rename {pulsar-otel-integration => pulsar-opentelemetry}/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java (100%) rename {pulsar-otel-integration => pulsar-opentelemetry}/src/main/java/org/apache/pulsar/common/stats/package-info.java (100%) rename {pulsar-otel-integration => pulsar-opentelemetry}/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java (100%) diff --git a/pom.xml b/pom.xml index 3335bdc7890db..1486c24bb18d1 100644 --- a/pom.xml +++ b/pom.xml @@ -2289,7 +2289,7 @@ flexible messaging model and an intuitive client API. pulsar-broker-auth-sasl pulsar-client-auth-sasl pulsar-config-validation - pulsar-otel-integration + pulsar-opentelemetry structured-event-log @@ -2354,7 +2354,7 @@ flexible messaging model and an intuitive client API. pulsar-broker-auth-sasl pulsar-client-auth-sasl pulsar-config-validation - pulsar-otel-integration + pulsar-opentelemetry pulsar-transaction diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index ce4fe9de453bb..18da38b43dc25 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -145,7 +145,7 @@ ${project.groupId} - pulsar-otel-integration + pulsar-opentelemetry ${project.version} diff --git a/pulsar-functions/worker/pom.xml b/pulsar-functions/worker/pom.xml index dd808c1918a3e..bb93eeb98d7e1 100644 --- a/pulsar-functions/worker/pom.xml +++ b/pulsar-functions/worker/pom.xml @@ -48,7 +48,7 @@ ${project.groupId} - pulsar-otel-integration + pulsar-opentelemetry ${project.version} diff --git a/pulsar-otel-integration/pom.xml b/pulsar-opentelemetry/pom.xml similarity index 98% rename from pulsar-otel-integration/pom.xml rename to pulsar-opentelemetry/pom.xml index 320b26e2844d1..d8d1f6952af0c 100644 --- a/pulsar-otel-integration/pom.xml +++ b/pulsar-opentelemetry/pom.xml @@ -29,7 +29,7 @@ 3.3.0-SNAPSHOT - pulsar-otel-integration + pulsar-opentelemetry OpenTelemetry Integration diff --git a/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryAttributes.java b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryAttributes.java similarity index 100% rename from pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryAttributes.java rename to pulsar-opentelemetry/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryAttributes.java diff --git a/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java similarity index 100% rename from pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java rename to pulsar-opentelemetry/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java diff --git a/pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/package-info.java b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/common/stats/package-info.java similarity index 100% rename from pulsar-otel-integration/src/main/java/org/apache/pulsar/common/stats/package-info.java rename to pulsar-opentelemetry/src/main/java/org/apache/pulsar/common/stats/package-info.java diff --git a/pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-opentelemetry/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java similarity index 100% rename from pulsar-otel-integration/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java rename to pulsar-opentelemetry/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java diff --git a/pulsar-proxy/pom.xml b/pulsar-proxy/pom.xml index dbe1a9a5eb8a7..55dfd11e40e93 100644 --- a/pulsar-proxy/pom.xml +++ b/pulsar-proxy/pom.xml @@ -51,7 +51,7 @@ ${project.groupId} - pulsar-otel-integration + pulsar-opentelemetry ${project.version} From c5e4ae2d18c00005d28f5e2c3ace6c0628000261 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 8 Feb 2024 12:46:39 -0800 Subject: [PATCH 128/133] Rename package to org.apache.pulsar.opentelemetry --- .../apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java | 2 +- .../functions/worker/stats/PulsarWorkerOpenTelemetry.java | 2 +- .../stats => opentelemetry}/OpenTelemetryAttributes.java | 2 +- .../{common/stats => opentelemetry}/OpenTelemetryService.java | 2 +- .../pulsar/{common/stats => opentelemetry}/package-info.java | 2 +- .../stats => opentelemetry}/OpenTelemetryServiceTest.java | 4 +++- .../apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java | 2 +- 7 files changed, 9 insertions(+), 7 deletions(-) rename pulsar-opentelemetry/src/main/java/org/apache/pulsar/{common/stats => opentelemetry}/OpenTelemetryAttributes.java (96%) rename pulsar-opentelemetry/src/main/java/org/apache/pulsar/{common/stats => opentelemetry}/OpenTelemetryService.java (99%) rename pulsar-opentelemetry/src/main/java/org/apache/pulsar/{common/stats => opentelemetry}/package-info.java (95%) rename pulsar-opentelemetry/src/test/java/org/apache/pulsar/{common/stats => opentelemetry}/OpenTelemetryServiceTest.java (98%) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java index 7314b9d78200c..4b76b993001c2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java @@ -23,7 +23,7 @@ import lombok.Getter; import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.common.stats.OpenTelemetryService; +import org.apache.pulsar.opentelemetry.OpenTelemetryService; public class PulsarBrokerOpenTelemetry implements Closeable { diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/PulsarWorkerOpenTelemetry.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/PulsarWorkerOpenTelemetry.java index 3ada834cbe0b0..73cc8c7872210 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/PulsarWorkerOpenTelemetry.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/PulsarWorkerOpenTelemetry.java @@ -22,7 +22,7 @@ import java.io.Closeable; import lombok.Getter; import org.apache.pulsar.PulsarVersion; -import org.apache.pulsar.common.stats.OpenTelemetryService; +import org.apache.pulsar.opentelemetry.OpenTelemetryService; import org.apache.pulsar.functions.worker.WorkerConfig; public class PulsarWorkerOpenTelemetry implements Closeable { diff --git a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryAttributes.java b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java similarity index 96% rename from pulsar-opentelemetry/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryAttributes.java rename to pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java index 4b71989a87c27..6a5532cf28427 100644 --- a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryAttributes.java +++ b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.common.stats; +package org.apache.pulsar.opentelemetry; import io.opentelemetry.api.common.AttributeKey; diff --git a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java similarity index 99% rename from pulsar-opentelemetry/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java rename to pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java index ab0ad6d2cebcb..5edc56a2df00f 100644 --- a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/common/stats/OpenTelemetryService.java +++ b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.common.stats; +package org.apache.pulsar.opentelemetry; import static com.google.common.base.Preconditions.checkArgument; import com.google.common.annotations.VisibleForTesting; diff --git a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/common/stats/package-info.java b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/package-info.java similarity index 95% rename from pulsar-opentelemetry/src/main/java/org/apache/pulsar/common/stats/package-info.java rename to pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/package-info.java index 2404947561c7f..c9697ee047144 100644 --- a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/common/stats/package-info.java +++ b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/package-info.java @@ -16,4 +16,4 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.common.stats; \ No newline at end of file +package org.apache.pulsar.opentelemetry; \ No newline at end of file diff --git a/pulsar-opentelemetry/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java b/pulsar-opentelemetry/src/test/java/org/apache/pulsar/opentelemetry/OpenTelemetryServiceTest.java similarity index 98% rename from pulsar-opentelemetry/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java rename to pulsar-opentelemetry/src/test/java/org/apache/pulsar/opentelemetry/OpenTelemetryServiceTest.java index ab03fd40bd4a0..7415ee5842fa9 100644 --- a/pulsar-opentelemetry/src/test/java/org/apache/pulsar/common/stats/OpenTelemetryServiceTest.java +++ b/pulsar-opentelemetry/src/test/java/org/apache/pulsar/opentelemetry/OpenTelemetryServiceTest.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.common.stats; +package org.apache.pulsar.opentelemetry; import static io.opentelemetry.sdk.testing.assertj.OpenTelemetryAssertions.assertThat; import static org.testng.Assert.assertEquals; @@ -38,6 +38,8 @@ import lombok.Cleanup; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; +import org.apache.pulsar.opentelemetry.OpenTelemetryService; import org.awaitility.Awaitility; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java index 43cd51627cb06..14bbc649466bb 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java @@ -22,7 +22,7 @@ import java.io.Closeable; import lombok.Getter; import org.apache.pulsar.PulsarVersion; -import org.apache.pulsar.common.stats.OpenTelemetryService; +import org.apache.pulsar.opentelemetry.OpenTelemetryService; import org.apache.pulsar.proxy.server.ProxyConfiguration; public class PulsarProxyOpenTelemetry implements Closeable { From 17c844d99942c29e9398309aecb1fcd69da177f2 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 8 Feb 2024 12:59:06 -0800 Subject: [PATCH 129/133] Move PulsarWorkerOpenTelemetry to org.apache.pulsar.functions.worker --- .../PulsarWorkerOpenTelemetry.java | 4 ++-- .../functions/worker/PulsarWorkerService.java | 1 - .../functions/worker/stats/package-info.java | 19 ------------------- .../metrics/OpenTelemetrySanityTest.java | 2 +- 4 files changed, 3 insertions(+), 23 deletions(-) rename pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/{stats => }/PulsarWorkerOpenTelemetry.java (94%) delete mode 100644 pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/package-info.java diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/PulsarWorkerOpenTelemetry.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerOpenTelemetry.java similarity index 94% rename from pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/PulsarWorkerOpenTelemetry.java rename to pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerOpenTelemetry.java index 73cc8c7872210..be7c15dfd85e0 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/PulsarWorkerOpenTelemetry.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerOpenTelemetry.java @@ -16,14 +16,13 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.functions.worker.stats; +package org.apache.pulsar.functions.worker; import io.opentelemetry.api.metrics.Meter; import java.io.Closeable; import lombok.Getter; import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.opentelemetry.OpenTelemetryService; -import org.apache.pulsar.functions.worker.WorkerConfig; public class PulsarWorkerOpenTelemetry implements Closeable { @@ -42,6 +41,7 @@ public PulsarWorkerOpenTelemetry(WorkerConfig workerConfig) { meter = openTelemetryService.getOpenTelemetry().getMeter("org.apache.pulsar.function_worker"); } + @Override public void close() { openTelemetryService.close(); } diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java index ee38ed31812a9..9f7d1996e0bb5 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java @@ -67,7 +67,6 @@ import org.apache.pulsar.functions.worker.service.api.Sinks; import org.apache.pulsar.functions.worker.service.api.Sources; import org.apache.pulsar.functions.worker.service.api.Workers; -import org.apache.pulsar.functions.worker.stats.PulsarWorkerOpenTelemetry; import org.apache.pulsar.metadata.api.MetadataStoreException.AlreadyExistsException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/package-info.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/package-info.java deleted file mode 100644 index 134d6189b49bb..0000000000000 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/stats/package-info.java +++ /dev/null @@ -1,19 +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.functions.worker.stats; \ No newline at end of file diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java index 4520050eb1ae0..2965606ccca78 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java @@ -29,7 +29,7 @@ import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.broker.stats.PulsarBrokerOpenTelemetry; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; -import org.apache.pulsar.functions.worker.stats.PulsarWorkerOpenTelemetry; +import org.apache.pulsar.functions.worker.PulsarWorkerOpenTelemetry; import org.apache.pulsar.proxy.stats.PulsarProxyOpenTelemetry; import org.apache.pulsar.tests.integration.containers.ChaosContainer; import org.apache.pulsar.tests.integration.containers.OpenTelemetryCollectorContainer; From 6d5dd72e18b79c2012285e4c3fc3f417e3a31bd0 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 8 Feb 2024 13:09:15 -0800 Subject: [PATCH 130/133] Checkstyle fixes --- .../apache/pulsar/opentelemetry/OpenTelemetryServiceTest.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/pulsar-opentelemetry/src/test/java/org/apache/pulsar/opentelemetry/OpenTelemetryServiceTest.java b/pulsar-opentelemetry/src/test/java/org/apache/pulsar/opentelemetry/OpenTelemetryServiceTest.java index 7415ee5842fa9..558d8b12502f5 100644 --- a/pulsar-opentelemetry/src/test/java/org/apache/pulsar/opentelemetry/OpenTelemetryServiceTest.java +++ b/pulsar-opentelemetry/src/test/java/org/apache/pulsar/opentelemetry/OpenTelemetryServiceTest.java @@ -38,8 +38,6 @@ import lombok.Cleanup; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; -import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; -import org.apache.pulsar.opentelemetry.OpenTelemetryService; import org.awaitility.Awaitility; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; From fc5ba4f3401b3111aeb3efafea2841089163cddf Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 8 Feb 2024 15:53:07 -0800 Subject: [PATCH 131/133] Add Javadoc --- .../opentelemetry/OpenTelemetryAttributes.java | 7 +++++++ .../pulsar/opentelemetry/OpenTelemetryService.java | 14 +++++++++++++- .../apache/pulsar/opentelemetry/package-info.java | 5 +++++ 3 files changed, 25 insertions(+), 1 deletion(-) diff --git a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java index 6a5532cf28427..bdb002cb359ff 100644 --- a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java +++ b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java @@ -20,6 +20,13 @@ import io.opentelemetry.api.common.AttributeKey; +/** + * Common OpenTelemetry attributes to be used by Pulsar components. + */ public interface OpenTelemetryAttributes { + /** + * The name of the Pulsar cluster. This attribute is automatically added to all signals by + * {@link OpenTelemetryService}. + */ AttributeKey PULSAR_CLUSTER = AttributeKey.stringKey("pulsar.cluster"); } diff --git a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java index 5edc56a2df00f..5ead1ff265c83 100644 --- a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java +++ b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java @@ -44,11 +44,23 @@ public class OpenTelemetryService implements Closeable { private final OpenTelemetrySdk openTelemetrySdk; + /** + * Instantiates the OpenTelemetry SDK. All attributes are overridden by system properties or environment + * variables. + * + * @param clusterName + * The name of the Pulsar cluster. Cannot be null or blank. + * @param serviceName + * The name of the service. Optional. + * @param serviceVersion + * The version of the service. Optional. + * @param sdkBuilderConsumer + * Allows customizing the SDK builder; for testing purposes only. + */ @Builder public OpenTelemetryService(String clusterName, String serviceName, String serviceVersion, - // Allows customizing the SDK builder; for testing purposes only. @VisibleForTesting Consumer sdkBuilderConsumer) { checkArgument(StringUtils.isNotBlank(clusterName), "Cluster name cannot be empty"); var sdkBuilder = AutoConfiguredOpenTelemetrySdk.builder(); diff --git a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/package-info.java b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/package-info.java index c9697ee047144..9a7426aa0471d 100644 --- a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/package-info.java +++ b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/package-info.java @@ -16,4 +16,9 @@ * specific language governing permissions and limitations * under the License. */ + +/** + * Provides a wrapper layer for the OpenTelemetry API to be used in Pulsar. + * @since 3.3.0 + */ package org.apache.pulsar.opentelemetry; \ No newline at end of file From 7bfda76b1da83d6423b409dfc7cab20c59c76c76 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Fri, 9 Feb 2024 01:24:03 -0800 Subject: [PATCH 132/133] Add automatic resource providers --- .../server/src/assemble/LICENSE.bin.txt | 3 ++ pom.xml | 6 +++ pulsar-opentelemetry/pom.xml | 4 ++ .../OpenTelemetryServiceTest.java | 42 ++++++------------- .../containers/PulsarContainer.java | 2 + .../metrics/OpenTelemetrySanityTest.java | 11 +++-- 6 files changed, 34 insertions(+), 34 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index eb7c1287d000f..7c95811faf7de 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -527,6 +527,9 @@ The Apache Software License, Version 2.0 - io.opentelemetry-opentelemetry-sdk-logs-1.34.1.jar - io.opentelemetry-opentelemetry-sdk-metrics-1.34.1.jar - io.opentelemetry-opentelemetry-sdk-trace-1.34.1.jar + - io.opentelemetry.instrumentation-opentelemetry-instrumentation-api-1.32.1.jar + - io.opentelemetry.instrumentation-opentelemetry-instrumentation-api-semconv-1.32.1-alpha.jar + - io.opentelemetry.instrumentation-opentelemetry-resources-1.32.1-alpha.jar - io.opentelemetry.semconv-opentelemetry-semconv-1.23.1-alpha.jar BSD 3-clause "New" or "Revised" License diff --git a/pom.xml b/pom.xml index 1486c24bb18d1..52a638ac09f3e 100644 --- a/pom.xml +++ b/pom.xml @@ -251,6 +251,7 @@ flexible messaging model and an intuitive client API. 2.0.6 1.34.1 1.34.1-alpha + 1.32.1-alpha 1.23.1-alpha @@ -1464,6 +1465,11 @@ flexible messaging model and an intuitive client API. pom import + + io.opentelemetry.instrumentation + opentelemetry-resources + ${opentelemetry.instrumentation.version} + io.opentelemetry.semconv opentelemetry-semconv diff --git a/pulsar-opentelemetry/pom.xml b/pulsar-opentelemetry/pom.xml index d8d1f6952af0c..82a9658cc9d31 100644 --- a/pulsar-opentelemetry/pom.xml +++ b/pulsar-opentelemetry/pom.xml @@ -50,6 +50,10 @@ io.opentelemetry opentelemetry-sdk-extension-autoconfigure + + io.opentelemetry.instrumentation + opentelemetry-resources + io.opentelemetry.semconv opentelemetry-semconv diff --git a/pulsar-opentelemetry/src/test/java/org/apache/pulsar/opentelemetry/OpenTelemetryServiceTest.java b/pulsar-opentelemetry/src/test/java/org/apache/pulsar/opentelemetry/OpenTelemetryServiceTest.java index 558d8b12502f5..093947b02be0f 100644 --- a/pulsar-opentelemetry/src/test/java/org/apache/pulsar/opentelemetry/OpenTelemetryServiceTest.java +++ b/pulsar-opentelemetry/src/test/java/org/apache/pulsar/opentelemetry/OpenTelemetryServiceTest.java @@ -19,8 +19,7 @@ package org.apache.pulsar.opentelemetry; import static io.opentelemetry.sdk.testing.assertj.OpenTelemetryAssertions.assertThat; -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertTrue; +import static io.opentelemetry.sdk.testing.assertj.OpenTelemetryAssertions.satisfies; import io.opentelemetry.api.common.AttributeKey; import io.opentelemetry.api.common.Attributes; import io.opentelemetry.api.metrics.LongCounterBuilder; @@ -33,11 +32,12 @@ import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Consumer; import lombok.Cleanup; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; +import org.assertj.core.api.AbstractCharSequenceAssert; import org.awaitility.Awaitility; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -91,25 +91,7 @@ public void testClusterNameCannotBeEmpty() { } @Test - public void testIsClusterNameSet() throws Exception { - @Cleanup - var reader = InMemoryMetricReader.create(); - - @Cleanup - var ots = OpenTelemetryService.builder(). - sdkBuilderConsumer(getSdkBuilderConsumer(reader, - Map.of(OpenTelemetryService.OTEL_SDK_DISABLED_KEY, "false"))). - clusterName("testCluster"). - build(); - - assertThat(reader.collectAllMetrics()) - .allSatisfy(metric -> assertThat(metric) - .hasResourceSatisfying( - resource -> resource.hasAttribute(OpenTelemetryAttributes.PULSAR_CLUSTER, "testCluster"))); - } - - @Test - public void testIsServiceNameAndVersionSet() throws Exception { + public void testResourceAttributesAreSet() throws Exception { @Cleanup var reader = InMemoryMetricReader.create(); @@ -127,7 +109,8 @@ public void testIsServiceNameAndVersionSet() throws Exception { .hasResourceSatisfying(resource -> resource .hasAttribute(OpenTelemetryAttributes.PULSAR_CLUSTER, "testServiceNameAndVersion") .hasAttribute(ResourceAttributes.SERVICE_NAME, "openTelemetryServiceTestService") - .hasAttribute(ResourceAttributes.SERVICE_VERSION, "1.0.0"))); + .hasAttribute(ResourceAttributes.SERVICE_VERSION, "1.0.0") + .hasAttribute(satisfies(ResourceAttributes.HOST_NAME, AbstractCharSequenceAssert::isNotBlank)))); } @Test @@ -200,18 +183,17 @@ public void testServiceIsDisabledByDefault() throws Exception { meter.counterBuilder("dummyCounterD").setUnit("unit") ); - var callbackCount = new AtomicInteger(); + var callback = new AtomicBoolean(); // Validate that no matter how the counters are being built, they are all backed by the same underlying object. // This ensures we conserve memory when the SDK is disabled. - assertEquals(builders.stream().map(LongCounterBuilder::build).distinct().count(), 1); - assertEquals(builders.stream().map(LongCounterBuilder::buildObserver).distinct().count(), 1); - assertEquals(builders.stream().map(b -> b.buildWithCallback(__ -> callbackCount.incrementAndGet())) - .distinct().count(), 1); + assertThat(builders.stream().map(LongCounterBuilder::build).distinct()).hasSize(1); + assertThat(builders.stream().map(LongCounterBuilder::buildObserver).distinct()).hasSize(1); + assertThat(builders.stream().map(b -> b.buildWithCallback(__ -> callback.set(true))).distinct()).hasSize(1); // Validate that no metrics are being emitted at all. - assertTrue(metricReader.collectAllMetrics().isEmpty()); + assertThat(metricReader.collectAllMetrics()).isEmpty(); // Validate that the callback has not being called. - assertEquals(callbackCount.get(), 0); + assertThat(callback).isFalse(); } } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PulsarContainer.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PulsarContainer.java index 56d64ce5b2c8e..77cdc1bfd28a9 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PulsarContainer.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PulsarContainer.java @@ -26,6 +26,7 @@ import java.time.Duration; import java.util.Objects; import java.util.UUID; +import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.commons.io.FileUtils; import org.apache.pulsar.tests.integration.docker.ContainerExecResult; @@ -70,6 +71,7 @@ public abstract class PulsarContainer> exte public static final boolean PULSAR_CONTAINERS_LEAVE_RUNNING = Boolean.parseBoolean(System.getenv("PULSAR_CONTAINERS_LEAVE_RUNNING")); + @Getter protected final String hostname; private final String serviceName; private final String serviceEntryPoint; diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java index 2965606ccca78..38afc1f127d18 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java @@ -122,25 +122,28 @@ public void testOpenTelemetryMetricsPrometheusExport() throws Exception { var metricName = "target_info"; // Sent automatically by the OpenTelemetry SDK. Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { - var metrics = getMetricsFromPrometheus(pulsarCluster.getAnyBroker(), prometheusExporterPort); + var metrics = getMetricsFromPrometheus(pulsarCluster.getBroker(0), prometheusExporterPort); return !metrics.findByNameAndLabels(metricName, Pair.of("pulsar_cluster", clusterName), Pair.of("service_name", PulsarBrokerOpenTelemetry.SERVICE_NAME), - Pair.of("service_version", PulsarVersion.getVersion())).isEmpty(); + Pair.of("service_version", PulsarVersion.getVersion()), + Pair.of("host_name", pulsarCluster.getBroker(0).getHostname())).isEmpty(); }); Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { var metrics = getMetricsFromPrometheus(pulsarCluster.getProxy(), prometheusExporterPort); return !metrics.findByNameAndLabels(metricName, Pair.of("pulsar_cluster", clusterName), Pair.of("service_name", PulsarProxyOpenTelemetry.SERVICE_NAME), - Pair.of("service_version", PulsarVersion.getVersion())).isEmpty(); + Pair.of("service_version", PulsarVersion.getVersion()), + Pair.of("host_name", pulsarCluster.getProxy().getHostname())).isEmpty(); }); Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { var metrics = getMetricsFromPrometheus(pulsarCluster.getAnyWorker(), prometheusExporterPort); return !metrics.findByNameAndLabels(metricName, Pair.of("pulsar_cluster", clusterName), Pair.of("service_name", PulsarWorkerOpenTelemetry.SERVICE_NAME), - Pair.of("service_version", PulsarVersion.getVersion())).isEmpty(); + Pair.of("service_version", PulsarVersion.getVersion()), + Pair.of("host_name", pulsarCluster.getAnyWorker().getHostname())).isEmpty(); }); } From 7241867c84165563d2d997b2199dec0c30382921 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Fri, 9 Feb 2024 09:24:23 -0800 Subject: [PATCH 133/133] Disable JarServiceNameDetector in unit tests --- .../apache/pulsar/opentelemetry/OpenTelemetryServiceTest.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pulsar-opentelemetry/src/test/java/org/apache/pulsar/opentelemetry/OpenTelemetryServiceTest.java b/pulsar-opentelemetry/src/test/java/org/apache/pulsar/opentelemetry/OpenTelemetryServiceTest.java index 093947b02be0f..e5c893794a069 100644 --- a/pulsar-opentelemetry/src/test/java/org/apache/pulsar/opentelemetry/OpenTelemetryServiceTest.java +++ b/pulsar-opentelemetry/src/test/java/org/apache/pulsar/opentelemetry/OpenTelemetryServiceTest.java @@ -24,6 +24,7 @@ import io.opentelemetry.api.common.Attributes; import io.opentelemetry.api.metrics.LongCounterBuilder; import io.opentelemetry.api.metrics.Meter; +import io.opentelemetry.instrumentation.resources.JarServiceNameDetector; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; import io.opentelemetry.sdk.common.InstrumentationScopeInfo; import io.opentelemetry.sdk.metrics.export.MetricReader; @@ -98,7 +99,8 @@ public void testResourceAttributesAreSet() throws Exception { @Cleanup var ots = OpenTelemetryService.builder(). sdkBuilderConsumer(getSdkBuilderConsumer(reader, - Map.of(OpenTelemetryService.OTEL_SDK_DISABLED_KEY, "false"))). + Map.of(OpenTelemetryService.OTEL_SDK_DISABLED_KEY, "false", + "otel.java.disabled.resource.providers", JarServiceNameDetector.class.getName()))). clusterName("testServiceNameAndVersion"). serviceName("openTelemetryServiceTestService"). serviceVersion("1.0.0").