diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OpenTelemetryTracingIntegrationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OpenTelemetryTracingIntegrationTest.java index fcc0bd1776eb5..1007f7747882f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OpenTelemetryTracingIntegrationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OpenTelemetryTracingIntegrationTest.java @@ -128,33 +128,30 @@ public void testBasicProducerConsumerTracing() throws Exception { // Force flush tracer provider flushSpans(); - // Verify spans - at least one span should be created + // Verify spans - expected 2 spans to be created List spans = spanExporter.getFinishedSpanItems(); - assertTrue(spans.size() > 0, "Expected at least one span, got: " + spans.size()); + assertEquals(spans.size(), 2, "Expected 2 spans, got: " + spans.size()); - // Verify producer span if present - spans.stream() + // Verify producer span + SpanData producerSpan = spans.stream() .filter(s -> s.getKind() == SpanKind.PRODUCER) .findFirst() - .ifPresent(producerSpan -> { - assertEquals(producerSpan.getName(), "send " + topic); - assertEquals(producerSpan.getAttributes().get( - io.opentelemetry.api.common.AttributeKey.stringKey("messaging.system")), "pulsar"); - }); + .orElseThrow(); + assertEquals(producerSpan.getName(), "send " + topic); + assertEquals(producerSpan.getAttributes().get( + io.opentelemetry.api.common.AttributeKey.stringKey("messaging.system")), "pulsar"); - // Verify consumer span if present - spans.stream() + // Verify consumer span + SpanData consumerSpan = spans.stream() .filter(s -> s.getKind() == SpanKind.CONSUMER) .findFirst() - .ifPresent(consumerSpan -> { - assertEquals(consumerSpan.getName(), "process " + topic); - assertEquals(consumerSpan.getAttributes().get( - io.opentelemetry.api.common.AttributeKey.stringKey("messaging.system")), "pulsar"); - assertEquals(consumerSpan.getAttributes().get( - io.opentelemetry.api.common.AttributeKey.stringKey( - "messaging.pulsar.acknowledgment.type")), - "acknowledge"); - }); + .orElseThrow(); + assertEquals(consumerSpan.getName(), "process " + topic); + assertEquals(consumerSpan.getAttributes().get( + io.opentelemetry.api.common.AttributeKey.stringKey("messaging.system")), "pulsar"); + assertEquals(consumerSpan.getAttributes().get( + io.opentelemetry.api.common.AttributeKey.stringKey("messaging.pulsar.acknowledgment.type")), + "acknowledge"); } @Test @@ -720,14 +717,16 @@ public void testBatchMessagesTracing() throws Exception { .subscriptionName("test-sub") .subscribe(); + int numMessages = 5; + // Send batch of messages - for (int i = 0; i < 5; i++) { + for (int i = 0; i < numMessages; i++) { producer.sendAsync("message-" + i); } producer.flush(); // Receive and acknowledge all messages - for (int i = 0; i < 5; i++) { + for (int i = 0; i < numMessages; i++) { Message msg = consumer.receive(5, TimeUnit.SECONDS); assertNotNull(msg); consumer.acknowledge(msg); @@ -741,9 +740,10 @@ public void testBatchMessagesTracing() throws Exception { flushSpans(); // Verify spans for batched messages - // Note: Tracing behavior may vary for batched messages depending on when spans are created List spans = spanExporter.getFinishedSpanItems(); - assertTrue(spans.size() > 0, "Expected at least some spans for batched messages"); + int expectedNumSpans = numMessages * 2; + assertEquals(spans.size(), expectedNumSpans, + "Expected " + expectedNumSpans + " spans for batched messages, got " + spans.size()); // Verify that spans have correct attributes spans.stream() diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/tracing/OpenTelemetryProducerInterceptor.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/tracing/OpenTelemetryProducerInterceptor.java index aa3fea9615582..9d49a77562a4a 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/tracing/OpenTelemetryProducerInterceptor.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/tracing/OpenTelemetryProducerInterceptor.java @@ -77,7 +77,7 @@ public void close() { @Override public boolean eligible(Message message) { - return tracer != null && propagator != null; + return true; } @Override @@ -85,7 +85,7 @@ public Message beforeSend(Producer producer, Message message) { // Initialize tracer from producer on first call initializeIfNeeded(producer); - if (!eligible(message)) { + if (tracer == null || propagator == null) { return message; }