diff --git a/TrafficCapture/captureKafkaOffloader/build.gradle b/TrafficCapture/captureKafkaOffloader/build.gradle index 11f4e3a06..ab99a0327 100644 --- a/TrafficCapture/captureKafkaOffloader/build.gradle +++ b/TrafficCapture/captureKafkaOffloader/build.gradle @@ -9,20 +9,24 @@ repositories { } dependencies { - api 'io.netty:netty-buffer:4.1.100.Final' + implementation platform("io.opentelemetry:opentelemetry-bom:1.32.0") + implementation project(':captureOffloader') implementation project(':coreUtilities') - implementation 'org.projectlombok:lombok:1.18.26' - implementation 'com.google.protobuf:protobuf-java:3.22.2' - implementation 'org.apache.kafka:kafka-clients:3.6.0' - implementation 'software.amazon.msk:aws-msk-iam-auth:1.1.9' - implementation 'org.slf4j:slf4j-api:2.0.7' + implementation group: 'com.google.protobuf', name:'protobuf-java', version:'3.22.2' + api group:'io.netty', name:'netty-buffer', version: '4.1.100.Final' + implementation group: 'io.opentelemetry', name:'opentelemetry-api' + implementation group: 'org.projectlombok', name:'lombok', version:'1.18.26' + implementation group: 'org.apache.kafka', name:'kafka-clients', version:'3.6.0' + implementation group: 'org.slf4j', name:'slf4j-api', version:'2.0.7' + implementation group: 'software.amazon.msk', name:'aws-msk-iam-auth', version:'1.1.9' + testImplementation project(':captureProtobufs') - testImplementation 'org.mockito:mockito-core:4.6.1' - testImplementation 'org.mockito:mockito-junit-jupiter:4.6.1' testImplementation group: 'org.apache.logging.log4j', name: 'log4j-api', version: '2.20.0' testImplementation group: 'org.apache.logging.log4j', name: 'log4j-core', version: '2.20.0' testImplementation group: 'org.apache.logging.log4j', name: 'log4j-slf4j2-impl', version: '2.20.0' + testImplementation group: 'org.mockito', name: 'mockito-core', version: '4.6.1' + testImplementation group: 'org.mockito', name: 'mockito-junit-jupiter', version: '4.6.1' testImplementation group: 'org.slf4j', name: 'slf4j-api', version: '2.0.7' } diff --git a/TrafficCapture/captureKafkaOffloader/src/main/java/org/opensearch/migrations/trafficcapture/kafkaoffloader/KafkaCaptureFactory.java b/TrafficCapture/captureKafkaOffloader/src/main/java/org/opensearch/migrations/trafficcapture/kafkaoffloader/KafkaCaptureFactory.java index 756ec9739..bf502596c 100644 --- a/TrafficCapture/captureKafkaOffloader/src/main/java/org/opensearch/migrations/trafficcapture/kafkaoffloader/KafkaCaptureFactory.java +++ b/TrafficCapture/captureKafkaOffloader/src/main/java/org/opensearch/migrations/trafficcapture/kafkaoffloader/KafkaCaptureFactory.java @@ -10,14 +10,21 @@ import org.opensearch.migrations.coreutils.MetricsAttributeKey; import org.opensearch.migrations.coreutils.MetricsEvent; import org.apache.kafka.clients.producer.RecordMetadata; +import org.opensearch.migrations.tracing.SimpleMeteringClosure; +import org.opensearch.migrations.tracing.commoncontexts.IConnectionContext; import org.opensearch.migrations.trafficcapture.CodedOutputStreamHolder; import org.opensearch.migrations.trafficcapture.IChannelConnectionCaptureSerializer; import org.opensearch.migrations.trafficcapture.IConnectionCaptureFactory; import org.opensearch.migrations.trafficcapture.OrderedStreamLifecyleManager; import org.opensearch.migrations.trafficcapture.StreamChannelConnectionCaptureSerializer; import org.opensearch.migrations.coreutils.MetricsLogger; +import org.opensearch.migrations.trafficcapture.kafkaoffloader.tracing.KafkaRecordContext; +import org.opensearch.migrations.trafficcapture.tracing.ConnectionContext; +import java.io.IOException; import java.nio.ByteBuffer; +import java.time.Duration; +import java.time.Instant; import java.util.Arrays; import java.util.concurrent.CompletableFuture; @@ -25,6 +32,9 @@ @Slf4j public class KafkaCaptureFactory implements IConnectionCaptureFactory { + private static final SimpleMeteringClosure METERING_CLOSURE = new SimpleMeteringClosure("KafkaCapture"); + + private static final MetricsLogger metricsLogger = new MetricsLogger("BacksideHandler"); private static final String DEFAULT_TOPIC_NAME_FOR_TRAFFIC = "logging-traffic-topic"; @@ -51,8 +61,10 @@ public KafkaCaptureFactory(String nodeId, Producer producer, int } @Override - public IChannelConnectionCaptureSerializer createOffloader(String connectionId) { - return new StreamChannelConnectionCaptureSerializer<>(nodeId, connectionId, new StreamManager(connectionId)); + public IChannelConnectionCaptureSerializer createOffloader(IConnectionContext ctx, + String connectionId) { + return new StreamChannelConnectionCaptureSerializer<>(nodeId, connectionId, + new StreamManager(ctx, connectionId)); } @AllArgsConstructor @@ -65,12 +77,33 @@ static class CodedOutputStreamWrapper implements CodedOutputStreamHolder { } } - @AllArgsConstructor class StreamManager extends OrderedStreamLifecyleManager { + IConnectionContext telemetryContext; String connectionId; + Instant startTime; + + public StreamManager(IConnectionContext ctx, String connectionId) { + this.telemetryContext = ctx; + METERING_CLOSURE.meterIncrementEvent(telemetryContext, "offloader_created"); + METERING_CLOSURE.meterDeltaEvent(telemetryContext, "offloaders_active", 1); + + this.connectionId = connectionId; + this.startTime = Instant.now(); + } + + @Override + public void close() throws IOException { + log.atInfo().setMessage(() -> "factory.close()").log(); + METERING_CLOSURE.meterHistogramMillis(telemetryContext, "offloader_stream_lifetime", + Duration.between(startTime, Instant.now())); + METERING_CLOSURE.meterDeltaEvent(telemetryContext, "offloaders_active", -1); + METERING_CLOSURE.meterIncrementEvent(telemetryContext, "offloader_closed"); + } @Override public CodedOutputStreamWrapper createStream() { + METERING_CLOSURE.meterIncrementEvent(telemetryContext, "stream_created"); + ByteBuffer bb = ByteBuffer.allocate(bufferSize); return new CodedOutputStreamWrapper(CodedOutputStream.newInstance(bb), bb); } @@ -93,8 +126,14 @@ public CodedOutputStreamWrapper createStream() { // Used to essentially wrap Future returned by Producer to CompletableFuture var cf = new CompletableFuture(); log.debug("Sending Kafka producer record: {} for topic: {}", recordId, topicNameForTraffic); + + var flushContext = new KafkaRecordContext(telemetryContext, + METERING_CLOSURE.makeSpanContinuation("flushRecord"), + topicNameForTraffic, recordId, kafkaRecord.value().length); + METERING_CLOSURE.meterIncrementEvent(telemetryContext, "stream_flush_called"); + // Async request to Kafka cluster - producer.send(kafkaRecord, handleProducerRecordSent(cf, recordId)); + producer.send(kafkaRecord, handleProducerRecordSent(cf, recordId, flushContext)); metricsLogger.atSuccess(MetricsEvent.RECORD_SENT_TO_KAFKA) .setAttribute(MetricsAttributeKey.CHANNEL_ID, connectionId) .setAttribute(MetricsAttributeKey.TOPIC_NAME, topicNameForTraffic) @@ -108,29 +147,42 @@ public CodedOutputStreamWrapper createStream() { throw e; } } - - /** - * The default KafkaProducer comes with built-in retry and error-handling logic that suits many cases. From the - * documentation here for retry: https://kafka.apache.org/35/javadoc/org/apache/kafka/clients/producer/KafkaProducer.html - * "If the request fails, the producer can automatically retry. The retries setting defaults to Integer.MAX_VALUE, - * and it's recommended to use delivery.timeout.ms to control retry behavior, instead of retries." - *

- * Apart from this the KafkaProducer has logic for deciding whether an error is transient and should be - * retried or not retried at all: https://kafka.apache.org/35/javadoc/org/apache/kafka/common/errors/RetriableException.html - * as well as basic retry backoff - */ - private Callback handleProducerRecordSent(CompletableFuture cf, String recordId) { - return (metadata, exception) -> { - if (exception != null) { - log.error("Error sending producer record: {}", recordId, exception); - cf.completeExceptionally(exception); - } else { - log.debug("Kafka producer record: {} has finished sending for topic: {} and partition {}", - recordId, metadata.topic(), metadata.partition()); - cf.complete(metadata); - } - }; - } } + /** + * The default KafkaProducer comes with built-in retry and error-handling logic that suits many cases. From the + * documentation here for retry: https://kafka.apache.org/35/javadoc/org/apache/kafka/clients/producer/KafkaProducer.html + * "If the request fails, the producer can automatically retry. The retries setting defaults to Integer.MAX_VALUE, + * and it's recommended to use delivery.timeout.ms to control retry behavior, instead of retries." + *

+ * Apart from this the KafkaProducer has logic for deciding whether an error is transient and should be + * retried or not retried at all: https://kafka.apache.org/35/javadoc/org/apache/kafka/common/errors/RetriableException.html + * as well as basic retry backoff + */ + private Callback handleProducerRecordSent(CompletableFuture cf, String recordId, + KafkaRecordContext flushContext) { + // Keep this out of the inner class because it is more unsafe to include it within + // the inner class since the inner class has context that shouldn't be used. This keeps + // that field out of scope. + return (metadata, exception) -> { + log.atInfo().setMessage(()->"kafka completed sending a record").log(); + METERING_CLOSURE.meterHistogramMicros(flushContext, + exception==null ? "stream_flush_success_ms" : "stream_flush_failure_ms"); + METERING_CLOSURE.meterIncrementEvent(flushContext, + exception==null ? "stream_flush_success" : "stream_flush_failure"); + METERING_CLOSURE.meterIncrementEvent(flushContext, + exception==null ? "stream_flush_success_bytes" : "stream_flush_failure_bytes", + flushContext.getRecordSize()); + flushContext.currentSpan.end(); + + if (exception != null) { + log.error("Error sending producer record: {}", recordId, exception); + cf.completeExceptionally(exception); + } else { + log.debug("Kafka producer record: {} has finished sending for topic: {} and partition {}", + recordId, metadata.topic(), metadata.partition()); + cf.complete(metadata); + } + }; + } } diff --git a/TrafficCapture/captureKafkaOffloader/src/main/java/org/opensearch/migrations/trafficcapture/kafkaoffloader/tracing/KafkaRecordContext.java b/TrafficCapture/captureKafkaOffloader/src/main/java/org/opensearch/migrations/trafficcapture/kafkaoffloader/tracing/KafkaRecordContext.java new file mode 100644 index 000000000..140cfb601 --- /dev/null +++ b/TrafficCapture/captureKafkaOffloader/src/main/java/org/opensearch/migrations/trafficcapture/kafkaoffloader/tracing/KafkaRecordContext.java @@ -0,0 +1,50 @@ +package org.opensearch.migrations.trafficcapture.kafkaoffloader.tracing; + +import io.opentelemetry.api.common.AttributeKey; +import io.opentelemetry.api.common.AttributesBuilder; +import io.opentelemetry.api.trace.Span; +import lombok.AllArgsConstructor; +import lombok.Getter; +import org.opensearch.migrations.tracing.ISpanWithParentGenerator; +import org.opensearch.migrations.tracing.commoncontexts.IConnectionContext; +import org.opensearch.migrations.tracing.IWithAttributes; +import org.opensearch.migrations.tracing.IWithStartTime; + +import java.time.Instant; + +@AllArgsConstructor +public class KafkaRecordContext implements IWithAttributes, IWithStartTime { + static final AttributeKey TOPIC_ATTR = AttributeKey.stringKey("topic"); + static final AttributeKey RECORD_ID_ATTR = AttributeKey.stringKey("recordId"); + static final AttributeKey RECORD_SIZE_ATTR = AttributeKey.longKey("recordSize"); + + @Getter + public final IConnectionContext enclosingScope; + @Getter + public final Span currentSpan; + @Getter + public final Instant startTime; + @Getter + public final String topic; + @Getter + public final String recordId; + @Getter + public final int recordSize; + + public KafkaRecordContext(IConnectionContext enclosingScope, ISpanWithParentGenerator incomingSpan, + String topic, String recordId, int recordSize) { + this.enclosingScope = enclosingScope; + this.topic = topic; + this.recordId = recordId; + this.recordSize = recordSize; + this.startTime = Instant.now(); + currentSpan = incomingSpan.apply(this.getPopulatedAttributes(), enclosingScope.getCurrentSpan()); + } + + @Override + public AttributesBuilder fillAttributes(AttributesBuilder builder) { + return builder.put(TOPIC_ATTR, getTopic()) + .put(RECORD_ID_ATTR, getRecordId()) + .put(RECORD_SIZE_ATTR, getRecordSize()); + } +} diff --git a/TrafficCapture/captureKafkaOffloader/src/test/java/org/opensearch/migrations/trafficcapture/kafkaoffloader/KafkaCaptureFactoryTest.java b/TrafficCapture/captureKafkaOffloader/src/test/java/org/opensearch/migrations/trafficcapture/kafkaoffloader/KafkaCaptureFactoryTest.java index 732cc7df6..403bf9ef1 100644 --- a/TrafficCapture/captureKafkaOffloader/src/test/java/org/opensearch/migrations/trafficcapture/kafkaoffloader/KafkaCaptureFactoryTest.java +++ b/TrafficCapture/captureKafkaOffloader/src/test/java/org/opensearch/migrations/trafficcapture/kafkaoffloader/KafkaCaptureFactoryTest.java @@ -1,6 +1,7 @@ package org.opensearch.migrations.trafficcapture.kafkaoffloader; import io.netty.buffer.Unpooled; +import io.opentelemetry.api.GlobalOpenTelemetry; import lombok.extern.slf4j.Slf4j; import org.apache.kafka.clients.ApiVersions; import org.apache.kafka.clients.producer.Callback; @@ -18,7 +19,9 @@ import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.migrations.tracing.EmptyContext; import org.opensearch.migrations.trafficcapture.IChannelConnectionCaptureSerializer; +import org.opensearch.migrations.trafficcapture.tracing.ConnectionContext; import java.io.IOException; import java.nio.charset.StandardCharsets; @@ -51,7 +54,7 @@ public void testLargeRequestIsWithinKafkaMessageSizeLimit() throws IOException, MockProducer producer = new MockProducer<>(true, new StringSerializer(), new ByteArraySerializer()); KafkaCaptureFactory kafkaCaptureFactory = new KafkaCaptureFactory(TEST_NODE_ID_STRING, producer, maxAllowableMessageSize); - IChannelConnectionCaptureSerializer serializer = kafkaCaptureFactory.createOffloader(connectionId); + IChannelConnectionCaptureSerializer serializer = kafkaCaptureFactory.createOffloader(createCtx(), connectionId); StringBuilder sb = new StringBuilder(); for (int i = 0; i < 15000; i++) { @@ -73,6 +76,11 @@ public void testLargeRequestIsWithinKafkaMessageSizeLimit() throws IOException, producer.close(); } + private static ConnectionContext createCtx() { + return new ConnectionContext("test", "test", + x->GlobalOpenTelemetry.getTracer("test").spanBuilder("test").startSpan()); + } + /** * This size calculation is based off the KafkaProducer client request size validation check done when Producer * records are sent. This validation appears to be consistent for several versions now, here is a reference to @@ -97,7 +105,7 @@ private int calculateRecordSize(ProducerRecord record, String re public void testLinearOffloadingIsSuccessful() throws IOException { KafkaCaptureFactory kafkaCaptureFactory = new KafkaCaptureFactory(TEST_NODE_ID_STRING, mockProducer, 1024*1024); - IChannelConnectionCaptureSerializer offloader = kafkaCaptureFactory.createOffloader(connectionId); + IChannelConnectionCaptureSerializer offloader = kafkaCaptureFactory.createOffloader(createCtx(), connectionId); List recordSentCallbacks = new ArrayList<>(3); when(mockProducer.send(any(), any())).thenAnswer(invocation -> { diff --git a/TrafficCapture/captureOffloader/build.gradle b/TrafficCapture/captureOffloader/build.gradle index c375d39a2..d4b9848dd 100644 --- a/TrafficCapture/captureOffloader/build.gradle +++ b/TrafficCapture/captureOffloader/build.gradle @@ -20,11 +20,14 @@ sourceSets { } } dependencies { + implementation platform("io.opentelemetry:opentelemetry-bom:1.32.0") api group: 'io.netty', name: 'netty-buffer', version: '4.1.100.Final' implementation project(':captureProtobufs') - implementation "com.google.protobuf:protobuf-java:3.22.2" - implementation 'org.projectlombok:lombok:1.18.26' + implementation project(':coreUtilities') + implementation group: 'io.opentelemetry', name:'opentelemetry-api' + implementation group: 'com.google.protobuf', name: 'protobuf-java', version: '3.22.2' + implementation group: 'org.projectlombok', name: 'lombok', version: '1.18.26' implementation group: 'org.slf4j', name: 'slf4j-api', version: '2.0.7' testImplementation group: 'org.apache.logging.log4j', name: 'log4j-api', version: '2.20.0' @@ -32,7 +35,7 @@ dependencies { testImplementation group: 'org.apache.logging.log4j', name: 'log4j-slf4j2-impl', version: '2.20.0' testImplementation group: 'org.slf4j', name: 'slf4j-api', version: '2.0.7' - testFixturesImplementation "com.google.protobuf:protobuf-java:3.22.2" testFixturesImplementation project(':captureProtobufs') - + testFixturesImplementation project(':coreUtilities') + testFixturesImplementation group: 'com.google.protobuf', name: 'protobuf-java', version: '3.22.2' } diff --git a/TrafficCapture/captureOffloader/src/main/java/org/opensearch/migrations/trafficcapture/FileConnectionCaptureFactory.java b/TrafficCapture/captureOffloader/src/main/java/org/opensearch/migrations/trafficcapture/FileConnectionCaptureFactory.java index 26cee767d..a566b9e9c 100644 --- a/TrafficCapture/captureOffloader/src/main/java/org/opensearch/migrations/trafficcapture/FileConnectionCaptureFactory.java +++ b/TrafficCapture/captureOffloader/src/main/java/org/opensearch/migrations/trafficcapture/FileConnectionCaptureFactory.java @@ -3,6 +3,8 @@ import lombok.AllArgsConstructor; import lombok.Lombok; import lombok.extern.slf4j.Slf4j; +import org.opensearch.migrations.tracing.commoncontexts.IConnectionContext; +import org.opensearch.migrations.trafficcapture.tracing.ConnectionContext; import java.io.FileNotFoundException; import java.io.FileOutputStream; @@ -51,6 +53,9 @@ public FileConnectionCaptureFactory(String nodeId, String path, int bufferSize) @AllArgsConstructor class StreamManager extends OrderedStreamLifecyleManager { String connectionId; + @Override + public void close() {} + @Override public CodedOutputStreamAndByteBufferWrapper createStream() { return new CodedOutputStreamAndByteBufferWrapper(bufferSize); @@ -80,7 +85,7 @@ public CodedOutputStreamAndByteBufferWrapper createStream() { } @Override - public IChannelConnectionCaptureSerializer createOffloader(String connectionId) { - return new StreamChannelConnectionCaptureSerializer(nodeId, connectionId, new StreamManager(connectionId)); + public IChannelConnectionCaptureSerializer createOffloader(IConnectionContext ctx, String connectionId) { + return new StreamChannelConnectionCaptureSerializer<>(nodeId, connectionId, new StreamManager(connectionId)); } } diff --git a/TrafficCapture/captureOffloader/src/main/java/org/opensearch/migrations/trafficcapture/IConnectionCaptureFactory.java b/TrafficCapture/captureOffloader/src/main/java/org/opensearch/migrations/trafficcapture/IConnectionCaptureFactory.java index 9f5ec26c0..1b8def0e1 100644 --- a/TrafficCapture/captureOffloader/src/main/java/org/opensearch/migrations/trafficcapture/IConnectionCaptureFactory.java +++ b/TrafficCapture/captureOffloader/src/main/java/org/opensearch/migrations/trafficcapture/IConnectionCaptureFactory.java @@ -1,7 +1,10 @@ package org.opensearch.migrations.trafficcapture; +import org.opensearch.migrations.tracing.commoncontexts.IConnectionContext; +import org.opensearch.migrations.trafficcapture.tracing.ConnectionContext; + import java.io.IOException; public interface IConnectionCaptureFactory { - IChannelConnectionCaptureSerializer createOffloader(String connectionId) throws IOException; + IChannelConnectionCaptureSerializer createOffloader(IConnectionContext ctx, String connectionId) throws IOException; } diff --git a/TrafficCapture/captureOffloader/src/main/java/org/opensearch/migrations/trafficcapture/StreamChannelConnectionCaptureSerializer.java b/TrafficCapture/captureOffloader/src/main/java/org/opensearch/migrations/trafficcapture/StreamChannelConnectionCaptureSerializer.java index 419f06060..b6fc61a46 100644 --- a/TrafficCapture/captureOffloader/src/main/java/org/opensearch/migrations/trafficcapture/StreamChannelConnectionCaptureSerializer.java +++ b/TrafficCapture/captureOffloader/src/main/java/org/opensearch/migrations/trafficcapture/StreamChannelConnectionCaptureSerializer.java @@ -189,20 +189,23 @@ public CompletableFuture flushCommitAndResetStream(boolean isFinal) throws IO if (streamHasBeenClosed || (currentCodedOutputStreamHolderOrNull == null && !isFinal)) { return CompletableFuture.completedFuture(null); } - CodedOutputStream currentStream = getOrCreateCodedOutputStream(); - var fieldNum = isFinal ? TrafficStream.NUMBEROFTHISLASTCHUNK_FIELD_NUMBER : TrafficStream.NUMBER_FIELD_NUMBER; - // e.g. 3: 1 - currentStream.writeInt32(fieldNum, ++numFlushesSoFar); - log.trace("Flushing the current CodedOutputStream for {}.{}", connectionIdString, numFlushesSoFar); - currentStream.flush(); - assert currentStream == currentCodedOutputStreamHolderOrNull.getOutputStream() : "Expected the stream that " + - "is being finalized to be the same stream contained by currentCodedOutputStreamHolderOrNull"; - var future = streamManager.closeStream(currentCodedOutputStreamHolderOrNull, numFlushesSoFar); - currentCodedOutputStreamHolderOrNull = null; - if (isFinal) { - streamHasBeenClosed = true; + try { + CodedOutputStream currentStream = getOrCreateCodedOutputStream(); + var fieldNum = isFinal ? TrafficStream.NUMBEROFTHISLASTCHUNK_FIELD_NUMBER : TrafficStream.NUMBER_FIELD_NUMBER; + // e.g. 3: 1 + currentStream.writeInt32(fieldNum, ++numFlushesSoFar); + log.trace("Flushing the current CodedOutputStream for {}.{}", connectionIdString, numFlushesSoFar); + currentStream.flush(); + assert currentStream == currentCodedOutputStreamHolderOrNull.getOutputStream() : "Expected the stream that " + + "is being finalized to be the same stream contained by currentCodedOutputStreamHolderOrNull"; + return streamManager.closeStream(currentCodedOutputStreamHolderOrNull, numFlushesSoFar); + } finally { + currentCodedOutputStreamHolderOrNull = null; + if (isFinal) { + streamHasBeenClosed = true; + streamManager.close(); + } } - return future; } @Override @@ -233,7 +236,8 @@ public void addDisconnectEvent(Instant timestamp) throws IOException { @Override public void addCloseEvent(Instant timestamp) throws IOException { beginSubstreamObservation(timestamp, TrafficObservation.CLOSE_FIELD_NUMBER, 1); - getOrCreateCodedOutputStream().writeMessage(TrafficObservation.CLOSE_FIELD_NUMBER, CloseObservation.getDefaultInstance()); + getOrCreateCodedOutputStream().writeMessage(TrafficObservation.CLOSE_FIELD_NUMBER, + CloseObservation.getDefaultInstance()); } @Override diff --git a/TrafficCapture/captureOffloader/src/main/java/org/opensearch/migrations/trafficcapture/StreamLifecycleManager.java b/TrafficCapture/captureOffloader/src/main/java/org/opensearch/migrations/trafficcapture/StreamLifecycleManager.java index 18db43cc4..b41af74a5 100644 --- a/TrafficCapture/captureOffloader/src/main/java/org/opensearch/migrations/trafficcapture/StreamLifecycleManager.java +++ b/TrafficCapture/captureOffloader/src/main/java/org/opensearch/migrations/trafficcapture/StreamLifecycleManager.java @@ -1,9 +1,11 @@ package org.opensearch.migrations.trafficcapture; +import java.io.IOException; import java.util.concurrent.CompletableFuture; -public interface StreamLifecycleManager { +public interface StreamLifecycleManager extends AutoCloseable { CodedOutputStreamHolder createStream(); CompletableFuture closeStream(CodedOutputStreamHolder outputStreamHolder, int index); + void close() throws IOException; } diff --git a/TrafficCapture/captureOffloader/src/main/java/org/opensearch/migrations/trafficcapture/tracing/ConnectionContext.java b/TrafficCapture/captureOffloader/src/main/java/org/opensearch/migrations/trafficcapture/tracing/ConnectionContext.java new file mode 100644 index 000000000..c6d3bc5f0 --- /dev/null +++ b/TrafficCapture/captureOffloader/src/main/java/org/opensearch/migrations/trafficcapture/tracing/ConnectionContext.java @@ -0,0 +1,35 @@ +package org.opensearch.migrations.trafficcapture.tracing; + +import io.opentelemetry.api.trace.Span; +import lombok.Getter; +import org.opensearch.migrations.tracing.ISpanGenerator; +import org.opensearch.migrations.tracing.ISpanWithParentGenerator; +import org.opensearch.migrations.tracing.commoncontexts.IConnectionContext; +import org.opensearch.migrations.tracing.IWithStartTime; + +import java.time.Instant; + +public class ConnectionContext implements IConnectionContext, IWithStartTime { + @Getter + public final String connectionId; + @Getter + public final String nodeId; + @Getter + public final Span currentSpan; + @Getter + private final Instant startTime; + + public ConnectionContext(IConnectionContext oldContext, ISpanWithParentGenerator spanGenerator) { + this.connectionId = oldContext.getConnectionId(); + this.nodeId = oldContext.getNodeId(); + this.startTime = Instant.now(); + this.currentSpan = spanGenerator.apply(getPopulatedAttributes(), oldContext.getCurrentSpan()); + } + + public ConnectionContext(String connectionId, String nodeId, ISpanGenerator spanGenerator) { + this.connectionId = connectionId; + this.nodeId = nodeId; + this.currentSpan = spanGenerator.apply(getPopulatedAttributes()); + this.startTime = Instant.now(); + } +} diff --git a/TrafficCapture/captureOffloader/src/test/java/org/opensearch/migrations/trafficcapture/StreamChannelConnectionCaptureSerializerTest.java b/TrafficCapture/captureOffloader/src/test/java/org/opensearch/migrations/trafficcapture/StreamChannelConnectionCaptureSerializerTest.java index 4a25aa4d7..97def992c 100644 --- a/TrafficCapture/captureOffloader/src/test/java/org/opensearch/migrations/trafficcapture/StreamChannelConnectionCaptureSerializerTest.java +++ b/TrafficCapture/captureOffloader/src/test/java/org/opensearch/migrations/trafficcapture/StreamChannelConnectionCaptureSerializerTest.java @@ -320,6 +320,9 @@ class StreamManager extends OrderedStreamLifecyleManager { int bufferSize; ConcurrentLinkedQueue outputBuffers; + @Override + public void close() {} + @Override public CodedOutputStreamHolder createStream() { return new CodedOutputStreamAndByteBufferWrapper(bufferSize); diff --git a/TrafficCapture/captureOffloader/src/testFixtures/java/org/opensearch/migrations/trafficcapture/InMemoryConnectionCaptureFactory.java b/TrafficCapture/captureOffloader/src/testFixtures/java/org/opensearch/migrations/trafficcapture/InMemoryConnectionCaptureFactory.java index 8af6b3a89..24c7718dd 100644 --- a/TrafficCapture/captureOffloader/src/testFixtures/java/org/opensearch/migrations/trafficcapture/InMemoryConnectionCaptureFactory.java +++ b/TrafficCapture/captureOffloader/src/testFixtures/java/org/opensearch/migrations/trafficcapture/InMemoryConnectionCaptureFactory.java @@ -1,14 +1,13 @@ package org.opensearch.migrations.trafficcapture; -import com.google.protobuf.CodedOutputStream; import com.google.protobuf.InvalidProtocolBufferException; import lombok.AllArgsConstructor; import lombok.Getter; -import lombok.NonNull; +import org.opensearch.migrations.tracing.commoncontexts.IConnectionContext; import org.opensearch.migrations.trafficcapture.protos.TrafficStream; +import org.opensearch.migrations.trafficcapture.tracing.ConnectionContext; import java.io.IOException; -import java.nio.ByteBuffer; import java.util.Arrays; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentLinkedQueue; @@ -36,6 +35,9 @@ public InMemoryConnectionCaptureFactory(String nodeId, int bufferSize, Runnable @AllArgsConstructor class StreamManager extends OrderedStreamLifecyleManager { + @Override + public void close() {} + @Override public CodedOutputStreamHolder createStream() { return new CodedOutputStreamAndByteBufferWrapper(bufferSize); @@ -59,7 +61,7 @@ protected CompletableFuture kickoffCloseStream(CodedOutputStreamHolder out } @Override - public IChannelConnectionCaptureSerializer createOffloader(String connectionId) throws IOException { + public IChannelConnectionCaptureSerializer createOffloader(IConnectionContext ctx, String connectionId) throws IOException { // This array is only an indirection to work around Java's constraint that lambda values are final return new StreamChannelConnectionCaptureSerializer<>(nodeId, connectionId, new StreamManager()); } diff --git a/TrafficCapture/coreUtilities/build.gradle b/TrafficCapture/coreUtilities/build.gradle index 1c4404477..fbe9a0307 100644 --- a/TrafficCapture/coreUtilities/build.gradle +++ b/TrafficCapture/coreUtilities/build.gradle @@ -40,6 +40,8 @@ repositories { } dependencies { + implementation platform("io.opentelemetry:opentelemetry-bom:1.32.0") + implementation project(':captureProtobufs') implementation "com.google.protobuf:protobuf-java:3.22.2" @@ -51,14 +53,16 @@ dependencies { // Log4j implementation(platform("org.apache.logging.log4j:log4j-bom:2.21.1")) - implementation("org.apache.logging.log4j:log4j-api") - implementation("org.apache.logging.log4j:log4j-core") - implementation("org.apache.logging.log4j:log4j-slf4j2-impl:2.20.0") + implementation group: 'org.apache.logging.log4j', name: 'log4j-api' + implementation group: 'org.apache.logging.log4j', name :'log4j-core' + implementation group: 'org.apache.logging.log4j', name: 'log4j-slf4j2-impl', version: '2.20.0' // OpenTelemetry core - implementation("io.opentelemetry:opentelemetry-sdk:1.30.0") - implementation("io.opentelemetry:opentelemetry-exporter-otlp:1.30.0") - implementation("io.opentelemetry:opentelemetry-semconv:1.30.1-alpha") + implementation group: 'io.opentelemetry', name:'opentelemetry-api' + implementation group: 'io.opentelemetry', name:'opentelemetry-exporter-otlp' + implementation group: 'io.opentelemetry', name:'opentelemetry-sdk' + implementation group: 'io.opentelemetry.instrumentation', name:'opentelemetry-log4j-appender-2.17', version: '1.30.0-alpha' + implementation group: 'io.opentelemetry', name:'opentelemetry-semconv', version: '1.30.0-alpha' // OpenTelemetry log4j appender implementation("io.opentelemetry.instrumentation:opentelemetry-log4j-appender-2.17:1.30.0-alpha") diff --git a/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/coreutils/MetricsLogBuilder.java b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/coreutils/MetricsLogBuilder.java index d5b2e8480..90744baa7 100644 --- a/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/coreutils/MetricsLogBuilder.java +++ b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/coreutils/MetricsLogBuilder.java @@ -5,10 +5,11 @@ import org.slf4j.Logger; import org.slf4j.spi.LoggingEventBuilder; +import java.util.Optional; + @Slf4j -public -class MetricsLogBuilder { +public class MetricsLogBuilder { private Logger logger; private LoggingEventBuilder loggingEventBuilder; @@ -16,6 +17,11 @@ public MetricsLogBuilder(Logger logger) { this.logger = logger; } + public static MetricsLogBuilder addMetricIfPresent(MetricsLogBuilder metricBuilder, + MetricsAttributeKey key, Optional value) { + return value.map(v -> metricBuilder.setAttribute(key, v)).orElse(metricBuilder); + } + public MetricsLogBuilder setAttribute(MetricsAttributeKey key, Object value) { loggingEventBuilder = loggingEventBuilder.addKeyValue(key.getKeyName(), value); return this; diff --git a/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/coreutils/MetricsLogger.java b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/coreutils/MetricsLogger.java index ec62c88d1..ef61af2d6 100644 --- a/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/coreutils/MetricsLogger.java +++ b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/coreutils/MetricsLogger.java @@ -1,17 +1,30 @@ package org.opensearch.migrations.coreutils; + import io.opentelemetry.api.GlobalOpenTelemetry; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.metrics.Meter; +import io.opentelemetry.api.trace.Tracer; +import io.opentelemetry.context.Context; import io.opentelemetry.exporter.otlp.logs.OtlpGrpcLogRecordExporter; -import io.opentelemetry.instrumentation.log4j.appender.v2_17.OpenTelemetryAppender; +import io.opentelemetry.exporter.otlp.metrics.OtlpGrpcMetricExporter; +import io.opentelemetry.exporter.otlp.trace.OtlpGrpcSpanExporter; import io.opentelemetry.sdk.OpenTelemetrySdk; import io.opentelemetry.sdk.logs.SdkLoggerProvider; import io.opentelemetry.sdk.logs.export.BatchLogRecordProcessor; +import io.opentelemetry.sdk.metrics.export.PeriodicMetricReader; +import io.opentelemetry.sdk.metrics.SdkMeterProvider; import io.opentelemetry.sdk.resources.Resource; +import io.opentelemetry.sdk.trace.SdkTracerProvider; +import io.opentelemetry.sdk.trace.export.BatchSpanProcessor; import io.opentelemetry.semconv.resource.attributes.ResourceAttributes; import org.slf4j.Logger; import lombok.extern.slf4j.Slf4j; import org.slf4j.LoggerFactory; +import java.time.Duration; +import java.util.concurrent.TimeUnit; + @Slf4j public class MetricsLogger { @@ -28,31 +41,6 @@ public MetricsLogger(String source) { logger = LoggerFactory.getLogger(String.format("MetricsLogger.%s", source)); } - public static void initializeOpenTelemetry(String serviceName, String collectorEndpoint) { - OpenTelemetrySdk sdk = - OpenTelemetrySdk.builder() - .setLoggerProvider( - SdkLoggerProvider.builder() - .setResource( - Resource.getDefault().toBuilder() - .put(ResourceAttributes.SERVICE_NAME, serviceName) - .build()) - .addLogRecordProcessor( - BatchLogRecordProcessor.builder( - OtlpGrpcLogRecordExporter.builder() - .setEndpoint(collectorEndpoint) - .build()) - .build()) - .build()) - .build(); - GlobalOpenTelemetry.set(sdk); - - // Add hook to close SDK, which flushes logs - Runtime.getRuntime().addShutdownHook(new Thread(sdk::close)); - OpenTelemetryAppender.install(GlobalOpenTelemetry.get()); - } - - /** * To indicate a successful event (e.g. data received or data sent) that may be a helpful * metric, this method can be used to return a LoggingEventBuilder. The LoggingEventBuilder @@ -84,6 +72,7 @@ public MetricsLogBuilder atError(MetricsEvent event, Throwable cause) { * there is a failure that isn't indicated by an Exception being thrown. */ public MetricsLogBuilder atError(MetricsEvent event) { + return new MetricsLogBuilder(logger).atError(event); } diff --git a/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/EmptyContext.java b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/EmptyContext.java new file mode 100644 index 000000000..d44a356c3 --- /dev/null +++ b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/EmptyContext.java @@ -0,0 +1,25 @@ +package org.opensearch.migrations.tracing; + +import io.opentelemetry.api.common.AttributesBuilder; +import io.opentelemetry.api.trace.Span; + +public class EmptyContext implements IWithAttributes { + public static final EmptyContext singleton = new EmptyContext(); + + private EmptyContext() {} + + @Override + public Span getCurrentSpan() { + throw new IllegalStateException("This class doesn't track spans"); + } + + @Override + public IWithAttributes getEnclosingScope() { + return null; + } + + @Override + public AttributesBuilder fillAttributes(AttributesBuilder builder) { + return builder; // nothing more to do + } +} diff --git a/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/ISpanGenerator.java b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/ISpanGenerator.java new file mode 100644 index 000000000..84eb59192 --- /dev/null +++ b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/ISpanGenerator.java @@ -0,0 +1,8 @@ +package org.opensearch.migrations.tracing; + +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.trace.Span; + +import java.util.function.Function; + +public interface ISpanGenerator extends Function { } diff --git a/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/ISpanWithParentGenerator.java b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/ISpanWithParentGenerator.java new file mode 100644 index 000000000..bdd4dc066 --- /dev/null +++ b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/ISpanWithParentGenerator.java @@ -0,0 +1,9 @@ +package org.opensearch.migrations.tracing; + +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.trace.Span; + +import java.util.function.BiFunction; + +public interface ISpanWithParentGenerator extends BiFunction { +} diff --git a/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/IWithAttributes.java b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/IWithAttributes.java new file mode 100644 index 000000000..6f4f244b0 --- /dev/null +++ b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/IWithAttributes.java @@ -0,0 +1,36 @@ +package org.opensearch.migrations.tracing; + +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.common.AttributesBuilder; +import io.opentelemetry.api.trace.Span; + +import java.util.ArrayList; + +public interface IWithAttributes { + T getEnclosingScope(); + + Span getCurrentSpan(); + + default AttributesBuilder fillAttributes(AttributesBuilder builder) { + return builder; + } + + default Attributes getPopulatedAttributes() { + return getPopulatedAttributesBuilder().build(); + } + + default AttributesBuilder getPopulatedAttributesBuilder() { + var currentObj = this; + var stack = new ArrayList(); + var builder = Attributes.builder(); + while (currentObj != null) { + stack.add(currentObj); + currentObj = currentObj.getEnclosingScope(); + } + // reverse the order so that the lowest attribute scopes will overwrite the upper ones if there were conflicts + for (int i=stack.size()-1; i>=0; --i) { + builder = stack.get(i).fillAttributes(builder); + } + return builder; + } +} diff --git a/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/IWithStartTime.java b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/IWithStartTime.java new file mode 100644 index 000000000..b8e362ddb --- /dev/null +++ b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/IWithStartTime.java @@ -0,0 +1,7 @@ +package org.opensearch.migrations.tracing; + +import java.time.Instant; + +public interface IWithStartTime { + Instant getStartTime(); +} diff --git a/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/IWithStartTimeAndAttributes.java b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/IWithStartTimeAndAttributes.java new file mode 100644 index 000000000..31be5b08c --- /dev/null +++ b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/IWithStartTimeAndAttributes.java @@ -0,0 +1,4 @@ +package org.opensearch.migrations.tracing; + +public interface IWithStartTimeAndAttributes extends IWithStartTime, IWithAttributes { +} diff --git a/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/SimpleMeteringClosure.java b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/SimpleMeteringClosure.java new file mode 100644 index 000000000..3714739c7 --- /dev/null +++ b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/SimpleMeteringClosure.java @@ -0,0 +1,151 @@ +package org.opensearch.migrations.tracing; + +import io.opentelemetry.api.GlobalOpenTelemetry; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.metrics.Meter; +import io.opentelemetry.api.trace.Span; +import io.opentelemetry.api.trace.SpanBuilder; +import io.opentelemetry.api.trace.Tracer; +import io.opentelemetry.context.Context; +import io.opentelemetry.exporter.otlp.logs.OtlpGrpcLogRecordExporter; +import io.opentelemetry.exporter.otlp.metrics.OtlpGrpcMetricExporter; +import io.opentelemetry.exporter.otlp.trace.OtlpGrpcSpanExporter; +import io.opentelemetry.sdk.OpenTelemetrySdk; +import io.opentelemetry.sdk.logs.SdkLoggerProvider; +import io.opentelemetry.sdk.logs.export.BatchLogRecordProcessor; +import io.opentelemetry.sdk.metrics.SdkMeterProvider; +import io.opentelemetry.sdk.metrics.export.PeriodicMetricReader; +import io.opentelemetry.sdk.resources.Resource; +import io.opentelemetry.sdk.trace.SdkTracerProvider; +import io.opentelemetry.sdk.trace.export.BatchSpanProcessor; +import io.opentelemetry.semconv.resource.attributes.ResourceAttributes; + +import java.time.Duration; +import java.time.Instant; +import java.util.Optional; +import java.util.concurrent.TimeUnit; + +public class SimpleMeteringClosure { + public final Meter meter; + public final Tracer tracer; + + public SimpleMeteringClosure(String scopeName) { + meter = GlobalOpenTelemetry.getMeter(scopeName); + tracer = GlobalOpenTelemetry.getTracer(scopeName); + } + + public static void initializeOpenTelemetry(String serviceName, String collectorEndpoint) { + var serviceResource = Resource.getDefault().toBuilder() + .put(ResourceAttributes.SERVICE_NAME, serviceName) + .build(); + + OpenTelemetrySdk openTelemetrySdk = + OpenTelemetrySdk.builder() + .setLoggerProvider( + SdkLoggerProvider.builder() + .setResource(serviceResource) + .addLogRecordProcessor( + BatchLogRecordProcessor.builder( + OtlpGrpcLogRecordExporter.builder() + .setEndpoint(collectorEndpoint) + .build()) + .build()) + .build()) + .setTracerProvider( + SdkTracerProvider.builder() + .setResource(serviceResource) + .addSpanProcessor( + BatchSpanProcessor.builder( + OtlpGrpcSpanExporter.builder() + .setEndpoint(collectorEndpoint) + .setTimeout(2, TimeUnit.SECONDS) + .build()) + .setScheduleDelay(100, TimeUnit.MILLISECONDS) + .build()) + .build()) + .setMeterProvider( + SdkMeterProvider.builder() + .setResource(serviceResource) + .registerMetricReader( + PeriodicMetricReader.builder( + OtlpGrpcMetricExporter.builder() + .setEndpoint(collectorEndpoint) + .build()) + .setInterval(Duration.ofMillis(1000)) + .build()) + .build()) + .buildAndRegisterGlobal(); + + // Add hook to close SDK, which flushes logs + Runtime.getRuntime().addShutdownHook(new Thread(openTelemetrySdk::close)); + //OpenTelemetryAppender.install(GlobalOpenTelemetry.get()); + } + + public void meterIncrementEvent(IWithAttributes ctx, String eventName) { + meterIncrementEvent(ctx, eventName, 1); + } + + public void meterIncrementEvent(IWithAttributes ctx, String eventName, long increment) { + if (ctx == null) { + return; + } + meter.counterBuilder(eventName) + .build().add(increment, ctx.getPopulatedAttributesBuilder() + .put("labelName", eventName) + .build()); + } + + public void meterDeltaEvent(IWithAttributes ctx, String eventName, long delta) { + if (ctx == null) { + return; + } + meter.upDownCounterBuilder(eventName) + .build().add(delta, ctx.getPopulatedAttributesBuilder() + .put("labelName", eventName) + .build()); + } + + public void meterHistogramMillis(T ctx, String eventName) { + meterHistogram(ctx, eventName, "ms", Duration.between(ctx.getStartTime(), Instant.now()).toMillis()); + } + + public void meterHistogramMicros(T ctx, String eventName) { + meterHistogram(ctx, eventName, "us", Duration.between(ctx.getStartTime(), Instant.now()).toNanos()*1000); + } + + public void meterHistogramMillis(IWithAttributes ctx, String eventName, Duration between) { + meterHistogram(ctx, eventName, "ms", between.toMillis()); + } + + public void meterHistogramMicros(IWithAttributes ctx, String eventName, Duration between) { + meterHistogram(ctx, eventName, "us", between.toNanos()*1000); + } + + public void meterHistogram(IWithAttributes ctx, String eventName, String units, long value) { + if (ctx == null) { + return; + } + meter.histogramBuilder(eventName) + .ofLongs() + .setUnit(units) + .build().record(value, ctx.getPopulatedAttributesBuilder() + .put("labelName", eventName) + .build()); + } + + public ISpanGenerator makeSpanContinuation(String spanName, Span parentSpan) { + var builder = tracer.spanBuilder(spanName); + return (attrs) -> getSpanWithParent(builder, attrs, parentSpan); + } + + public static Span getSpanWithParent(SpanBuilder builder, Attributes attrs, Span parentSpan) { + return Optional.ofNullable(parentSpan).map(p -> builder.setParent(Context.current().with(p))) + .orElseGet(builder::setNoParent) + .startSpan().setAllAttributes(attrs); + } + + public ISpanWithParentGenerator makeSpanContinuation(String spanName) { + var builder = tracer.spanBuilder(spanName); + return (attrs,parentSpan) -> getSpanWithParent(builder, attrs, parentSpan); + } +} diff --git a/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/commoncontexts/IConnectionContext.java b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/commoncontexts/IConnectionContext.java new file mode 100644 index 000000000..83476b9fa --- /dev/null +++ b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/commoncontexts/IConnectionContext.java @@ -0,0 +1,23 @@ +package org.opensearch.migrations.tracing.commoncontexts; + +import io.opentelemetry.api.common.AttributeKey; +import io.opentelemetry.api.common.AttributesBuilder; +import org.opensearch.migrations.tracing.EmptyContext; +import org.opensearch.migrations.tracing.IWithAttributes; + +public interface IConnectionContext extends IWithAttributes { + static final AttributeKey CONNECTION_ID_ATTR = AttributeKey.stringKey("connectionId"); + static final AttributeKey NODE_ID_ATTR = AttributeKey.stringKey("nodeId"); + + String getConnectionId(); + String getNodeId(); + + @Override + default EmptyContext getEnclosingScope() { return EmptyContext.singleton; } + + @Override + default AttributesBuilder fillAttributes(AttributesBuilder builder) { + return builder.put(CONNECTION_ID_ATTR, getConnectionId()) + .put(NODE_ID_ATTR, getNodeId()); + } +} diff --git a/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/commoncontexts/IRequestContext.java b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/commoncontexts/IRequestContext.java new file mode 100644 index 000000000..c6b932551 --- /dev/null +++ b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/commoncontexts/IRequestContext.java @@ -0,0 +1,16 @@ +package org.opensearch.migrations.tracing.commoncontexts; + +import io.opentelemetry.api.common.AttributeKey; +import io.opentelemetry.api.common.AttributesBuilder; +import org.opensearch.migrations.tracing.IWithAttributes; + +public interface IRequestContext extends IWithAttributes { + static final AttributeKey SOURCE_REQUEST_INDEX_KEY = AttributeKey.longKey("sourceRequestIndex"); + + long getSourceRequestIndex(); + + @Override + default AttributesBuilder fillAttributes(AttributesBuilder builder) { + return builder.put(SOURCE_REQUEST_INDEX_KEY, getSourceRequestIndex()); + } +} diff --git a/TrafficCapture/dockerSolution/src/main/docker/docker-compose-single.yml b/TrafficCapture/dockerSolution/src/main/docker/docker-compose-single.yml index febda3d42..0ae8ffbcb 100644 --- a/TrafficCapture/dockerSolution/src/main/docker/docker-compose-single.yml +++ b/TrafficCapture/dockerSolution/src/main/docker/docker-compose-single.yml @@ -14,7 +14,7 @@ services: - http.port=19200 - discovery.type=single-node # Run processes for elasticsearch and capture proxy, and exit if either one ends - command: /bin/sh -c '/usr/local/bin/docker-entrypoint.sh eswrapper & /runJavaWithClasspath.sh org.opensearch.migrations.trafficcapture.proxyserver.CaptureProxy --kafkaConnection kafka:9092 --destinationUri https://localhost:19200 --insecureDestination --listenPort 9200 --sslConfigFile /usr/share/elasticsearch/config/proxy_tls.yml & wait -n 1' + command: /bin/sh -c '/usr/local/bin/docker-entrypoint.sh eswrapper & /runJavaWithClasspath.sh org.opensearch.migrations.trafficcapture.proxyserver.CaptureProxy --kafkaConnection kafka:9092 --destinationUri https://localhost:19200 --insecureDestination --listenPort 9200 --sslConfigFile /usr/share/elasticsearch/config/proxy_tls.yml --otelCollectorEndpoint http://otel-collector:4317 & wait -n 1' depends_on: - kafka @@ -25,7 +25,7 @@ services: # - migrations # ports: # - "9200:9200" -# command: /runJavaWithClasspath.sh org.opensearch.migrations.trafficcapture.proxyserver.CaptureProxy --kafkaConnection kafka:9092 --destinationUri https://elasticsearch:9200 --insecureDestination --listenPort 9200 --sslConfigFile /usr/share/elasticsearch/config/proxy_tls.yml +# command: /runJavaWithClasspath.sh org.opensearch.migrations.trafficcapture.proxyserver.CaptureProxy --kafkaConnection kafka:9092 --destinationUri https://elasticsearch:9200 --insecureDestination --listenPort 9200 --sslConfigFile /usr/share/elasticsearch/config/proxy_tls.yml --otelCollectorEndpoint http://otel-collector:4317 --otelCollectorEndpoint http://otel-collector:4317 # depends_on: # - kafka # - elasticsearch diff --git a/TrafficCapture/dockerSolution/src/main/docker/docker-compose.yml b/TrafficCapture/dockerSolution/src/main/docker/docker-compose.yml index 14c1b00ff..18a2c9f89 100644 --- a/TrafficCapture/dockerSolution/src/main/docker/docker-compose.yml +++ b/TrafficCapture/dockerSolution/src/main/docker/docker-compose.yml @@ -1,39 +1,57 @@ version: '3.7' services: - # Run combined instance of Capture Proxy and Elasticsearch - capture-proxy-es: - image: 'migrations/capture_proxy:latest' + + prometheus: + container_name: prometheus + image: prom/prometheus:latest networks: - migrations + volumes: + - ./prometheus.yaml:/etc/prometheus/prometheus.yml ports: - - "9200:9200" - - "19200:19200" + - "9090:9090" + + # Jaeger + jaeger: + image: jaegertracing/all-in-one:latest + networks: + - migrations + ports: + - "16686:16686" + - "4317" + - "4318" environment: - - http.port=19200 - # Run processes for elasticsearch and capture proxy, and exit if either one ends - command: /bin/sh -c '/usr/local/bin/docker-entrypoint.sh eswrapper & /runJavaWithClasspath.sh org.opensearch.migrations.trafficcapture.proxyserver.CaptureProxy --kafkaConnection kafka:9092 --destinationUri https://localhost:19200 --insecureDestination --listenPort 9200 --sslConfigFile /usr/share/elasticsearch/config/proxy_tls.yml --otelCollectorEndpoint http://otel-collector:4317 & wait -n 1' - depends_on: - - kafka + - COLLECTOR_OTLP_ENABLED=true -# Run separate instances of Capture Proxy and Elasticsearch -# capture-proxy: -# image: 'migrations/capture_proxy:latest' -# networks: -# - migrations -# ports: -# - "9200:9200" -# command: /runJavaWithClasspath.sh org.opensearch.migrations.trafficcapture.proxyserver.CaptureProxy --kafkaConnection kafka:9092 --destinationUri https://elasticsearch:9200 --insecureDestination --listenPort 9200 --sslConfigFile /usr/share/elasticsearch/config/proxy_tls.yml -# depends_on: -# - kafka -# - elasticsearch -# -# elasticsearch: -# image: 'migrations/elasticsearch_searchguard:latest' -# networks: -# - migrations -# ports: -# - '19200:9200' + # Zipkin + zipkin: + image: openzipkin/zipkin:latest + networks: + - migrations + ports: + - "9411:9411" + + # Collector + otel-collector: + image: otel/opentelemetry-collector:latest +# command: ["--config=/etc/otel-collector-config-demo.yaml", "${OTELCOL_ARGS}"] + networks: + - migrations + volumes: +# - ./otel-collector-config-demo.yaml:/etc/otel-collector-config-demo.yaml + - ./otel-collector-config-demo.yaml:/etc/otelcol/config.yaml + - /Users/schohn/dev/opensearch-migrations/TrafficCapture/containerLogs:/logs + ports: + - "1888:1888" # pprof extension + - "8888:8888" # Prometheus metrics exposed by the collector + - "8889:8889" # Prometheus exporter metrics + - "13133:13133" # health_check extension + - "55679:55679" # zpages extension + - "4317:4317" # otlp receiver + depends_on: + - jaeger + - zipkin zookeeper: image: docker.io/bitnami/zookeeper:3.8 @@ -70,6 +88,7 @@ services: - migrations volumes: - sharedReplayerOutput:/shared-replayer-output + - /Users/schohn/dev/opensearch-migrations/TrafficCapture/containerLogs:/logs environment: - TUPLE_DIR_PATH=/shared-replayer-output/traffic-replayer-default depends_on: @@ -111,18 +130,20 @@ services: depends_on: - opensearchanalytics - otel-collector: - image: public.ecr.aws/a0w2c5q7/otelcol-with-opensearch:latest - container_name: otel-collector - ports: - - "4317:4317" - - "13133:13133" - volumes: - - ./otelcol/otel-config.yml:/etc/otel-config.yml - networks: - - migrations - depends_on: - - opensearchanalytics +# otel-collector: +# image: public.ecr.aws/a0w2c5q7/otelcol-with-opensearch:latest +# container_name: otel-collector +# ports: +# - "4317:4317" +# - "13133:13133" +# volumes: +# - ./otelcol/otel-config.yml:/etc/otel-config.yml +# - /Users/schohn/dev/opensearch-migrations/TrafficCapture/containerLogs:/logs +# networks: +# - migrations +# depends_on: +# - opensearchanalytics +# command: tail -f /dev/null migration-console: image: 'migrations/migration_console:latest' @@ -132,6 +153,7 @@ services: - sharedReplayerOutput:/shared-replayer-output environment: - MIGRATION_KAFKA_BROKER_ENDPOINTS=kafka:9092 +# command: ./runTestBenchmarks.sh volumes: zookeeper_data: diff --git a/TrafficCapture/dockerSolution/src/main/docker/otel-collector-config-demo.yaml b/TrafficCapture/dockerSolution/src/main/docker/otel-collector-config-demo.yaml new file mode 100644 index 000000000..ac9a2a6d5 --- /dev/null +++ b/TrafficCapture/dockerSolution/src/main/docker/otel-collector-config-demo.yaml @@ -0,0 +1,39 @@ +receivers: + otlp: + protocols: + grpc: + +exporters: + prometheus: + endpoint: "0.0.0.0:8889" + const_labels: + label1: value1 + logging: + loglevel: debug + + zipkin: + endpoint: "http://zipkin:9411/api/v2/spans" + format: proto + + otlp/jaeger: # Jaeger supports OTLP directly. The default port for OTLP/gRPC is 4317 + endpoint: jaeger:4317 + tls: + insecure: true + +extensions: + health_check: + pprof: + endpoint: :1888 + zpages: + endpoint: :55679 + +service: + extensions: [pprof, zpages, health_check] + pipelines: + traces: + receivers: [otlp] + processors: [] + exporters: [logging, zipkin, otlp/jaeger] + metrics: + receivers: [otlp] + exporters: [logging, prometheus] diff --git a/TrafficCapture/dockerSolution/src/main/docker/otelcol/otel-config.yml b/TrafficCapture/dockerSolution/src/main/docker/otelcol/otel-config.yml index f6ccc70e6..e84720698 100644 --- a/TrafficCapture/dockerSolution/src/main/docker/otelcol/otel-config.yml +++ b/TrafficCapture/dockerSolution/src/main/docker/otelcol/otel-config.yml @@ -52,6 +52,8 @@ exporters: insecure_skip_verify: true logging: verbosity: detailed + file: + path: /logs/filename.json debug: service: @@ -63,4 +65,4 @@ service: logs: receivers: [otlp] processors: [attributes] - exporters: [logging, debug, opensearch] \ No newline at end of file + exporters: [logging, debug, opensearch, file] \ No newline at end of file diff --git a/TrafficCapture/dockerSolution/src/main/docker/prometheus.yaml b/TrafficCapture/dockerSolution/src/main/docker/prometheus.yaml new file mode 100644 index 000000000..ddea76205 --- /dev/null +++ b/TrafficCapture/dockerSolution/src/main/docker/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'] diff --git a/TrafficCapture/nettyWireLogging/build.gradle b/TrafficCapture/nettyWireLogging/build.gradle index ade3744c9..523fe0078 100644 --- a/TrafficCapture/nettyWireLogging/build.gradle +++ b/TrafficCapture/nettyWireLogging/build.gradle @@ -8,20 +8,30 @@ plugins { } dependencies { + implementation platform("io.opentelemetry:opentelemetry-bom:1.32.0") + implementation platform("io.netty:netty-bom:4.1.100.Final") + implementation project(':captureOffloader') implementation project(':coreUtilities') - api group: 'io.netty', name: 'netty-all', version: '4.1.100.Final' + api group: 'io.netty', name: 'netty-buffer' + api group: 'io.netty', name: 'netty-codec-http' + api group: 'io.netty', name: 'netty-handler' + + implementation group: 'io.opentelemetry', name:'opentelemetry-api' implementation group: 'org.slf4j', name: 'slf4j-api', version: '2.0.7' testImplementation project(':captureProtobufs') - testImplementation group: 'org.apache.httpcomponents.client5', name: 'httpclient5', version: '5.2.1' - testImplementation group: 'org.slf4j', name: 'slf4j-api', version: '2.0.7' testImplementation group: 'com.google.guava', name: 'guava', version: '32.0.1-jre' testImplementation group: 'com.google.protobuf', name: 'protobuf-java', version:'3.22.2' - + testImplementation group: 'io.opentelemetry', name: 'opentelemetry-sdk' + testImplementation group: 'io.opentelemetry', name: 'opentelemetry-sdk-testing' + testImplementation group: 'org.apache.httpcomponents.client5', name: 'httpclient5', version: '5.2.1' testImplementation group: 'org.apache.logging.log4j', name: 'log4j-api', version: '2.20.0' testImplementation group: 'org.apache.logging.log4j', name: 'log4j-core', version: '2.20.0' testImplementation group: 'org.apache.logging.log4j', name: 'log4j-slf4j2-impl', version: '2.20.0' + testImplementation group: 'org.slf4j', name: 'slf4j-api', version: '2.0.7' testImplementation testFixtures(project(path: ':testUtilities')) + + } diff --git a/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/ConditionallyReliableLoggingHttpRequestHandler.java b/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/ConditionallyReliableLoggingHttpRequestHandler.java index 441dafbbf..a641b4bfe 100644 --- a/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/ConditionallyReliableLoggingHttpRequestHandler.java +++ b/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/ConditionallyReliableLoggingHttpRequestHandler.java @@ -3,21 +3,33 @@ import io.netty.channel.ChannelHandlerContext; import io.netty.handler.codec.http.HttpRequest; import io.netty.util.ReferenceCountUtil; +import io.opentelemetry.api.trace.Span; +import lombok.Getter; import lombok.Lombok; import lombok.NonNull; import lombok.extern.slf4j.Slf4j; +import org.opensearch.migrations.tracing.IWithStartTimeAndAttributes; +import org.opensearch.migrations.tracing.commoncontexts.IConnectionContext; import org.opensearch.migrations.trafficcapture.IChannelConnectionCaptureSerializer; +import org.opensearch.migrations.trafficcapture.IConnectionCaptureFactory; +import org.opensearch.migrations.trafficcapture.netty.tracing.HttpMessageContext; +import org.opensearch.migrations.trafficcapture.tracing.ConnectionContext; +import java.io.IOException; +import java.time.Instant; +import java.util.function.Function; import java.util.function.Predicate; @Slf4j public class ConditionallyReliableLoggingHttpRequestHandler extends LoggingHttpRequestHandler { private final Predicate shouldBlockPredicate; - public ConditionallyReliableLoggingHttpRequestHandler(@NonNull IChannelConnectionCaptureSerializer trafficOffloader, + public ConditionallyReliableLoggingHttpRequestHandler(@NonNull String nodeId, String connectionId, + @NonNull IConnectionCaptureFactory trafficOffloaderFactory, @NonNull RequestCapturePredicate requestCapturePredicate, - @NonNull Predicate headerPredicateForWhenToBlock) { - super(trafficOffloader, requestCapturePredicate); + @NonNull Predicate headerPredicateForWhenToBlock) + throws IOException { + super(nodeId, connectionId, trafficOffloaderFactory, requestCapturePredicate); this.shouldBlockPredicate = headerPredicateForWhenToBlock; } @@ -26,12 +38,21 @@ protected void channelFinishedReadingAnHttpMessage(ChannelHandlerContext ctx, Ob boolean shouldCapture, HttpRequest httpRequest) throws Exception { if (shouldCapture && shouldBlockPredicate.test(httpRequest)) { + METERING_CLOSURE.meterIncrementEvent(messageContext, "blockingRequestUntilFlush"); + rotateNextMessageContext(HttpMessageContext.HttpTransactionState.INTERNALLY_BLOCKED); trafficOffloader.flushCommitAndResetStream(false).whenComplete((result, t) -> { + log.atInfo().setMessage(()->"Done flushing").log(); + METERING_CLOSURE.meterIncrementEvent(messageContext, + t != null ? "blockedFlushFailure" : "blockedFlushSuccess"); + METERING_CLOSURE.meterHistogramMicros(messageContext, + t==null ? "blockedFlushFailure_micro" : "stream_flush_failure_micro"); + messageContext.getCurrentSpan().end(); + if (t != null) { // This is a spot where we would benefit from having a behavioral policy that different users // could set as needed. Some users may be fine with just logging a failed offloading of a request // where other users may want to stop entirely. JIRA here: https://opensearch.atlassian.net/browse/MIGRATIONS-1276 - log.atWarn().setCause(t).setMessage("Got error").log(); + log.atWarn().setCause(t).setMessage("Dropping request - Got error").log(); ReferenceCountUtil.release(msg); } else { try { @@ -42,6 +63,8 @@ protected void channelFinishedReadingAnHttpMessage(ChannelHandlerContext ctx, Ob } }); } else { + METERING_CLOSURE.meterIncrementEvent(messageContext, "nonBlockingRequest"); + // TODO - log capturing vs non-capturing too super.channelFinishedReadingAnHttpMessage(ctx, msg, shouldCapture, httpRequest); } } diff --git a/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/LoggingHttpRequestHandler.java b/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/LoggingHttpRequestHandler.java index ca3e5075f..f941acf3c 100644 --- a/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/LoggingHttpRequestHandler.java +++ b/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/LoggingHttpRequestHandler.java @@ -1,8 +1,10 @@ package org.opensearch.migrations.trafficcapture.netty; import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelDuplexHandler; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelInboundHandlerAdapter; +import io.netty.channel.ChannelPromise; import io.netty.channel.embedded.EmbeddedChannel; import io.netty.handler.codec.http.DefaultHttpRequest; import io.netty.handler.codec.http.HttpContent; @@ -19,14 +21,25 @@ import lombok.extern.slf4j.Slf4j; import org.opensearch.migrations.coreutils.MetricsAttributeKey; import org.opensearch.migrations.coreutils.MetricsEvent; +import org.opensearch.migrations.tracing.SimpleMeteringClosure; import org.opensearch.migrations.trafficcapture.IChannelConnectionCaptureSerializer; import org.opensearch.migrations.coreutils.MetricsLogger; +import org.opensearch.migrations.trafficcapture.IConnectionCaptureFactory; +import org.opensearch.migrations.trafficcapture.netty.tracing.HttpMessageContext; +import org.opensearch.migrations.trafficcapture.tracing.ConnectionContext; +import java.io.IOException; import java.time.Instant; @Slf4j -public class LoggingHttpRequestHandler extends ChannelInboundHandlerAdapter { +public class LoggingHttpRequestHandler extends ChannelDuplexHandler { + public static final String TELEMETRY_SCOPE_NAME = "CapturingHttpHandler"; + public static final SimpleMeteringClosure METERING_CLOSURE = new SimpleMeteringClosure(TELEMETRY_SCOPE_NAME); private static final MetricsLogger metricsLogger = new MetricsLogger("LoggingHttpRequestHandler"); + public static final String GATHERING_REQUEST = "gatheringRequest"; + public static final String WAITING_FOR_RESPONSE = "waitingForResponse"; + public static final String GATHERING_RESPONSE = "gatheringResponse"; + public static final String BLOCKED = "blocked"; static class SimpleHttpRequestDecoder extends HttpRequestDecoder { private final PassThruHttpHeaders.HttpHeadersToPreserve headersToPreserve; @@ -95,15 +108,50 @@ public HttpRequest resetCurrentRequest() { protected final EmbeddedChannel httpDecoderChannel; - public LoggingHttpRequestHandler(IChannelConnectionCaptureSerializer trafficOffloader, - @NonNull RequestCapturePredicate httpHeadersCapturePredicate) { - this.trafficOffloader = trafficOffloader; + protected HttpMessageContext messageContext; + + public LoggingHttpRequestHandler(String nodeId, String channelKey, + @NonNull IConnectionCaptureFactory trafficOffloaderFactory, + @NonNull RequestCapturePredicate httpHeadersCapturePredicate) + throws IOException { + var parentContext = new ConnectionContext(channelKey, nodeId, + METERING_CLOSURE.makeSpanContinuation("connectionLifetime", null)); + + this.messageContext = new HttpMessageContext(parentContext, 0, HttpMessageContext.HttpTransactionState.REQUEST, + METERING_CLOSURE.makeSpanContinuation(GATHERING_REQUEST)); + METERING_CLOSURE.meterIncrementEvent(messageContext, "requestStarted"); + + this.trafficOffloader = trafficOffloaderFactory.createOffloader(parentContext, channelKey); httpDecoderChannel = new EmbeddedChannel( new SimpleHttpRequestDecoder(httpHeadersCapturePredicate.getHeadersRequiredForMatcher()), new SimpleDecodedHttpRequestHandler(httpHeadersCapturePredicate) ); } + + static String getSpanLabelForState(HttpMessageContext.HttpTransactionState state) { + switch (state) { + case REQUEST: + return GATHERING_REQUEST; + case INTERNALLY_BLOCKED: + return BLOCKED; + case WAITING: + return WAITING_FOR_RESPONSE; + case RESPONSE: + return GATHERING_RESPONSE; + default: + throw new IllegalStateException("Unknown enum value: "+state); + } + } + + protected void rotateNextMessageContext(HttpMessageContext.HttpTransactionState nextState) { + messageContext = new HttpMessageContext(messageContext.getEnclosingScope(), + (nextState== HttpMessageContext.HttpTransactionState.REQUEST ? 1 : 0) + + messageContext.getSourceRequestIndex(), + nextState, + METERING_CLOSURE.makeSpanContinuation(getSpanLabelForState(nextState))); + } + private SimpleDecodedHttpRequestHandler getHandlerThatHoldsParsedHttpRequest() { return (SimpleDecodedHttpRequestHandler) httpDecoderChannel.pipeline().last(); } @@ -111,6 +159,7 @@ private SimpleDecodedHttpRequestHandler getHandlerThatHoldsParsedHttpRequest() { @Override public void channelUnregistered(ChannelHandlerContext ctx) throws Exception { trafficOffloader.addCloseEvent(Instant.now()); + METERING_CLOSURE.meterIncrementEvent(messageContext, "unregistered"); trafficOffloader.flushCommitAndResetStream(true).whenComplete((result, t) -> { if (t != null) { log.warn("Got error: " + t.getMessage()); @@ -127,6 +176,10 @@ public void channelUnregistered(ChannelHandlerContext ctx) throws Exception { @Override public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { + METERING_CLOSURE.meterIncrementEvent(messageContext, "handlerRemoved"); + messageContext.getCurrentSpan().end(); + messageContext.getEnclosingScope().currentSpan.end(); + trafficOffloader.flushCommitAndResetStream(true).whenComplete((result, t) -> { if (t != null) { log.warn("Got error: " + t.getMessage()); @@ -142,7 +195,10 @@ public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { protected void channelFinishedReadingAnHttpMessage(ChannelHandlerContext ctx, Object msg, boolean shouldCapture, HttpRequest httpRequest) throws Exception { + rotateNextMessageContext(HttpMessageContext.HttpTransactionState.WAITING); super.channelRead(ctx, msg); + METERING_CLOSURE.meterIncrementEvent(messageContext, "requestReceived"); + metricsLogger.atSuccess(MetricsEvent.RECEIVED_FULL_HTTP_REQUEST) .setAttribute(MetricsAttributeKey.CHANNEL_ID, ctx.channel().id().asLongText()) .setAttribute(MetricsAttributeKey.HTTP_METHOD, httpRequest.method().toString()) @@ -151,22 +207,36 @@ protected void channelFinishedReadingAnHttpMessage(ChannelHandlerContext ctx, Ob @Override public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + if (messageContext.getState() == HttpMessageContext.HttpTransactionState.RESPONSE) { + messageContext.getCurrentSpan().end(); + rotateNextMessageContext(HttpMessageContext.HttpTransactionState.REQUEST); + } var timestamp = Instant.now(); var requestParsingHandler = getHandlerThatHoldsParsedHttpRequest(); var bb = ((ByteBuf) msg); httpDecoderChannel.writeInbound(bb.retainedDuplicate()); // the ByteBuf is consumed/release by this method + + METERING_CLOSURE.meterIncrementEvent(messageContext, + getHandlerThatHoldsParsedHttpRequest().isDone ? "requestFullyParsed" : "requestPartiallyParsed"); + var shouldCapture = requestParsingHandler.shouldCapture; if (shouldCapture) { requestParsingHandler.liveReadObservationsInOffloader = true; trafficOffloader.addReadEvent(timestamp, bb); + } else if (requestParsingHandler.liveReadObservationsInOffloader) { trafficOffloader.cancelCaptureForCurrentRequest(timestamp); requestParsingHandler.liveReadObservationsInOffloader = false; } + metricsLogger.atSuccess(MetricsEvent.RECEIVED_REQUEST_COMPONENT) .setAttribute(MetricsAttributeKey.CHANNEL_ID, ctx.channel().id().asLongText()).emit(); + METERING_CLOSURE.meterIncrementEvent(messageContext, "read"); + METERING_CLOSURE.meterIncrementEvent(messageContext, "readBytes", bb.readableBytes()); + if (requestParsingHandler.isDone) { + messageContext.getCurrentSpan().end(); var httpRequest = requestParsingHandler.resetCurrentRequest(); if (shouldCapture) { var decoderResultLoose = httpRequest.decoderResult(); @@ -183,9 +253,26 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception } } + @Override + public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) throws Exception { + if (messageContext.getState() != HttpMessageContext.HttpTransactionState.RESPONSE) { + messageContext.getCurrentSpan().end(); + rotateNextMessageContext(HttpMessageContext.HttpTransactionState.RESPONSE); + } + var bb = (ByteBuf) msg; + trafficOffloader.addWriteEvent(Instant.now(), bb); + metricsLogger.atSuccess(MetricsEvent.RECEIVED_RESPONSE_COMPONENT) + .setAttribute(MetricsAttributeKey.CHANNEL_ID, ctx.channel().id().asLongText()).emit(); + METERING_CLOSURE.meterIncrementEvent(messageContext, "write"); + METERING_CLOSURE.meterIncrementEvent(messageContext, "writeBytes", bb.readableBytes()); + + super.write(ctx, msg, promise); + } + @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { trafficOffloader.addExceptionCaughtEvent(Instant.now(), cause); + METERING_CLOSURE.meterIncrementEvent(messageContext, "exception"); httpDecoderChannel.close(); super.exceptionCaught(ctx, cause); } diff --git a/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/LoggingHttpResponseHandler.java b/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/LoggingHttpResponseHandler.java index 66d8912a4..50d263550 100644 --- a/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/LoggingHttpResponseHandler.java +++ b/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/LoggingHttpResponseHandler.java @@ -8,70 +8,69 @@ import org.opensearch.migrations.coreutils.MetricsAttributeKey; import org.opensearch.migrations.coreutils.MetricsEvent; import org.opensearch.migrations.coreutils.MetricsLogger; +import org.opensearch.migrations.tracing.SimpleMeteringClosure; import org.opensearch.migrations.trafficcapture.IChannelConnectionCaptureSerializer; +import org.opensearch.migrations.trafficcapture.tracing.ConnectionContext; -import java.net.SocketAddress; import java.time.Instant; @Slf4j public class LoggingHttpResponseHandler extends ChannelOutboundHandlerAdapter { - - private final IChannelConnectionCaptureSerializer trafficOffloader; + public static final String TELEMETRY_SCOPE_NAME = "LoggingHttpOutboundHandler"; + public static final SimpleMeteringClosure METERING_CLOSURE = new SimpleMeteringClosure(TELEMETRY_SCOPE_NAME); private static final MetricsLogger metricsLogger = new MetricsLogger("LoggingHttpResponseHandler"); + private final IChannelConnectionCaptureSerializer trafficOffloader; + private ConnectionContext telemetryContext; - public LoggingHttpResponseHandler(IChannelConnectionCaptureSerializer trafficOffloader) { + public LoggingHttpResponseHandler(ConnectionContext incomingContext, + IChannelConnectionCaptureSerializer trafficOffloader) { this.trafficOffloader = trafficOffloader; + this.telemetryContext = incomingContext; } +// +// @Override +// public void connect(ChannelHandlerContext ctx, SocketAddress remoteAddress, SocketAddress localAddress, ChannelPromise promise) throws Exception { +// trafficOffloader.addConnectEvent(Instant.now(), remoteAddress, localAddress); +// +// telemetryContext = new ConnectionContext(telemetryContext, +// METERING_CLOSURE.makeSpanContinuation("backendConnection")); +// METERING_CLOSURE.meterIncrementEvent(telemetryContext, "connect"); +// METERING_CLOSURE.meterDeltaEvent(telemetryContext, "connections", 1); +// +// super.connect(ctx, remoteAddress, localAddress, promise); +// } - @Override - public void bind(ChannelHandlerContext ctx, SocketAddress localAddress, ChannelPromise promise) throws Exception { - trafficOffloader.addBindEvent(Instant.now(), localAddress); - super.bind(ctx, localAddress, promise); - } - - @Override - public void connect(ChannelHandlerContext ctx, SocketAddress remoteAddress, SocketAddress localAddress, ChannelPromise promise) throws Exception { - trafficOffloader.addConnectEvent(Instant.now(), remoteAddress, localAddress); - super.connect(ctx, remoteAddress, localAddress, promise); - } +// @Override +// public void disconnect(ChannelHandlerContext ctx, ChannelPromise promise) throws Exception { +// trafficOffloader.addDisconnectEvent(Instant.now()); +// METERING_CLOSURE.meterIncrementEvent(telemetryContext, "disconnect"); +// super.disconnect(ctx, promise); +// } - @Override - public void disconnect(ChannelHandlerContext ctx, ChannelPromise promise) throws Exception { - trafficOffloader.addDisconnectEvent(Instant.now()); - super.disconnect(ctx, promise); - } +// @Override +// public void close(ChannelHandlerContext ctx, ChannelPromise promise) throws Exception { +// trafficOffloader.addCloseEvent(Instant.now()); +// +// METERING_CLOSURE.meterIncrementEvent(telemetryContext, "close"); +// METERING_CLOSURE.meterDeltaEvent(telemetryContext, "connections", -1); +// METERING_CLOSURE.meterHistogramMillis(telemetryContext, "connectionDuration"); +// telemetryContext.currentSpan.end(); +// } - @Override - public void close(ChannelHandlerContext ctx, ChannelPromise promise) throws Exception { - trafficOffloader.addCloseEvent(Instant.now()); - super.close(ctx, promise); - } +// @Override +// public void deregister(ChannelHandlerContext ctx, ChannelPromise promise) throws Exception { +// trafficOffloader.addDeregisterEvent(Instant.now()); +// METERING_CLOSURE.meterIncrementEvent(telemetryContext, "deregister"); +// super.deregister(ctx, promise); +// } - @Override - public void deregister(ChannelHandlerContext ctx, ChannelPromise promise) throws Exception { - trafficOffloader.addDeregisterEvent(Instant.now()); - super.deregister(ctx, promise); - } - - @Override - public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) throws Exception { - trafficOffloader.addWriteEvent(Instant.now(), (ByteBuf) msg); - metricsLogger.atSuccess(MetricsEvent.RECEIVED_RESPONSE_COMPONENT) - .setAttribute(MetricsAttributeKey.CHANNEL_ID, ctx.channel().id().asLongText()).emit(); - super.write(ctx, msg, promise); - } - - @Override - public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { - flush(ctx); - super.handlerRemoved(ctx); - } - - @Override - public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { - trafficOffloader.addExceptionCaughtEvent(Instant.now(), cause); - super.exceptionCaught(ctx, cause); - } +// +// @Override +// public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { +// trafficOffloader.addExceptionCaughtEvent(Instant.now(), cause); +// METERING_CLOSURE.meterIncrementEvent(telemetryContext, "exception"); +// super.exceptionCaught(ctx, cause); +// } } diff --git a/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/RequestContextStateMachine.java b/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/RequestContextStateMachine.java new file mode 100644 index 000000000..4a670e8d0 --- /dev/null +++ b/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/RequestContextStateMachine.java @@ -0,0 +1,24 @@ +package org.opensearch.migrations.trafficcapture.netty; + +import lombok.Getter; +import lombok.extern.slf4j.Slf4j; +import org.opensearch.migrations.trafficcapture.netty.tracing.HttpMessageContext; +import org.opensearch.migrations.trafficcapture.tracing.ConnectionContext; + +/** + * This is a helper class so that we can emit metrics and traces for when we're + * accumulating a request vs waiting for the next response, then repeating indefinitely. + * + * TODO - this may be a performance bottleneck and we should carefully evaluate it's utility. + */ +@Slf4j +public class RequestContextStateMachine { + @Getter + public final ConnectionContext connectionContext; + @Getter + HttpMessageContext currentRequestContext; + + public RequestContextStateMachine(ConnectionContext incoming) { + connectionContext = incoming; + } +} diff --git a/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/tracing/HttpMessageContext.java b/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/tracing/HttpMessageContext.java new file mode 100644 index 000000000..e406c4225 --- /dev/null +++ b/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/tracing/HttpMessageContext.java @@ -0,0 +1,41 @@ +package org.opensearch.migrations.trafficcapture.netty.tracing; + +import io.opentelemetry.api.trace.Span; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import org.opensearch.migrations.tracing.ISpanWithParentGenerator; +import org.opensearch.migrations.tracing.IWithStartTimeAndAttributes; +import org.opensearch.migrations.tracing.commoncontexts.IConnectionContext; +import org.opensearch.migrations.tracing.commoncontexts.IRequestContext; +import org.opensearch.migrations.trafficcapture.tracing.ConnectionContext; + +import java.time.Instant; + +public class HttpMessageContext implements IRequestContext, IWithStartTimeAndAttributes { + public enum HttpTransactionState { + REQUEST, + INTERNALLY_BLOCKED, + WAITING, + RESPONSE + } + + @Getter + final long sourceRequestIndex; + @Getter + final ConnectionContext enclosingScope; + @Getter + final Instant startTime; + @Getter + final HttpTransactionState state; + @Getter + final Span currentSpan; + + public HttpMessageContext(ConnectionContext enclosingScope, long sourceRequestIndex, HttpTransactionState state, + ISpanWithParentGenerator spanGenerator) { + this.sourceRequestIndex = sourceRequestIndex; + this.enclosingScope = enclosingScope; + this.startTime = Instant.now(); + this.state = state; + this.currentSpan = spanGenerator.apply(getPopulatedAttributes(), enclosingScope.getCurrentSpan()); + } +} diff --git a/TrafficCapture/nettyWireLogging/src/test/java/org/opensearch/migrations/trafficcapture/netty/ConditionallyReliableLoggingHttpRequestHandlerTest.java b/TrafficCapture/nettyWireLogging/src/test/java/org/opensearch/migrations/trafficcapture/netty/ConditionallyReliableLoggingHttpRequestHandlerTest.java index 292b3fa1b..b5a8936d3 100644 --- a/TrafficCapture/nettyWireLogging/src/test/java/org/opensearch/migrations/trafficcapture/netty/ConditionallyReliableLoggingHttpRequestHandlerTest.java +++ b/TrafficCapture/nettyWireLogging/src/test/java/org/opensearch/migrations/trafficcapture/netty/ConditionallyReliableLoggingHttpRequestHandlerTest.java @@ -4,10 +4,15 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import io.netty.channel.embedded.EmbeddedChannel; +import io.opentelemetry.api.GlobalOpenTelemetry; +import io.opentelemetry.api.metrics.Meter; +import io.opentelemetry.api.trace.Tracer; +import io.opentelemetry.sdk.testing.junit5.OpenTelemetryExtension; import lombok.AllArgsConstructor; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -18,6 +23,7 @@ import org.opensearch.migrations.trafficcapture.CodedOutputStreamHolder; import org.opensearch.migrations.trafficcapture.OrderedStreamLifecyleManager; import org.opensearch.migrations.trafficcapture.StreamChannelConnectionCaptureSerializer; +import org.opensearch.migrations.trafficcapture.tracing.ConnectionContext; import org.opensearch.migrations.trafficcapture.protos.TrafficStream; import java.io.ByteArrayInputStream; @@ -38,20 +44,21 @@ @Slf4j public class ConditionallyReliableLoggingHttpRequestHandlerTest { + @RegisterExtension + static final OpenTelemetryExtension otelTesting = OpenTelemetryExtension.create(); + private final Tracer tracer = otelTesting.getOpenTelemetry().getTracer("test"); + private final Meter meter = otelTesting.getOpenTelemetry().getMeter("test"); static class TestStreamManager extends OrderedStreamLifecyleManager implements AutoCloseable { AtomicReference byteBufferAtomicReference = new AtomicReference<>(); AtomicInteger flushCount = new AtomicInteger(); - ByteArrayOutputStream collectedSerializedTrafficStream = new ByteArrayOutputStream(); @Override - public CodedOutputStreamAndByteBufferWrapper createStream() { - return new CodedOutputStreamAndByteBufferWrapper(1024*1024); - } + public void close() {} @Override - public void close() throws Exception { - collectedSerializedTrafficStream.close(); + public CodedOutputStreamAndByteBufferWrapper createStream() { + return new CodedOutputStreamAndByteBufferWrapper(1024*1024); } @SneakyThrows @@ -68,7 +75,6 @@ public void close() throws Exception { cos.flush(); byteBufferAtomicReference.set(osh.getByteBuffer().flip().asReadOnlyBuffer()); log.trace("byteBufferAtomicReference.get="+byteBufferAtomicReference.get()); - //collectedSerializedTrafficStream.write(byteBufferAtomicReference.get().array()); return CompletableFuture.completedFuture(flushCount.incrementAndGet()); } @@ -81,7 +87,8 @@ private static void writeMessageAndVerify(byte[] fullTrafficBytes, Consumertrue)); // true: block every request + new ConditionallyReliableLoggingHttpRequestHandler("n", "c", (ctx, connectionId) -> offloader, + new RequestCapturePredicate(), x->true)); // true: block every request channelWriter.accept(channel); // we wrote the correct data to the downstream handler/channel @@ -104,6 +111,9 @@ private static void writeMessageAndVerify(byte[] fullTrafficBytes, Consumertrue)); + new ConditionallyReliableLoggingHttpRequestHandler("n", "c", + (ctx, connectionId) -> offloader, headerCapturePredicate, x->true)); getWriter(false, true, SimpleRequests.HEALTH_CHECK.getBytes(StandardCharsets.UTF_8)).accept(channel); channel.close(); var requestBytes = SimpleRequests.HEALTH_CHECK.getBytes(StandardCharsets.UTF_8); @@ -173,7 +184,8 @@ public void testThatHealthCheckCaptureCanBeSuppressed(boolean singleBytes) throw var headerCapturePredicate = new HeaderValueFilteringCapturePredicate(Map.of("user-Agent", ".*uploader.*")); EmbeddedChannel channel = new EmbeddedChannel( - new ConditionallyReliableLoggingHttpRequestHandler(offloader, headerCapturePredicate, x->true)); + new ConditionallyReliableLoggingHttpRequestHandler("n", "c", + (ctx, connectionId) -> offloader, headerCapturePredicate, x->true)); getWriter(singleBytes, true, SimpleRequests.HEALTH_CHECK.getBytes(StandardCharsets.UTF_8)).accept(channel); getWriter(singleBytes, true, SimpleRequests.SMALL_POST.getBytes(StandardCharsets.UTF_8)).accept(channel); var requestBytes = (SimpleRequests.HEALTH_CHECK + SimpleRequests.SMALL_POST).getBytes(StandardCharsets.UTF_8); diff --git a/TrafficCapture/replayerPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/java/org/opensearch/migrations/replay/AddCompressionEncodingTest.java b/TrafficCapture/replayerPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/java/org/opensearch/migrations/replay/AddCompressionEncodingTest.java index 44da58363..3bb3c936b 100644 --- a/TrafficCapture/replayerPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/java/org/opensearch/migrations/replay/AddCompressionEncodingTest.java +++ b/TrafficCapture/replayerPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/java/org/opensearch/migrations/replay/AddCompressionEncodingTest.java @@ -33,8 +33,8 @@ public void addingCompressionRequestHeaderCompressesPayload() throws ExecutionEx var compressingTransformer = new HttpJsonTransformingConsumer( JsonJoltTransformer.newBuilder() .addCannedOperation(JsonJoltTransformBuilder.CANNED_OPERATION.ADD_GZIP) - .build(), null, testPacketCapture, "TEST", - TestRequestKey.getTestConnectionRequestId(0)); + .build(), null, testPacketCapture, + TestRequestKey.getTestConnectionRequestContext(0)); final var payloadPartSize = 511; final var numParts = 1025; diff --git a/TrafficCapture/trafficCaptureProxyServer/build.gradle b/TrafficCapture/trafficCaptureProxyServer/build.gradle index f9f14abe4..81d2281dd 100644 --- a/TrafficCapture/trafficCaptureProxyServer/build.gradle +++ b/TrafficCapture/trafficCaptureProxyServer/build.gradle @@ -13,6 +13,8 @@ configurations { } dependencies { + implementation platform("io.opentelemetry:opentelemetry-bom:1.32.0") + implementation 'org.opensearch.plugin:opensearch-security:2.6.0.0' opensearchSecurityPlugin 'org.opensearch.plugin:opensearch-security:2.6.0.0' implementation files(zipTree("$configurations.opensearchSecurityPlugin.singleFile").matching { @@ -38,9 +40,14 @@ dependencies { implementation group: 'com.beust', name: 'jcommander', version: '1.82' implementation 'com.google.protobuf:protobuf-java:3.22.2' + implementation group: 'io.opentelemetry', name:'opentelemetry-api' + implementation group: 'io.opentelemetry', name: 'opentelemetry-sdk' + testImplementation project(':captureProtobufs') testImplementation testFixtures(project(path: ':testUtilities')) testImplementation testFixtures(project(path: ':captureOffloader')) + + testImplementation group: 'io.opentelemetry', name: 'opentelemetry-sdk-testing' } tasks.withType(Tar){ diff --git a/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/CaptureProxy.java b/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/CaptureProxy.java index 89f7b7cb1..f641ad41e 100644 --- a/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/CaptureProxy.java +++ b/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/CaptureProxy.java @@ -17,6 +17,7 @@ import org.apache.kafka.common.config.SaslConfigs; import org.apache.logging.log4j.core.util.NullOutputStream; import org.opensearch.common.settings.Settings; +import org.opensearch.migrations.tracing.SimpleMeteringClosure; import org.opensearch.migrations.trafficcapture.CodedOutputStreamHolder; import org.opensearch.migrations.trafficcapture.FileConnectionCaptureFactory; import org.opensearch.migrations.trafficcapture.IConnectionCaptureFactory; @@ -48,8 +49,6 @@ import java.util.function.Supplier; import java.util.stream.Stream; -import static org.opensearch.migrations.coreutils.MetricsLogger.initializeOpenTelemetry; - @Slf4j public class CaptureProxy { @@ -187,8 +186,11 @@ private static Settings getSettings(@NonNull String configFile) { private static IConnectionCaptureFactory getNullConnectionCaptureFactory() { System.err.println("No trace log directory specified. Logging to /dev/null"); - return connectionId -> new StreamChannelConnectionCaptureSerializer<>(null, connectionId, + return (ctx,connectionId) -> new StreamChannelConnectionCaptureSerializer<>(null, connectionId, new StreamLifecycleManager<>() { + @Override + public void close() {} + @Override public CodedOutputStreamHolder createStream() { return () -> CodedOutputStream.newInstance(NullOutputStream.getInstance()); @@ -302,7 +304,7 @@ public static void main(String[] args) throws InterruptedException, IOException var backsideUri = convertStringToUri(params.backsideUriString); if (params.otelCollectorEndpoint != null) { - initializeOpenTelemetry("capture-proxy", params.otelCollectorEndpoint); + SimpleMeteringClosure.initializeOpenTelemetry("capture", params.otelCollectorEndpoint); } var sksOp = Optional.ofNullable(params.sslConfigFilePath) diff --git a/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/FrontsideHandler.java b/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/FrontsideHandler.java index da0f8564a..266db7f13 100644 --- a/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/FrontsideHandler.java +++ b/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/FrontsideHandler.java @@ -8,6 +8,10 @@ import io.netty.util.ReferenceCountUtil; import lombok.extern.slf4j.Slf4j; +/** + * TODO - this should be renamed ForwardingHandler. It's the last handler of the front, + * but since it isn't the front of the frontside handlers, this name seems misleading. + */ @Slf4j public class FrontsideHandler extends ChannelInboundHandlerAdapter { diff --git a/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/ProxyChannelInitializer.java b/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/ProxyChannelInitializer.java index 37cc20ce5..d9dc29a63 100644 --- a/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/ProxyChannelInitializer.java +++ b/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/ProxyChannelInitializer.java @@ -5,17 +5,21 @@ import io.netty.handler.codec.http.HttpMethod; import io.netty.handler.codec.http.HttpRequest; import io.netty.handler.ssl.SslHandler; +import org.opensearch.migrations.tracing.IWithAttributes; +import org.opensearch.migrations.tracing.SimpleMeteringClosure; import lombok.NonNull; import org.opensearch.migrations.trafficcapture.IConnectionCaptureFactory; import org.opensearch.migrations.trafficcapture.netty.ConditionallyReliableLoggingHttpRequestHandler; import org.opensearch.migrations.trafficcapture.netty.RequestCapturePredicate; import org.opensearch.migrations.trafficcapture.netty.LoggingHttpResponseHandler; +import org.opensearch.migrations.trafficcapture.tracing.ConnectionContext; import javax.net.ssl.SSLEngine; import java.io.IOException; import java.util.function.Supplier; public class ProxyChannelInitializer extends ChannelInitializer { + static final SimpleMeteringClosure METERING_CLOSURE = new SimpleMeteringClosure("FrontendConnection"); private final IConnectionCaptureFactory connectionCaptureFactory; private final Supplier sslEngineProvider; @@ -46,10 +50,9 @@ protected void initChannel(SocketChannel ch) throws IOException { ch.pipeline().addLast(new SslHandler(sslEngineProvider.get())); } - var offloader = connectionCaptureFactory.createOffloader(ch.id().asLongText()); - ch.pipeline().addLast(new LoggingHttpResponseHandler(offloader)); - ch.pipeline().addLast(new ConditionallyReliableLoggingHttpRequestHandler(offloader, - requestCapturePredicate, this::shouldGuaranteeMessageOffloading)); + var connectionId = ch.id().asLongText(); + ch.pipeline().addLast(new ConditionallyReliableLoggingHttpRequestHandler("n", "c", + connectionCaptureFactory, requestCapturePredicate, this::shouldGuaranteeMessageOffloading)); ch.pipeline().addLast(new FrontsideHandler(backsideConnectionPool)); } } diff --git a/TrafficCapture/trafficCaptureProxyServer/src/main/resources/log4j2.properties b/TrafficCapture/trafficCaptureProxyServer/src/main/resources/log4j2.properties index 6bd32ae07..c35476b38 100644 --- a/TrafficCapture/trafficCaptureProxyServer/src/main/resources/log4j2.properties +++ b/TrafficCapture/trafficCaptureProxyServer/src/main/resources/log4j2.properties @@ -1,4 +1,4 @@ -status = info +status = debug packages = io.opentelemetry.instrumentation.log4j.appender.v2_17 appenders = console, METRICS @@ -18,8 +18,3 @@ appender.METRICS.captureContextDataAttributes = * rootLogger.level = info rootLogger.appenderRefs = stderr rootLogger.appenderRef.stderr.ref = STDERR - -logger.MetricsLogger.name = MetricsLogger -logger.MetricsLogger.level = info -logger.MetricsLogger.additivity = false -logger.MetricsLogger.appenderRef.METRICS.ref = METRICS diff --git a/TrafficCapture/trafficCaptureProxyServer/src/main/resources/logging.properties b/TrafficCapture/trafficCaptureProxyServer/src/main/resources/logging.properties new file mode 100644 index 000000000..42fe83cdd --- /dev/null +++ b/TrafficCapture/trafficCaptureProxyServer/src/main/resources/logging.properties @@ -0,0 +1,9 @@ +# Set the global logging level for all loggers +.level=FINE + +# Configure the console handler (or other handlers if you use them) +handlers=java.util.logging.ConsoleHandler + +# Set the logging level for the console handler +java.util.logging.ConsoleHandler.level=FINE +java.util.logging.ConsoleHandler.formatter=java.util.logging.SimpleFormatter \ No newline at end of file diff --git a/TrafficCapture/trafficReplayer/build.gradle b/TrafficCapture/trafficReplayer/build.gradle index 7ca67b9d0..c50a37d27 100644 --- a/TrafficCapture/trafficReplayer/build.gradle +++ b/TrafficCapture/trafficReplayer/build.gradle @@ -35,6 +35,7 @@ repositories { dependencies { //spotbugs 'com.github.spotbugs:spotbugs:4.7.3' def resilience4jVersion = "1.7.0"; + implementation platform("io.opentelemetry:opentelemetry-bom:1.32.0") implementation project(':captureProtobufs') implementation project(':coreUtilities') @@ -50,6 +51,8 @@ dependencies { implementation group: 'io.github.resilience4j', name: 'resilience4j-ratelimiter', version:"${resilience4jVersion}" implementation group: 'io.github.resilience4j', name: 'resilience4j-retry', version:"${resilience4jVersion}" implementation group: 'io.netty', name: 'netty-all', version: '4.1.100.Final' + implementation group: 'io.opentelemetry', name:'opentelemetry-api' + implementation group: 'io.opentelemetry', name: 'opentelemetry-sdk' implementation group: 'org.apache.kafka', name: 'kafka-clients', version: '3.6.0' implementation group: 'org.apache.logging.log4j', name: 'log4j-api', version: '2.20.0' implementation group: 'org.apache.logging.log4j', name: 'log4j-core', version: '2.20.0' @@ -63,13 +66,17 @@ dependencies { implementation 'org.apache.commons:commons-compress:1.24.0' testFixturesImplementation project(':replayerPlugins:jsonMessageTransformers:jsonMessageTransformerInterface') + testFixturesImplementation project(':coreUtilities') testFixturesImplementation testFixtures(project(path: ':testUtilities')) + testFixturesImplementation platform("io.opentelemetry:opentelemetry-bom:1.32.0") testFixturesImplementation group: 'org.slf4j', name: 'slf4j-api', version: '2.0.7' testFixturesImplementation group: 'com.fasterxml.jackson.core', name: 'jackson-databind', version: '2.15.0' testFixturesImplementation group: 'io.netty', name: 'netty-all', version: '4.1.100.Final' testFixturesImplementation group: 'org.junit.jupiter', name:'junit-jupiter-api', version:'5.9.3' - + testFixturesImplementation group: 'io.opentelemetry', name:'opentelemetry-api' + testFixturesImplementation group: 'io.opentelemetry', name: 'opentelemetry-sdk' + testFixturesImplementation group: 'io.opentelemetry', name: 'opentelemetry-sdk-testing' testImplementation project(':captureOffloader') testImplementation testFixtures(project(path: ':captureOffloader')) @@ -78,6 +85,7 @@ dependencies { testImplementation project(':replayerPlugins:jsonMessageTransformers:jsonJoltMessageTransformerProvider') testImplementation project(':replayerPlugins:jsonMessageTransformers:openSearch23PlusTargetTransformerProvider') + testImplementation group: 'io.opentelemetry', name: 'opentelemetry-sdk-testing' testImplementation group: 'org.apache.httpcomponents.client5', name: 'httpclient5', version: '5.2.1' testImplementation group: 'org.junit.jupiter', name:'junit-jupiter-api', version:'5.x.x' testImplementation group: 'org.testcontainers', name: 'junit-jupiter', version: '1.19.0' diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/Accumulation.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/Accumulation.java index f43440c08..ca239a472 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/Accumulation.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/Accumulation.java @@ -1,9 +1,12 @@ package org.opensearch.migrations.replay; import lombok.NonNull; +import org.opensearch.migrations.tracing.SimpleMeteringClosure; import org.opensearch.migrations.replay.datatypes.ISourceTrafficChannelKey; import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; import org.opensearch.migrations.replay.datatypes.UniqueReplayerRequestKey; +import org.opensearch.migrations.replay.tracing.ChannelKeyContext; +import org.opensearch.migrations.replay.tracing.RequestContext; import org.opensearch.migrations.trafficcapture.protos.TrafficStream; import java.time.Instant; @@ -11,6 +14,8 @@ import java.util.concurrent.atomic.AtomicLong; public class Accumulation { + public static final String TELEMETRY_SCOPE_NAME = "Accumulator"; + public static final SimpleMeteringClosure METERING_CLOSURE = new SimpleMeteringClosure(TELEMETRY_SCOPE_NAME); enum State { // Ignore all initial READs, the first EOM & the following WRITEs (if they or EOMs exist) @@ -23,6 +28,7 @@ enum State { } public final ISourceTrafficChannelKey trafficChannelKey; + public final ChannelKeyContext channelContext; private RequestResponsePacketPair rrPair; AtomicLong newestPacketTimestampInMillis; State state; @@ -46,19 +52,26 @@ public Accumulation(@NonNull ITrafficStreamKey trafficChannelKey, this.startingSourceRequestIndex = startingSourceRequestIndex; this.state = dropObservationsLeftoverFromPrevious ? State.IGNORING_LAST_REQUEST : State.WAITING_FOR_NEXT_READ_CHUNK; + channelContext = new ChannelKeyContext(trafficChannelKey, + METERING_CLOSURE.makeSpanContinuation("processingChannel", null)); } public RequestResponsePacketPair getOrCreateTransactionPair(ITrafficStreamKey forTrafficStreamKey) { if (rrPair != null) { return rrPair; } - rrPair = new RequestResponsePacketPair(forTrafficStreamKey); + this.rrPair = new RequestResponsePacketPair(forTrafficStreamKey, + new RequestContext(channelContext, getRequestKey(forTrafficStreamKey), + METERING_CLOSURE.makeSpanContinuation("accumulatingRequest"))); return rrPair; } public UniqueReplayerRequestKey getRequestKey() { - return new UniqueReplayerRequestKey(getRrPair().getBeginningTrafficStreamKey(), - startingSourceRequestIndex, getIndexOfCurrentRequest()); + return getRequestKey(getRrPair().getBeginningTrafficStreamKey()); + } + + private UniqueReplayerRequestKey getRequestKey(@NonNull ITrafficStreamKey tsk) { + return new UniqueReplayerRequestKey(tsk, startingSourceRequestIndex, getIndexOfCurrentRequest()); } public boolean hasSignaledRequests() { @@ -79,6 +92,14 @@ public boolean hasRrPair() { return rrPair; } + public void rotateRequestGatheringToResponse() { + var ctx = rrPair.requestContext; + ctx.getCurrentSpan().end(); + rrPair.requestContext = new RequestContext(ctx.getEnclosingScope(), + ctx.getReplayerRequestKey(), + METERING_CLOSURE.makeSpanContinuation("accumulatingResponse")); + } + public Instant getLastTimestamp() { return Instant.ofEpochMilli(newestPacketTimestampInMillis.get()); } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/AccumulationCallbacks.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/AccumulationCallbacks.java index c581ffa0c..28d988f3d 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/AccumulationCallbacks.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/AccumulationCallbacks.java @@ -4,17 +4,21 @@ import org.opensearch.migrations.replay.datatypes.ISourceTrafficChannelKey; import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; import org.opensearch.migrations.replay.datatypes.UniqueReplayerRequestKey; +import org.opensearch.migrations.replay.tracing.ChannelKeyContext; +import org.opensearch.migrations.replay.tracing.RequestContext; import java.time.Instant; import java.util.List; public interface AccumulationCallbacks { - void onRequestReceived(@NonNull UniqueReplayerRequestKey key, @NonNull HttpMessageAndTimestamp request); - void onFullDataReceived(@NonNull UniqueReplayerRequestKey key, @NonNull RequestResponsePacketPair rrpp); - void onTrafficStreamsExpired(RequestResponsePacketPair.ReconstructionStatus status, + void onRequestReceived(@NonNull UniqueReplayerRequestKey key, RequestContext ctx, + @NonNull HttpMessageAndTimestamp request); + void onFullDataReceived(@NonNull UniqueReplayerRequestKey key, RequestContext ctx, + @NonNull RequestResponsePacketPair rrpp); + void onTrafficStreamsExpired(RequestResponsePacketPair.ReconstructionStatus status, ChannelKeyContext ctx, @NonNull List trafficStreamKeysBeingHeld); - void onConnectionClose(@NonNull ISourceTrafficChannelKey key, int channelInteractionNumber, + void onConnectionClose(@NonNull ISourceTrafficChannelKey key, int channelInteractionNumber, ChannelKeyContext ctx, RequestResponsePacketPair.ReconstructionStatus status, @NonNull Instant when, @NonNull List trafficStreamKeysBeingHeld); - void onTrafficStreamIgnored(@NonNull ITrafficStreamKey tsk); + void onTrafficStreamIgnored(@NonNull ITrafficStreamKey tsk, ChannelKeyContext ctx); } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/CapturedTrafficToHttpTransactionAccumulator.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/CapturedTrafficToHttpTransactionAccumulator.java index b660abf74..b78a72231 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/CapturedTrafficToHttpTransactionAccumulator.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/CapturedTrafficToHttpTransactionAccumulator.java @@ -145,7 +145,7 @@ public void accept(ITrafficStreamWithKey trafficStreamAndKey) { assert accum.state == Accumulation.State.WAITING_FOR_NEXT_READ_CHUNK || accum.state == Accumulation.State.IGNORING_LAST_REQUEST || trafficStream.getSubStreamCount() == 0; - listener.onTrafficStreamIgnored(tsk); + listener.onTrafficStreamIgnored(tsk, accum.channelContext); } } @@ -223,8 +223,10 @@ private static List getTrafficStreamsHeldByAccum(Accumulation accum.getOrCreateTransactionPair(trafficStreamKey).holdTrafficStream(trafficStreamKey); rotateAccumulationIfNecessary(trafficStreamKey.getConnectionId(), accum); closedConnectionCounter.incrementAndGet(); - listener.onConnectionClose(accum.trafficChannelKey, accum.getIndexOfCurrentRequest(), - RequestResponsePacketPair.ReconstructionStatus.COMPLETE, timestamp, getTrafficStreamsHeldByAccum(accum)); + accum.channelContext.getCurrentSpan().end(); + listener.onConnectionClose(accum.trafficChannelKey, accum.getIndexOfCurrentRequest(), accum.channelContext, + RequestResponsePacketPair.ReconstructionStatus.COMPLETE, + timestamp, getTrafficStreamsHeldByAccum(accum)); return Optional.of(CONNECTION_STATUS.CLOSED); } else if (observation.hasConnectionException()) { accum.getOrCreateTransactionPair(trafficStreamKey).holdTrafficStream(trafficStreamKey); @@ -319,7 +321,9 @@ private Optional handleObservationForWriteState(Accumulation private void handleDroppedRequestForAccumulation(Accumulation accum) { if (accum.hasRrPair()) { - accum.getRrPair().getTrafficStreamsHeld().forEach(listener::onTrafficStreamIgnored); + var rrPair = accum.getRrPair(); + rrPair.getTrafficStreamsHeld().forEach(ts-> + listener.onTrafficStreamIgnored(ts, rrPair.requestContext.getEnclosingScope())); } log.atTrace().setMessage(()->"resetting to forget "+ accum.trafficChannelKey).log(); accum.resetToIgnoreAndForgetCurrentRequest(); @@ -340,7 +344,7 @@ private boolean rotateAccumulationIfNecessary(String connectionId, Accumulation return false; } - private boolean rotateAccumulationOnReadIfNecessary(String connectionId, Accumulation accum) { + private boolean rotateAccumulationOnReadIfNecessary(String connectionId, Accumulation accum) { if (rotateAccumulationIfNecessary(connectionId, accum)) { reusedKeepAliveCounter.incrementAndGet(); return true; @@ -354,26 +358,28 @@ private boolean rotateAccumulationOnReadIfNecessary(String connectionId, Accumu */ private boolean handleEndOfRequest(Accumulation accumulation) { assert accumulation.state == Accumulation.State.ACCUMULATING_READS : "state == " + accumulation.state; - var requestPacketBytes = accumulation.getRrPair().requestData; + var rrPair = accumulation.getRrPair(); + var requestPacketBytes = rrPair.requestData; metricsLogger.atSuccess(MetricsEvent.ACCUMULATED_FULL_CAPTURED_SOURCE_RESPONSE) .setAttribute(MetricsAttributeKey.REQUEST_ID, accumulation.getRequestKey().toString()) .setAttribute(MetricsAttributeKey.CONNECTION_ID, accumulation.getRequestKey().getTrafficStreamKey().getConnectionId()).emit(); assert (requestPacketBytes != null); assert (!requestPacketBytes.hasInProgressSegment()); - listener.onRequestReceived(accumulation.getRequestKey(), requestPacketBytes); + accumulation.rotateRequestGatheringToResponse(); + listener.onRequestReceived(accumulation.getRequestKey(), rrPair.requestContext, requestPacketBytes); accumulation.state = Accumulation.State.ACCUMULATING_WRITES; return true; } - private void handleEndOfResponse(Accumulation accumulation, - RequestResponsePacketPair.ReconstructionStatus status) { + private void handleEndOfResponse(Accumulation accumulation, RequestResponsePacketPair.ReconstructionStatus status) { assert accumulation.state == Accumulation.State.ACCUMULATING_WRITES; metricsLogger.atSuccess(MetricsEvent.ACCUMULATED_FULL_CAPTURED_SOURCE_RESPONSE) .setAttribute(MetricsAttributeKey.REQUEST_ID, accumulation.getRequestKey().toString()) .setAttribute(MetricsAttributeKey.CONNECTION_ID, accumulation.getRequestKey().getTrafficStreamKey().getConnectionId()).emit(); var rrPair = accumulation.getRrPair(); rrPair.completionStatus = status; - listener.onFullDataReceived(accumulation.getRequestKey(), rrPair); + rrPair.requestContext.getCurrentSpan().end(); + listener.onFullDataReceived(accumulation.getRequestKey(), rrPair.requestContext, rrPair); log.atTrace().setMessage("resetting for end of response").log(); accumulation.resetForNextRequest(); } @@ -381,7 +387,8 @@ private void handleEndOfResponse(Accumulation accumulation, public void close() { liveStreams.values().forEach(accum -> { requestsTerminatedUponAccumulatorCloseCounter.incrementAndGet(); - fireAccumulationsCallbacksAndClose(accum, RequestResponsePacketPair.ReconstructionStatus.CLOSED_PREMATURELY); + fireAccumulationsCallbacksAndClose(accum, + RequestResponsePacketPair.ReconstructionStatus.CLOSED_PREMATURELY); }); liveStreams.clear(); } @@ -403,7 +410,7 @@ private void fireAccumulationsCallbacksAndClose(Accumulation accumulation, "for " + accumulation.trafficChannelKey + " assuming an empty server interaction and NOT " + "reproducing this to the target cluster.").log(); if (accumulation.hasRrPair()) { - listener.onTrafficStreamsExpired(status, + listener.onTrafficStreamsExpired(status, accumulation.channelContext, Collections.unmodifiableList(accumulation.getRrPair().trafficStreamKeysBeingHeld)); } return; @@ -418,8 +425,10 @@ private void fireAccumulationsCallbacksAndClose(Accumulation accumulation, } } finally { if (accumulation.hasSignaledRequests()) { + accumulation.channelContext.getCurrentSpan().end(); listener.onConnectionClose(accumulation.trafficChannelKey, accumulation.getIndexOfCurrentRequest(), - status, accumulation.getLastTimestamp(), getTrafficStreamsHeldByAccum(accumulation)); + accumulation.channelContext, status, accumulation.getLastTimestamp(), + getTrafficStreamsHeldByAccum(accumulation)); } } } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/ClientConnectionPool.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/ClientConnectionPool.java index fe6b8a00c..e1eaebac6 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/ClientConnectionPool.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/ClientConnectionPool.java @@ -11,11 +11,13 @@ import io.netty.util.concurrent.DefaultPromise; import io.netty.util.concurrent.DefaultThreadFactory; import io.netty.util.concurrent.Future; +import io.opentelemetry.context.ContextKey; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; +import org.opensearch.migrations.tracing.SimpleMeteringClosure; import org.opensearch.migrations.replay.datahandlers.NettyPacketToHttpConsumer; import org.opensearch.migrations.replay.datatypes.ConnectionReplaySession; -import org.opensearch.migrations.replay.datatypes.ISourceTrafficChannelKey; +import org.opensearch.migrations.replay.tracing.ChannelKeyContext; import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; import org.opensearch.migrations.replay.util.StringTrackableCompletableFuture; @@ -26,6 +28,9 @@ @Slf4j public class ClientConnectionPool { + private static final ContextKey RECORD_ID_KEY = ContextKey.named("recordId"); + public static final String TELEMETRY_SCOPE_NAME = "ClientConnectionPool"; + public static final SimpleMeteringClosure METERING_CLOSURE = new SimpleMeteringClosure(TELEMETRY_SCOPE_NAME); public static final String TARGET_CONNECTION_POOL_NAME = "targetConnectionPool"; private final URI serverUri; @@ -61,17 +66,17 @@ public ConnectionReplaySession load(final String s) { } private DiagnosticTrackableCompletableFuture - getResilientClientChannelProducer(EventLoop eventLoop, String diagnosticLabel) { + getResilientClientChannelProducer(EventLoop eventLoop, ChannelKeyContext connectionContext) { return new AdaptiveRateLimiter() .get(() -> { var clientConnectionChannelCreatedFuture = new StringTrackableCompletableFuture(new CompletableFuture<>(), () -> "waiting for createClientConnection to finish"); var channelFuture = NettyPacketToHttpConsumer.createClientConnection(eventLoop, - sslContext, serverUri, diagnosticLabel); + sslContext, serverUri, connectionContext); channelFuture.addListener(f -> { log.atInfo().setMessage(()-> - "New network connection result for " + diagnosticLabel + "=" + f.isSuccess()).log(); + "New network connection result for " + connectionContext + "=" + f.isSuccess()).log(); if (f.isSuccess()) { clientConnectionChannelCreatedFuture.future.complete(channelFuture); } else { @@ -135,7 +140,7 @@ public void closeConnection(String connId) { } public Future - submitEventualSessionGet(ISourceTrafficChannelKey channelKey, boolean ignoreIfNotPresent) { + submitEventualSessionGet(ChannelKeyContext channelKey, boolean ignoreIfNotPresent, ChannelKeyContext ctx) { ConnectionReplaySession channelFutureAndSchedule = getCachedSession(channelKey, ignoreIfNotPresent); if (channelFutureAndSchedule == null) { @@ -146,19 +151,18 @@ public void closeConnection(String connId) { return channelFutureAndSchedule.eventLoop.submit(() -> { if (channelFutureAndSchedule.getChannelFutureFuture() == null) { channelFutureAndSchedule.setChannelFutureFuture( - getResilientClientChannelProducer(channelFutureAndSchedule.eventLoop, - channelKey.getConnectionId())); + getResilientClientChannelProducer(channelFutureAndSchedule.eventLoop, ctx)); } return channelFutureAndSchedule; }); } @SneakyThrows - public ConnectionReplaySession getCachedSession(ISourceTrafficChannelKey channelKey, boolean dontCreate) { + public ConnectionReplaySession getCachedSession(ChannelKeyContext channelKey, boolean dontCreate) { var crs = dontCreate ? connectionId2ChannelCache.getIfPresent(channelKey.getConnectionId()) : connectionId2ChannelCache.get(channelKey.getConnectionId()); if (crs != null) { - crs.setChannelId(channelKey); + crs.setChannelContext(channelKey); } return crs; } @@ -183,7 +187,7 @@ public ConnectionReplaySession getCachedSession(ISourceTrafficChannelKey channel if (channelAndFutureWork.hasWorkRemaining()) { log.atWarn().setMessage(()->"Work items are still remaining for this connection session" + "(last associated with connection=" + - channelAndFutureWork.getChannelId() + + channelAndFutureWork.getChannelContext() + "). " + channelAndFutureWork.calculateSizeSlowly() + " requests that were enqueued won't be run").log(); } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/PacketConsumerFactory.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/PacketConsumerFactory.java index 3bb96fdee..e87ac3492 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/PacketConsumerFactory.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/PacketConsumerFactory.java @@ -2,7 +2,8 @@ import org.opensearch.migrations.replay.datahandlers.IPacketFinalizingConsumer; import org.opensearch.migrations.replay.datatypes.UniqueReplayerRequestKey; +import org.opensearch.migrations.replay.tracing.RequestContext; public interface PacketConsumerFactory { - IPacketFinalizingConsumer create(UniqueReplayerRequestKey requestKey); + IPacketFinalizingConsumer create(UniqueReplayerRequestKey requestKey, RequestContext context); } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/PacketToTransformingHttpHandlerFactory.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/PacketToTransformingHttpHandlerFactory.java index 29eff701b..005c7896e 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/PacketToTransformingHttpHandlerFactory.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/PacketToTransformingHttpHandlerFactory.java @@ -7,12 +7,14 @@ import org.opensearch.migrations.replay.datatypes.TransformedOutputAndResult; import org.opensearch.migrations.replay.datatypes.TransformedPackets; import org.opensearch.migrations.replay.datatypes.UniqueReplayerRequestKey; +import org.opensearch.migrations.replay.tracing.RequestContext; import org.opensearch.migrations.transform.IAuthTransformerFactory; import org.opensearch.migrations.transform.IJsonTransformer; @Slf4j public class PacketToTransformingHttpHandlerFactory implements PacketConsumerFactory> { + private final IJsonTransformer jsonTransformer; private final IAuthTransformerFactory authTransformerFactory; @@ -25,9 +27,9 @@ public PacketToTransformingHttpHandlerFactory(IJsonTransformer jsonTransformer, @Override public IPacketFinalizingConsumer> - create(UniqueReplayerRequestKey requestKey) { + create(UniqueReplayerRequestKey requestKey, RequestContext requestContext) { log.trace("creating HttpJsonTransformingConsumer"); return new HttpJsonTransformingConsumer<>(jsonTransformer, authTransformerFactory, - new TransformedPacketReceiver(), requestKey.toString(), requestKey); + new TransformedPacketReceiver(), requestContext); } } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/ParsedHttpMessagesAsDicts.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/ParsedHttpMessagesAsDicts.java index ab11edd3e..8f690d8dc 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/ParsedHttpMessagesAsDicts.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/ParsedHttpMessagesAsDicts.java @@ -92,11 +92,6 @@ public ParsedHttpMessagesAsDicts(Optional> sourceRequestOp1, this.targetResponseOp = targetResponseOp4; } - private static MetricsLogBuilder addMetricIfPresent(MetricsLogBuilder metricBuilder, - MetricsAttributeKey key, Optional value) { - return value.map(v -> metricBuilder.setAttribute(key, v)).orElse(metricBuilder); - } - public MetricsLogBuilder buildStatusCodeMetrics(MetricsLogger logger, UniqueSourceRequestKey requestKey) { var builder = logger.atSuccess(MetricsEvent.STATUS_CODE_COMPARISON); return buildStatusCodeMetrics(builder, requestKey); @@ -114,14 +109,14 @@ public static MetricsLogBuilder buildStatusCodeMetrics(MetricsLogBuilder builder var targetStatus = targetResponseOp.map(r -> r.get(STATUS_CODE_KEY)); builder = builder.setAttribute(MetricsAttributeKey.REQUEST_ID, requestKey.getTrafficStreamKey().getConnectionId() + "." + requestKey.getSourceRequestIndex()); - builder = addMetricIfPresent(builder, MetricsAttributeKey.SOURCE_HTTP_STATUS, sourceStatus); - builder = addMetricIfPresent(builder, MetricsAttributeKey.TARGET_HTTP_STATUS, targetStatus); - builder = addMetricIfPresent(builder, MetricsAttributeKey.HTTP_STATUS_MATCH, + builder = MetricsLogBuilder.addMetricIfPresent(builder, MetricsAttributeKey.SOURCE_HTTP_STATUS, sourceStatus); + builder = MetricsLogBuilder.addMetricIfPresent(builder, MetricsAttributeKey.TARGET_HTTP_STATUS, targetStatus); + builder = MetricsLogBuilder.addMetricIfPresent(builder, MetricsAttributeKey.HTTP_STATUS_MATCH, sourceStatus.flatMap(ss -> targetStatus.map(ts -> ss.equals(ts))) .filter(x -> x).map(b -> (Object) 1).or(() -> Optional.of(Integer.valueOf(0)))); - builder = addMetricIfPresent(builder, MetricsAttributeKey.HTTP_METHOD, + builder = MetricsLogBuilder.addMetricIfPresent(builder, MetricsAttributeKey.HTTP_METHOD, sourceResponseOp.map(r -> r.get("Method"))); - builder = addMetricIfPresent(builder, MetricsAttributeKey.HTTP_ENDPOINT, + builder = MetricsLogBuilder.addMetricIfPresent(builder, MetricsAttributeKey.HTTP_ENDPOINT, sourceResponseOp.map(r -> r.get("Request-URI"))); return builder; } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/ReplayEngine.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/ReplayEngine.java index dd823156a..c80fdabfa 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/ReplayEngine.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/ReplayEngine.java @@ -1,7 +1,6 @@ package org.opensearch.migrations.replay; import io.netty.buffer.ByteBuf; -import io.netty.util.concurrent.Future; import io.netty.util.concurrent.ScheduledFuture; import lombok.extern.slf4j.Slf4j; import org.opensearch.migrations.coreutils.MetricsAttributeKey; @@ -10,6 +9,8 @@ import org.opensearch.migrations.replay.datatypes.ISourceTrafficChannelKey; import org.opensearch.migrations.replay.datatypes.IndexedChannelInteraction; import org.opensearch.migrations.replay.datatypes.UniqueReplayerRequestKey; +import org.opensearch.migrations.replay.tracing.ChannelKeyContext; +import org.opensearch.migrations.replay.tracing.RequestContext; import org.opensearch.migrations.replay.traffic.source.BufferedFlowController; import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; @@ -123,19 +124,19 @@ private static void logStartOfWork(Object stringableKey, long newCount, Instant } public DiagnosticTrackableCompletableFuture - scheduleTransformationWork(UniqueReplayerRequestKey requestKey, Instant originalStart, + scheduleTransformationWork(RequestContext requestCtx, Instant originalStart, Supplier> task) { var newCount = totalCountOfScheduledTasksOutstanding.incrementAndGet(); final String label = "processing"; var start = timeShifter.transformSourceTimeToRealTime(originalStart); - logStartOfWork(requestKey, newCount, start, label); - var result = networkSendOrchestrator.scheduleWork(requestKey.trafficStreamKey, + logStartOfWork(requestCtx, newCount, start, label); + var result = networkSendOrchestrator.scheduleWork(requestCtx.getEnclosingScope(), start.minus(EXPECTED_TRANSFORMATION_DURATION), task); - return hookWorkFinishingUpdates(result, originalStart, requestKey, label); + return hookWorkFinishingUpdates(result, originalStart, requestCtx, label); } public DiagnosticTrackableCompletableFuture - scheduleRequest(UniqueReplayerRequestKey requestKey, Instant originalStart, Instant originalEnd, + scheduleRequest(UniqueReplayerRequestKey requestKey, RequestContext ctx, Instant originalStart, Instant originalEnd, int numPackets, Stream packets) { var newCount = totalCountOfScheduledTasksOutstanding.incrementAndGet(); final String label = "request"; @@ -148,16 +149,17 @@ private static void logStartOfWork(Object stringableKey, long newCount, Instant .setAttribute(MetricsAttributeKey.CONNECTION_ID, requestKey.getTrafficStreamKey().getConnectionId()) .setAttribute(MetricsAttributeKey.DELAY_FROM_ORIGINAL_TO_SCHEDULED_START, Duration.between(originalStart, start).toMillis()) .setAttribute(MetricsAttributeKey.SCHEDULED_SEND_TIME, start.toString()).emit(); - var sendResult = networkSendOrchestrator.scheduleRequest(requestKey, start, interval, packets); + var sendResult = networkSendOrchestrator.scheduleRequest(requestKey, ctx, start, interval, packets); return hookWorkFinishingUpdates(sendResult, originalStart, requestKey, label); } - public void closeConnection(ISourceTrafficChannelKey channelKey, int channelInteractionNum, Instant timestamp) { + public void closeConnection(ISourceTrafficChannelKey channelKey, int channelInteractionNum, + ChannelKeyContext ctx, Instant timestamp) { var newCount = totalCountOfScheduledTasksOutstanding.incrementAndGet(); final String label = "close"; var atTime = timeShifter.transformSourceTimeToRealTime(timestamp); logStartOfWork(new IndexedChannelInteraction(channelKey, channelInteractionNum), newCount, atTime, label); - var future = networkSendOrchestrator.scheduleClose(channelKey, channelInteractionNum, atTime); + var future = networkSendOrchestrator.scheduleClose(ctx, channelInteractionNum, atTime); hookWorkFinishingUpdates(future, timestamp, channelKey, label); } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/RequestResponsePacketPair.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/RequestResponsePacketPair.java index dd25c96b7..352c2cda1 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/RequestResponsePacketPair.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/RequestResponsePacketPair.java @@ -4,6 +4,7 @@ import lombok.NonNull; import lombok.extern.slf4j.Slf4j; import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; +import org.opensearch.migrations.replay.tracing.RequestContext; import java.nio.charset.StandardCharsets; import java.time.Instant; @@ -25,9 +26,11 @@ public enum ReconstructionStatus { @NonNull final ITrafficStreamKey firstTrafficStreamKeyForRequest; List trafficStreamKeysBeingHeld; ReconstructionStatus completionStatus; + RequestContext requestContext; - public RequestResponsePacketPair(@NonNull ITrafficStreamKey startingAtTrafficStreamKey) { + public RequestResponsePacketPair(@NonNull ITrafficStreamKey startingAtTrafficStreamKey, RequestContext requestContext) { firstTrafficStreamKeyForRequest = startingAtTrafficStreamKey; + this.requestContext = requestContext; } @NonNull ITrafficStreamKey getBeginningTrafficStreamKey() { diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/RequestSenderOrchestrator.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/RequestSenderOrchestrator.java index 2acf777e4..037670ab1 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/RequestSenderOrchestrator.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/RequestSenderOrchestrator.java @@ -11,6 +11,8 @@ import org.opensearch.migrations.replay.datatypes.IndexedChannelInteraction; import org.opensearch.migrations.replay.datatypes.ChannelTask; import org.opensearch.migrations.replay.datatypes.UniqueReplayerRequestKey; +import org.opensearch.migrations.replay.tracing.ChannelKeyContext; +import org.opensearch.migrations.replay.tracing.RequestContext; import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; import org.opensearch.migrations.replay.util.StringTrackableCompletableFuture; @@ -36,13 +38,13 @@ public RequestSenderOrchestrator(ClientConnectionPool clientConnectionPool) { } public DiagnosticTrackableCompletableFuture - scheduleWork(ISourceTrafficChannelKey channelKey, Instant timestamp, + scheduleWork(ChannelKeyContext ctx, Instant timestamp, Supplier> task) { - var connectionSession = clientConnectionPool.getCachedSession(channelKey, false); + var connectionSession = clientConnectionPool.getCachedSession(ctx, false); var finalTunneledResponse = new StringTrackableCompletableFuture(new CompletableFuture<>(), ()->"waiting for final signal to confirm processing work has finished"); - log.atDebug().setMessage(()->"Scheduling work for "+channelKey+" at time "+timestamp).log(); + log.atDebug().setMessage(()->"Scheduling work for "+ctx.getConnectionId()+" at time "+timestamp).log(); // this method doesn't use the scheduling that scheduleRequest and scheduleClose use because // doing work associated with a connection is considered to be preprocessing work independent // of the underlying network connection itself, so it's fair to be able to do this without @@ -62,30 +64,31 @@ public RequestSenderOrchestrator(ClientConnectionPool clientConnectionPool) { } public DiagnosticTrackableCompletableFuture - scheduleRequest(UniqueReplayerRequestKey requestKey, Instant start, Duration interval, Stream packets) { + scheduleRequest(UniqueReplayerRequestKey requestKey, RequestContext ctx, + Instant start, Duration interval, Stream packets) { var finalTunneledResponse = new StringTrackableCompletableFuture(new CompletableFuture<>(), ()->"waiting for final aggregated response"); log.atDebug().setMessage(()->"Scheduling request for "+requestKey+" at start time "+start).log(); - return asynchronouslyInvokeRunnableToSetupFuture(requestKey.getTrafficStreamKey(), - requestKey.getReplayerRequestIndex(), - false, finalTunneledResponse, - channelFutureAndRequestSchedule-> scheduleSendOnConnectionReplaySession(requestKey, + return asynchronouslyInvokeRunnableToSetupFuture( + ctx.getEnclosingScope(), requestKey.getReplayerRequestIndex(), false, finalTunneledResponse, + channelFutureAndRequestSchedule-> scheduleSendOnConnectionReplaySession(ctx, channelFutureAndRequestSchedule, finalTunneledResponse, start, interval, packets)); } - public StringTrackableCompletableFuture scheduleClose(ISourceTrafficChannelKey channelKey, - int channelInteractionNum, Instant timestamp) { + public StringTrackableCompletableFuture scheduleClose(ChannelKeyContext ctx, int channelInteractionNum, + Instant timestamp) { + var channelKey = ctx.getChannelKey(); var channelInteraction = new IndexedChannelInteraction(channelKey, channelInteractionNum); var finalTunneledResponse = new StringTrackableCompletableFuture(new CompletableFuture<>(), ()->"waiting for final signal to confirm close has finished"); log.atDebug().setMessage(()->"Scheduling CLOSE for "+channelInteraction+" at time "+timestamp).log(); - asynchronouslyInvokeRunnableToSetupFuture(channelKey, channelInteractionNum, true, + asynchronouslyInvokeRunnableToSetupFuture(ctx, channelInteractionNum, true, finalTunneledResponse, channelFutureAndRequestSchedule-> - scheduleOnConnectionReplaySession(channelKey, channelInteractionNum, channelFutureAndRequestSchedule, - finalTunneledResponse, timestamp, + scheduleOnConnectionReplaySession(ctx, channelInteractionNum, + channelFutureAndRequestSchedule, finalTunneledResponse, timestamp, new ChannelTask(ChannelTaskType.CLOSE, () -> { log.trace("Closing client connection " + channelInteraction); clientConnectionPool.closeConnection(channelKey.getConnectionId()); @@ -95,21 +98,20 @@ public StringTrackableCompletableFuture scheduleClose(ISourceTrafficChanne } private DiagnosticTrackableCompletableFuture - asynchronouslyInvokeRunnableToSetupFuture(ISourceTrafficChannelKey channelKey, int channelInteractionNumber, + asynchronouslyInvokeRunnableToSetupFuture(ChannelKeyContext ctx, int channelInteractionNumber, boolean ignoreIfChannelNotPresent, DiagnosticTrackableCompletableFuture finalTunneledResponse, Consumer successFn) { var channelFutureAndScheduleFuture = - clientConnectionPool.submitEventualSessionGet(channelKey, ignoreIfChannelNotPresent); + clientConnectionPool.submitEventualSessionGet(ctx, ignoreIfChannelNotPresent, ctx); channelFutureAndScheduleFuture.addListener(submitFuture->{ if (!submitFuture.isSuccess()) { log.atError().setCause(submitFuture.cause()) - .setMessage(()->channelKey.toString() + " unexpected issue found from a scheduled task") - .log(); + .setMessage(()->ctx + " unexpected issue found from a scheduled task").log(); finalTunneledResponse.future.completeExceptionally(submitFuture.cause()); } else { - log.atTrace().setMessage(()->channelKey.toString() + - " on the channel's thread... getting a ConnectionReplaySession for it").log(); + log.atTrace().setMessage(()->ctx + " on the channel's thread... " + + "getting a ConnectionReplaySession for it").log(); var channelFutureAndRequestSchedule = ((ConnectionReplaySession) submitFuture.get()); if (channelFutureAndRequestSchedule == null) { finalTunneledResponse.future.complete(null); @@ -118,8 +120,8 @@ public StringTrackableCompletableFuture scheduleClose(ISourceTrafficChanne channelFutureAndRequestSchedule.getChannelFutureFuture() .map(channelFutureGetAttemptFuture->channelFutureGetAttemptFuture .thenAccept(v->{ - log.atTrace().setMessage(()->channelKey.toString() + " in submitFuture(success) and scheduling the task" + - " for " + finalTunneledResponse.toString()).log(); + log.atTrace().setMessage(()->ctx + " in submitFuture(success) and " + + "scheduling the task for " + finalTunneledResponse.toString()).log(); assert v.channel() == channelFutureAndRequestSchedule.getChannelFutureFuture().future .getNow(null).channel(); @@ -130,13 +132,13 @@ public StringTrackableCompletableFuture scheduleClose(ISourceTrafficChanne () -> successFn.accept(channelFutureAndRequestSchedule), x -> x.run()); if (replaySession.scheduleSequencer.hasPending()) { - log.atDebug().setMessage(()->"Sequencer for "+channelKey+ + log.atDebug().setMessage(()->"Sequencer for "+ctx+ " = "+replaySession.scheduleSequencer).log(); } }); }) .exceptionally(t->{ - log.atTrace().setCause(t).setMessage(()->channelKey.toString() + + log.atTrace().setCause(t).setMessage(()->ctx + " ChannelFuture creation threw an exception").log(); finalTunneledResponse.future.completeExceptionally(t); return null; @@ -147,11 +149,11 @@ public StringTrackableCompletableFuture scheduleClose(ISourceTrafficChanne return finalTunneledResponse; } - private void scheduleOnConnectionReplaySession(ISourceTrafficChannelKey channelKey, int channelInteractionIdx, + private void scheduleOnConnectionReplaySession(ChannelKeyContext ctx, int channelInteractionIdx, ConnectionReplaySession channelFutureAndRequestSchedule, StringTrackableCompletableFuture futureToBeCompletedByTask, Instant atTime, ChannelTask task) { - var channelInteraction = new IndexedChannelInteraction(channelKey, channelInteractionIdx); + var channelInteraction = new IndexedChannelInteraction(ctx.getChannelKey(), channelInteractionIdx); log.atInfo().setMessage(()->channelInteraction + " scheduling " + task.kind + " at " + atTime).log(); var schedule = channelFutureAndRequestSchedule.schedule; @@ -162,7 +164,7 @@ private void scheduleOnConnectionReplaySession(ISourceTrafficChannelKey chan eventLoop.schedule(task.runnable, getDelayFromNowMs(atTime), TimeUnit.MILLISECONDS); scheduledFuture.addListener(f->{ if (!f.isSuccess()) { - log.atError().setCause(f.cause()).setMessage(()->"Error scheduling task for " + channelKey).log(); + log.atError().setCause(f.cause()).setMessage(()->"Error scheduling task for " + ctx).log(); } else { log.atInfo().setMessage(()->"scheduled future has finished for "+channelInteraction).log(); } @@ -195,17 +197,18 @@ private void scheduleOnConnectionReplaySession(ISourceTrafficChannelKey chan }), ()->""); } - private void scheduleSendOnConnectionReplaySession(UniqueReplayerRequestKey requestKey, + private void scheduleSendOnConnectionReplaySession(RequestContext ctx, ConnectionReplaySession channelFutureAndRequestSchedule, StringTrackableCompletableFuture responseFuture, Instant start, Duration interval, Stream packets) { var eventLoop = channelFutureAndRequestSchedule.eventLoop; var packetReceiverRef = new AtomicReference(); Runnable packetSender = () -> sendNextPartAndContinue(() -> - getPacketReceiver(requestKey, channelFutureAndRequestSchedule.getInnerChannelFuture(), + getPacketReceiver(ctx, channelFutureAndRequestSchedule.getInnerChannelFuture(), packetReceiverRef), eventLoop, packets.iterator(), start, interval, new AtomicInteger(), responseFuture); - scheduleOnConnectionReplaySession(requestKey.trafficStreamKey, requestKey.getSourceRequestIndex(), + scheduleOnConnectionReplaySession(ctx.getEnclosingScope(), + ctx.getReplayerRequestKey().getSourceRequestIndex(), channelFutureAndRequestSchedule, responseFuture, start, new ChannelTask(ChannelTaskType.TRANSMIT, packetSender)); } @@ -234,10 +237,10 @@ private long getDelayFromNowMs(Instant to) { } private static NettyPacketToHttpConsumer - getPacketReceiver(UniqueReplayerRequestKey requestKey, ChannelFuture channelFuture, + getPacketReceiver(RequestContext requestContext, ChannelFuture channelFuture, AtomicReference packetReceiver) { if (packetReceiver.get() == null) { - packetReceiver.set(new NettyPacketToHttpConsumer(channelFuture, requestKey.toString(), requestKey)); + packetReceiver.set(new NettyPacketToHttpConsumer(channelFuture, requestContext)); } return packetReceiver.get(); } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/TrafficReplayer.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/TrafficReplayer.java index c44232a31..e59ef545d 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/TrafficReplayer.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/TrafficReplayer.java @@ -14,7 +14,10 @@ import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import org.opensearch.migrations.coreutils.MetricsLogger; +import org.opensearch.migrations.tracing.SimpleMeteringClosure; import org.opensearch.migrations.replay.datahandlers.IPacketFinalizingConsumer; +import org.opensearch.migrations.replay.tracing.ChannelKeyContext; +import org.opensearch.migrations.replay.tracing.RequestContext; import org.opensearch.migrations.transform.IHttpMessage; import org.opensearch.migrations.replay.datatypes.HttpRequestTransformationStatus; import org.opensearch.migrations.replay.datatypes.ISourceTrafficChannelKey; @@ -67,8 +70,6 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.opensearch.migrations.coreutils.MetricsLogger.initializeOpenTelemetry; - @Slf4j public class TrafficReplayer { @@ -384,7 +385,7 @@ public static void main(String[] args) return; } if (params.otelCollectorEndpoint != null) { - initializeOpenTelemetry("traffic-replayer", params.otelCollectorEndpoint); + SimpleMeteringClosure.initializeOpenTelemetry("replay", params.otelCollectorEndpoint); } try (var blockingTrafficSource = TrafficCaptureSourceFactory.createTrafficCaptureSource(params, @@ -601,11 +602,12 @@ class TrafficReplayerAccumulationCallbacks implements AccumulationCallbacks { private ITrafficCaptureSource trafficCaptureSource; @Override - public void onRequestReceived(UniqueReplayerRequestKey requestKey, HttpMessageAndTimestamp request) { + public void onRequestReceived(UniqueReplayerRequestKey requestKey, RequestContext ctx, + HttpMessageAndTimestamp request) { replayEngine.setFirstTimestamp(request.getFirstPacketTimestamp()); liveTrafficStreamLimiter.addWork(1); - var requestPushFuture = transformAndSendRequest(replayEngine, request, requestKey); + var requestPushFuture = transformAndSendRequest(replayEngine, request, requestKey, ctx); requestFutureMap.put(requestKey, requestPushFuture); liveRequests.put(requestKey, true); requestPushFuture.map(f->f.whenComplete((v,t)->{ @@ -618,7 +620,7 @@ public void onRequestReceived(UniqueReplayerRequestKey requestKey, HttpMessageAn } @Override - public void onFullDataReceived(@NonNull UniqueReplayerRequestKey requestKey, + public void onFullDataReceived(@NonNull UniqueReplayerRequestKey requestKey, RequestContext ctx, @NonNull RequestResponsePacketPair rrPair) { log.atInfo().setMessage(()->"Done receiving captured stream for " + requestKey + ":" + rrPair.requestData).log(); @@ -675,7 +677,7 @@ Void handleCompletedTransaction(@NonNull UniqueReplayerRequestKey requestKey, Re @Override public void onTrafficStreamsExpired(RequestResponsePacketPair.ReconstructionStatus status, - List trafficStreamKeysBeingHeld) { + ChannelKeyContext ctx, List trafficStreamKeysBeingHeld) { commitTrafficStreams(trafficStreamKeysBeingHeld, status); } @@ -697,15 +699,15 @@ private void commitTrafficStreams(List trafficStreamKeysBeing @Override public void onConnectionClose(ISourceTrafficChannelKey channelKey, int channelInteractionNum, - RequestResponsePacketPair.ReconstructionStatus status, Instant timestamp, - List trafficStreamKeysBeingHeld) { + ChannelKeyContext ctx, RequestResponsePacketPair.ReconstructionStatus status, + Instant timestamp, List trafficStreamKeysBeingHeld) { replayEngine.setFirstTimestamp(timestamp); - replayEngine.closeConnection(channelKey, channelInteractionNum, timestamp); + replayEngine.closeConnection(channelKey, channelInteractionNum, ctx, timestamp); commitTrafficStreams(trafficStreamKeysBeingHeld, status); } @Override - public void onTrafficStreamIgnored(@NonNull ITrafficStreamKey tsk) { + public void onTrafficStreamIgnored(@NonNull ITrafficStreamKey tsk, ChannelKeyContext ctx) { commitTrafficStreams(List.of(tsk), true); } @@ -859,29 +861,29 @@ private static String formatWorkItem(DiagnosticTrackableCompletableFuture transformAndSendRequest(ReplayEngine replayEngine, HttpMessageAndTimestamp request, - UniqueReplayerRequestKey requestKey) { - return transformAndSendRequest(inputRequestTransformerFactory, replayEngine, + UniqueReplayerRequestKey requestKey, RequestContext ctx) { + return transformAndSendRequest(inputRequestTransformerFactory, replayEngine, ctx, request.getFirstPacketTimestamp(), request.getLastPacketTimestamp(), requestKey, request.packetBytes::stream); } public static DiagnosticTrackableCompletableFuture transformAndSendRequest(PacketToTransformingHttpHandlerFactory inputRequestTransformerFactory, - ReplayEngine replayEngine, + ReplayEngine replayEngine, RequestContext ctx, @NonNull Instant start, @NonNull Instant end, UniqueReplayerRequestKey requestKey, Supplier> packetsSupplier) { try { - var transformationCompleteFuture = replayEngine.scheduleTransformationWork(requestKey, start, ()-> - transformAllData(inputRequestTransformerFactory.create(requestKey), packetsSupplier)); + var transformationCompleteFuture = replayEngine.scheduleTransformationWork(ctx, start, ()-> + transformAllData(inputRequestTransformerFactory.create(requestKey, ctx), packetsSupplier)); log.atDebug().setMessage(()->"finalizeRequest future for transformation of " + requestKey + " = " + transformationCompleteFuture).log(); // It might be safer to chain this work directly inside the scheduleWork call above so that the // read buffer horizons aren't set after the transformation work finishes, but after the packets // are fully handled return transformationCompleteFuture.thenCompose(transformedResult -> - replayEngine.scheduleRequest(requestKey, start, end, + replayEngine.scheduleRequest(requestKey, ctx, start, end, transformedResult.transformedOutput.size(), transformedResult.transformedOutput.streamRetained()) .map(future->future.thenApply(t-> diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/NettyPacketToHttpConsumer.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/NettyPacketToHttpConsumer.java index 47ea9ad0a..361688ced 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/NettyPacketToHttpConsumer.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/NettyPacketToHttpConsumer.java @@ -18,14 +18,17 @@ import io.netty.handler.logging.LoggingHandler; import io.netty.handler.ssl.SslContext; import io.netty.handler.ssl.SslHandler; +import io.opentelemetry.context.ContextKey; import lombok.extern.slf4j.Slf4j; import org.opensearch.migrations.coreutils.MetricsAttributeKey; import org.opensearch.migrations.coreutils.MetricsEvent; import org.opensearch.migrations.coreutils.MetricsLogger; +import org.opensearch.migrations.tracing.SimpleMeteringClosure; import org.opensearch.migrations.replay.AggregatedRawResponse; -import org.opensearch.migrations.replay.datatypes.UniqueReplayerRequestKey; import org.opensearch.migrations.replay.netty.BacksideHttpWatcherHandler; import org.opensearch.migrations.replay.netty.BacksideSnifferHandler; +import org.opensearch.migrations.replay.tracing.ChannelKeyContext; +import org.opensearch.migrations.replay.tracing.RequestContext; import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; import org.opensearch.migrations.replay.util.StringTrackableCompletableFuture; @@ -36,6 +39,9 @@ @Slf4j public class NettyPacketToHttpConsumer implements IPacketFinalizingConsumer { + public static final String TELEMETRY_SCOPE_NAME = "HttpSender"; + public static final SimpleMeteringClosure METERING_CLOSURE = new SimpleMeteringClosure(TELEMETRY_SCOPE_NAME); + /** * Set this to of(LogLevel.ERROR) or whatever level you'd like to get logging between each handler. * Set this to Optional.empty() to disable intra-handler logging. @@ -52,18 +58,16 @@ public class NettyPacketToHttpConsumer implements IPacketFinalizingConsumer activeChannelFuture; private final Channel channel; AggregatedRawResponse.Builder responseBuilder; - final String diagnosticLabel; - private UniqueReplayerRequestKey uniqueRequestKeyForMetricsLogging; + RequestContext tracingContext; public NettyPacketToHttpConsumer(NioEventLoopGroup eventLoopGroup, URI serverUri, SslContext sslContext, - String diagnosticLabel, UniqueReplayerRequestKey uniqueRequestKeyForMetricsLogging) { - this(createClientConnection(eventLoopGroup, sslContext, serverUri, diagnosticLabel), - diagnosticLabel, uniqueRequestKeyForMetricsLogging); + RequestContext requestContext) { + this(createClientConnection(eventLoopGroup, sslContext, serverUri, requestContext.getEnclosingScope()), + requestContext); } - public NettyPacketToHttpConsumer(ChannelFuture clientConnection, String diagnosticLabel, UniqueReplayerRequestKey uniqueRequestKeyForMetricsLogging) { - this.diagnosticLabel = "[" + diagnosticLabel + "] "; - this.uniqueRequestKeyForMetricsLogging = uniqueRequestKeyForMetricsLogging; + public NettyPacketToHttpConsumer(ChannelFuture clientConnection, RequestContext ctx) { + this.tracingContext = ctx; responseBuilder = AggregatedRawResponse.builder(Instant.now()); DiagnosticTrackableCompletableFuture initialFuture = new StringTrackableCompletableFuture<>(new CompletableFuture<>(), @@ -87,7 +91,7 @@ public NettyPacketToHttpConsumer(ChannelFuture clientConnection, String diagnost } public static ChannelFuture createClientConnection(EventLoopGroup eventLoopGroup, SslContext sslContext, - URI serverUri, String diagnosticLabel) { + URI serverUri, ChannelKeyContext channelKeyContext) { String host = serverUri.getHost(); int port = serverUri.getPort(); log.atTrace().setMessage(()->"Active - setting up backend connection to " + host + ":" + port).log(); @@ -105,8 +109,8 @@ public static ChannelFuture createClientConnection(EventLoopGroup eventLoopGroup if (connectFuture.isSuccess()) { var pipeline = connectFuture.channel().pipeline(); pipeline.removeFirst(); - log.atTrace() - .setMessage(()->diagnosticLabel + " Done setting up client channel & it was successful").log(); + log.atTrace().setMessage(()-> channelKeyContext.getChannelKey() + + " Done setting up client channel & it was successful").log(); if (sslContext != null) { var sslEngine = sslContext.newEngine(connectFuture.channel().alloc()); sslEngine.setUseClientMode(true); @@ -126,7 +130,7 @@ public static ChannelFuture createClientConnection(EventLoopGroup eventLoopGroup } else { // Close the connection if the connection attempt has failed. log.atWarn().setCause(connectFuture.cause()) - .setMessage(() -> diagnosticLabel + " CONNECT future was not successful, " + + .setMessage(() -> channelKeyContext.getChannelKey() + " CONNECT future was not successful, " + "so setting the channel future's result to an exception").log(); rval.setFailure(connectFuture.cause()); } @@ -190,8 +194,8 @@ public DiagnosticTrackableCompletableFuture consumeBytes(ByteBuf pa System.identityHashCode(packetData) + ")").log(); return writePacketAndUpdateFuture(packetData); } else { - log.atWarn().setMessage(()->diagnosticLabel + "outbound channel was not set up successfully, " + - "NOT writing bytes hash=" + System.identityHashCode(packetData)).log(); + log.atWarn().setMessage(()->tracingContext.getReplayerRequestKey() + "outbound channel was not set " + + "up successfully, NOT writing bytes hash=" + System.identityHashCode(packetData)).log(); channel.close(); return DiagnosticTrackableCompletableFuture.Factory.failedFuture(channelInitException, ()->""); } @@ -206,13 +210,14 @@ public DiagnosticTrackableCompletableFuture consumeBytes(ByteBuf pa final var completableFuture = new DiagnosticTrackableCompletableFuture(new CompletableFuture<>(), ()->"CompletableFuture that will wait for the netty future to fill in the completion value"); final int readableBytes = packetData.readableBytes(); + METERING_CLOSURE.meterIncrementEvent(tracingContext, "readBytes", packetData.readableBytes()); channel.writeAndFlush(packetData) .addListener((ChannelFutureListener) future -> { Throwable cause = null; try { if (!future.isSuccess()) { - log.atWarn().setMessage(()->diagnosticLabel + "closing outbound channel because WRITE " + - "future was not successful " + future.cause() + " hash=" + + log.atWarn().setMessage(()->tracingContext.getReplayerRequestKey() + "closing outbound channel " + + "because WRITE future was not successful " + future.cause() + " hash=" + System.identityHashCode(packetData) + " will be sending the exception to " + completableFuture).log(); future.channel().close(); // close the backside @@ -222,17 +227,17 @@ public DiagnosticTrackableCompletableFuture consumeBytes(ByteBuf pa cause = e; } if (cause == null) { - log.atTrace().setMessage(()->"Signaling previously returned CompletableFuture packet write was successful: " - + packetData + " hash=" + System.identityHashCode(packetData)).log(); + log.atTrace().setMessage(()->"Previously returned CompletableFuture packet write was " + + "successful: " + packetData + " hash=" + System.identityHashCode(packetData)).log(); completableFuture.future.complete(null); } else { - log.atInfo().setMessage(()->"Signaling previously returned CompletableFuture packet write had an exception : " - + packetData + " hash=" + System.identityHashCode(packetData)).log(); + log.atInfo().setMessage(()->"Previously returned CompletableFuture packet write had " + + " an exception :" + packetData + " hash=" + System.identityHashCode(packetData)).log(); metricsLogger.atError(MetricsEvent.WRITING_REQUEST_COMPONENT_FAILED, cause) .setAttribute(MetricsAttributeKey.CHANNEL_ID, channel.id().asLongText()) - .setAttribute(MetricsAttributeKey.REQUEST_ID, uniqueRequestKeyForMetricsLogging) + .setAttribute(MetricsAttributeKey.REQUEST_ID, tracingContext.getReplayerRequestKey().toString()) .setAttribute(MetricsAttributeKey.CONNECTION_ID, - uniqueRequestKeyForMetricsLogging.getTrafficStreamKey().getConnectionId()).emit(); + tracingContext.getReplayerRequestKey().getTrafficStreamKey().getConnectionId()).emit(); completableFuture.future.completeExceptionally(cause); channel.close(); } @@ -241,8 +246,9 @@ public DiagnosticTrackableCompletableFuture consumeBytes(ByteBuf pa ". Created future for writing data="+completableFuture).log(); metricsLogger.atSuccess(MetricsEvent.WROTE_REQUEST_COMPONENT) .setAttribute(MetricsAttributeKey.CHANNEL_ID, channel.id().asLongText()) - .setAttribute(MetricsAttributeKey.REQUEST_ID, uniqueRequestKeyForMetricsLogging) - .setAttribute(MetricsAttributeKey.CONNECTION_ID, uniqueRequestKeyForMetricsLogging.getTrafficStreamKey().getConnectionId()) + .setAttribute(MetricsAttributeKey.REQUEST_ID, tracingContext.getReplayerRequestKey()) + .setAttribute(MetricsAttributeKey.CONNECTION_ID, + tracingContext.getConnectionId()) .setAttribute(MetricsAttributeKey.SIZE_IN_BYTES, readableBytes).emit(); return completableFuture; } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/HttpJsonTransformingConsumer.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/HttpJsonTransformingConsumer.java index 1121730df..2076a5529 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/HttpJsonTransformingConsumer.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/HttpJsonTransformingConsumer.java @@ -3,22 +3,27 @@ import io.netty.buffer.ByteBuf; import io.netty.channel.embedded.EmbeddedChannel; import io.netty.handler.codec.http.HttpRequestDecoder; +import io.opentelemetry.api.trace.Span; +import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.opensearch.migrations.coreutils.MetricsAttributeKey; import org.opensearch.migrations.coreutils.MetricsEvent; import org.opensearch.migrations.coreutils.MetricsLogger; +import org.opensearch.migrations.tracing.SimpleMeteringClosure; import org.opensearch.migrations.replay.datatypes.HttpRequestTransformationStatus; import org.opensearch.migrations.replay.datatypes.TransformedOutputAndResult; import org.opensearch.migrations.replay.Utils; import org.opensearch.migrations.replay.datahandlers.IPacketFinalizingConsumer; -import org.opensearch.migrations.replay.datatypes.UniqueReplayerRequestKey; +import org.opensearch.migrations.replay.tracing.RequestContext; import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; import org.opensearch.migrations.replay.util.StringTrackableCompletableFuture; +import org.opensearch.migrations.tracing.IWithStartTimeAndAttributes; import org.opensearch.migrations.transform.IAuthTransformerFactory; import org.opensearch.migrations.transform.IJsonTransformer; import org.slf4j.event.Level; import java.nio.charset.StandardCharsets; +import java.time.Instant; import java.util.ArrayList; import java.util.List; import java.util.Optional; @@ -45,12 +50,15 @@ */ @Slf4j public class HttpJsonTransformingConsumer implements IPacketFinalizingConsumer> { + public static final String TELEMETRY_SCOPE_NAME = "HttpTransformer"; + public static final SimpleMeteringClosure METERING_CLOSURE = new SimpleMeteringClosure(TELEMETRY_SCOPE_NAME); + public static final int HTTP_MESSAGE_NUM_SEGMENTS = 2; public static final int EXPECTED_PACKET_COUNT_GUESS_FOR_HEADERS = 4; private final RequestPipelineOrchestrator pipelineOrchestrator; private final EmbeddedChannel channel; private static final MetricsLogger metricsLogger = new MetricsLogger("HttpJsonTransformingConsumer"); - private UniqueReplayerRequestKey requestKeyForMetricsLogging; + private IWithStartTimeAndAttributes transformationContext; /** * Roughly try to keep track of how big each data chunk was that came into the transformer. These values @@ -67,16 +75,20 @@ public class HttpJsonTransformingConsumer implements IPacketFinalizingConsume public HttpJsonTransformingConsumer(IJsonTransformer transformer, IAuthTransformerFactory authTransformerFactory, IPacketFinalizingConsumer transformedPacketReceiver, - String diagnosticLabel, - UniqueReplayerRequestKey requestKeyForMetricsLogging) { + RequestContext requestContext) { + this.transformationContext = new IWithStartTimeAndAttributes<>() { + @Getter Span currentSpan = METERING_CLOSURE.makeSpanContinuation("httpRequestTransformation") + .apply(requestContext.getPopulatedAttributes(), requestContext.getCurrentSpan()); + @Getter Instant startTime = Instant.now(); + @Override public RequestContext getEnclosingScope() { return requestContext; } + }; chunkSizes = new ArrayList<>(HTTP_MESSAGE_NUM_SEGMENTS); chunkSizes.add(new ArrayList<>(EXPECTED_PACKET_COUNT_GUESS_FOR_HEADERS)); chunks = new ArrayList<>(HTTP_MESSAGE_NUM_SEGMENTS + EXPECTED_PACKET_COUNT_GUESS_FOR_HEADERS); channel = new EmbeddedChannel(); pipelineOrchestrator = new RequestPipelineOrchestrator<>(chunkSizes, transformedPacketReceiver, - authTransformerFactory, diagnosticLabel, requestKeyForMetricsLogging); + authTransformerFactory, requestContext); pipelineOrchestrator.addInitialHandlers(channel.pipeline(), transformer); - this.requestKeyForMetricsLogging = requestKeyForMetricsLogging; } private NettySendByteBufsToPacketHandlerHandler getOffloadingHandler() { @@ -125,7 +137,7 @@ public DiagnosticTrackableCompletableFuture { - if (t != null) { - t = unwindPossibleCompletionException(t); - if (t instanceof NoContentException) { - return redriveWithoutTransformation(offloadingHandler.packetReceiver, t); - } else { - metricsLogger.atError(MetricsEvent.TRANSFORMING_REQUEST_FAILED, t) - .setAttribute(MetricsAttributeKey.REQUEST_ID, requestKeyForMetricsLogging.toString()) - .setAttribute(MetricsAttributeKey.CONNECTION_ID, requestKeyForMetricsLogging.getTrafficStreamKey().getConnectionId()) - .setAttribute(MetricsAttributeKey.CHANNEL_ID, channel.id().asLongText()).emit(); - throw new CompletionException(t); - } - } else { - metricsLogger.atSuccess(MetricsEvent.REQUEST_WAS_TRANSFORMED) - .setAttribute(MetricsAttributeKey.REQUEST_ID, requestKeyForMetricsLogging) - .setAttribute(MetricsAttributeKey.CONNECTION_ID, requestKeyForMetricsLogging.getTrafficStreamKey().getConnectionId()) - .setAttribute(MetricsAttributeKey.CHANNEL_ID, channel.id().asLongText()).emit(); - return StringTrackableCompletableFuture.completedFuture(v, ()->"transformedHttpMessageValue"); - } - }, ()->"HttpJsonTransformingConsumer.finalizeRequest() is waiting to handle"); + (v, t) -> { + transformationContext.getCurrentSpan().end(); + METERING_CLOSURE.meterIncrementEvent(transformationContext, + t != null ? "transformRequestFailed" : "transformRequestSuccess"); + METERING_CLOSURE.meterHistogramMicros(transformationContext, "transformationDuration"); + if (t != null) { + t = unwindPossibleCompletionException(t); + if (t instanceof NoContentException) { + return redriveWithoutTransformation(offloadingHandler.packetReceiver, t); + } else { + metricsLogger.atError(MetricsEvent.TRANSFORMING_REQUEST_FAILED, t) + .setAttribute(MetricsAttributeKey.REQUEST_ID, transformationContext.toString()) + .setAttribute(MetricsAttributeKey.CONNECTION_ID, transformationContext.getEnclosingScope().getConnectionId()) + .setAttribute(MetricsAttributeKey.CHANNEL_ID, channel.id().asLongText()).emit(); + throw new CompletionException(t); + } + } else { + metricsLogger.atSuccess(MetricsEvent.REQUEST_WAS_TRANSFORMED) + .setAttribute(MetricsAttributeKey.REQUEST_ID, transformationContext) + .setAttribute(MetricsAttributeKey.CONNECTION_ID, transformationContext.getEnclosingScope().getConnectionId()) + .setAttribute(MetricsAttributeKey.CHANNEL_ID, channel.id().asLongText()).emit(); + return StringTrackableCompletableFuture.completedFuture(v, ()->"transformedHttpMessageValue"); + } + }, ()->"HttpJsonTransformingConsumer.finalizeRequest() is waiting to handle"); } private static Throwable unwindPossibleCompletionException(Throwable t) { @@ -176,8 +192,8 @@ private static Throwable unwindPossibleCompletionException(Throwable t) { consumptionChainedFuture.thenCompose(v -> packetConsumer.finalizeRequest(), ()->"HttpJsonTransformingConsumer.redriveWithoutTransformation.compose()"); metricsLogger.atError(MetricsEvent.REQUEST_REDRIVEN_WITHOUT_TRANSFORMATION, reason) - .setAttribute(MetricsAttributeKey.REQUEST_ID, requestKeyForMetricsLogging) - .setAttribute(MetricsAttributeKey.CONNECTION_ID, requestKeyForMetricsLogging.getTrafficStreamKey().getConnectionId()) + .setAttribute(MetricsAttributeKey.REQUEST_ID, transformationContext) + .setAttribute(MetricsAttributeKey.CONNECTION_ID, transformationContext.getEnclosingScope().getConnectionId()) .setAttribute(MetricsAttributeKey.CHANNEL_ID, channel.id().asLongText()).emit(); return finalizedFuture.map(f->f.thenApply(r->reason == null ? new TransformedOutputAndResult(r, HttpRequestTransformationStatus.SKIPPED, null) : diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettyDecodedHttpRequestPreliminaryConvertHandler.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettyDecodedHttpRequestPreliminaryConvertHandler.java index 8debf6c1e..2957a7a70 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettyDecodedHttpRequestPreliminaryConvertHandler.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettyDecodedHttpRequestPreliminaryConvertHandler.java @@ -10,7 +10,7 @@ import org.opensearch.migrations.coreutils.MetricsLogger; import org.opensearch.migrations.replay.datahandlers.PayloadAccessFaultingMap; import org.opensearch.migrations.replay.datahandlers.PayloadNotLoadedException; -import org.opensearch.migrations.replay.datatypes.UniqueReplayerRequestKey; +import org.opensearch.migrations.replay.tracing.RequestContext; import org.opensearch.migrations.transform.IAuthTransformer; import org.opensearch.migrations.transform.IJsonTransformer; @@ -27,19 +27,18 @@ public class NettyDecodedHttpRequestPreliminaryConvertHandler extends Channel final IJsonTransformer transformer; final List> chunkSizes; final String diagnosticLabel; - private UniqueReplayerRequestKey requestKeyForMetricsLogging; + private RequestContext requestContext; static final MetricsLogger metricsLogger = new MetricsLogger("NettyDecodedHttpRequestPreliminaryConvertHandler"); public NettyDecodedHttpRequestPreliminaryConvertHandler(IJsonTransformer transformer, List> chunkSizes, RequestPipelineOrchestrator requestPipelineOrchestrator, - String diagnosticLabel, - UniqueReplayerRequestKey requestKeyForMetricsLogging) { + RequestContext requestContext) { this.transformer = transformer; this.chunkSizes = chunkSizes; this.requestPipelineOrchestrator = requestPipelineOrchestrator; - this.diagnosticLabel = "[" + diagnosticLabel + "] "; - this.requestKeyForMetricsLogging = requestKeyForMetricsLogging; + this.diagnosticLabel = "[" + requestContext + "] "; + this.requestContext = requestContext; } @Override @@ -55,8 +54,8 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception .append(request.protocolVersion().text()) .toString()); metricsLogger.atSuccess(MetricsEvent.CAPTURED_REQUEST_PARSED_TO_HTTP) - .setAttribute(MetricsAttributeKey.REQUEST_ID, requestKeyForMetricsLogging) - .setAttribute(MetricsAttributeKey.CONNECTION_ID, requestKeyForMetricsLogging.getTrafficStreamKey().getConnectionId()) + .setAttribute(MetricsAttributeKey.REQUEST_ID, requestContext) + .setAttribute(MetricsAttributeKey.CONNECTION_ID, requestContext.getConnectionId()) .setAttribute(MetricsAttributeKey.HTTP_METHOD, request.method()) .setAttribute(MetricsAttributeKey.HTTP_ENDPOINT, request.uri()).emit(); diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettySendByteBufsToPacketHandlerHandler.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettySendByteBufsToPacketHandlerHandler.java index a0f405c8e..e46763082 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettySendByteBufsToPacketHandlerHandler.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettySendByteBufsToPacketHandlerHandler.java @@ -8,6 +8,7 @@ import org.opensearch.migrations.replay.datatypes.HttpRequestTransformationStatus; import org.opensearch.migrations.replay.datatypes.TransformedOutputAndResult; import org.opensearch.migrations.replay.datahandlers.IPacketFinalizingConsumer; +import org.opensearch.migrations.replay.tracing.RequestContext; import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; import org.opensearch.migrations.replay.util.StringTrackableCompletableFuture; @@ -28,15 +29,15 @@ public class NettySendByteBufsToPacketHandlerHandler extends ChannelInboundHa DiagnosticTrackableCompletableFuture currentFuture; private AtomicReference>> packetReceiverCompletionFutureRef; - String diagnosticLabel; + RequestContext requestContext; public NettySendByteBufsToPacketHandlerHandler(IPacketFinalizingConsumer packetReceiver, - String diagnosticLabel) { + RequestContext requestContext) { this.packetReceiver = packetReceiver; this.packetReceiverCompletionFutureRef = new AtomicReference<>(); - this.diagnosticLabel = diagnosticLabel; + this.requestContext = requestContext; currentFuture = DiagnosticTrackableCompletableFuture.Factory.completedFuture(null, - ()->"currentFuture for NettySendByteBufsToPacketHandlerHandler initialized to the base case for " + diagnosticLabel); + ()->"currentFuture for NettySendByteBufsToPacketHandlerHandler initialized to the base case for " + requestContext); } @Override diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/RequestPipelineOrchestrator.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/RequestPipelineOrchestrator.java index e434efce9..36c72e28b 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/RequestPipelineOrchestrator.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/RequestPipelineOrchestrator.java @@ -10,7 +10,7 @@ import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.opensearch.migrations.replay.datahandlers.IPacketFinalizingConsumer; -import org.opensearch.migrations.replay.datatypes.UniqueReplayerRequestKey; +import org.opensearch.migrations.replay.tracing.RequestContext; import org.opensearch.migrations.transform.IAuthTransformer; import org.opensearch.migrations.transform.IAuthTransformerFactory; import org.opensearch.migrations.transform.IJsonTransformer; @@ -42,22 +42,19 @@ public class RequestPipelineOrchestrator { public static final String HTTP_REQUEST_DECODER_NAME = "HTTP_REQUEST_DECODER"; private final List> chunkSizes; final IPacketFinalizingConsumer packetReceiver; - final String diagnosticLabel; - private UniqueReplayerRequestKey requestKeyForMetricsLogging; + private RequestContext requestContext; @Getter final IAuthTransformerFactory authTransfomerFactory; public RequestPipelineOrchestrator(List> chunkSizes, IPacketFinalizingConsumer packetReceiver, IAuthTransformerFactory incomingAuthTransformerFactory, - String diagnosticLabel, - UniqueReplayerRequestKey requestKeyForMetricsLogging) { + RequestContext requestContext) { this.chunkSizes = chunkSizes; this.packetReceiver = packetReceiver; this.authTransfomerFactory = incomingAuthTransformerFactory != null ? incomingAuthTransformerFactory : IAuthTransformerFactory.NullAuthTransformerFactory.instance; - this.diagnosticLabel = diagnosticLabel; - this.requestKeyForMetricsLogging = requestKeyForMetricsLogging; + this.requestContext = requestContext; } static void removeThisAndPreviousHandlers(ChannelPipeline pipeline, ChannelHandler targetHandler) { @@ -99,8 +96,8 @@ void addInitialHandlers(ChannelPipeline pipeline, IJsonTransformer transformer) // Note3: ByteBufs will be sent through when there were pending bytes left to be parsed by the // HttpRequestDecoder when the HttpRequestDecoder is removed from the pipeline BEFORE the // NettyDecodedHttpRequestHandler is removed. - pipeline.addLast(new NettyDecodedHttpRequestPreliminaryConvertHandler(transformer, chunkSizes, this, - diagnosticLabel, requestKeyForMetricsLogging)); + pipeline.addLast(new NettyDecodedHttpRequestPreliminaryConvertHandler(transformer, chunkSizes, + this, requestContext)); addLoggingHandler(pipeline, "B"); } @@ -150,7 +147,7 @@ void addBaselineHandlers(ChannelPipeline pipeline) { // OUT: nothing - terminal! ByteBufs are routed to the packet handler! addLoggingHandler(pipeline, "K"); pipeline.addLast(OFFLOADING_HANDLER_NAME, - new NettySendByteBufsToPacketHandlerHandler(packetReceiver, diagnosticLabel)); + new NettySendByteBufsToPacketHandlerHandler(packetReceiver, requestContext)); } private void addLoggingHandler(ChannelPipeline pipeline, String name) { diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/ConnectionReplaySession.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/ConnectionReplaySession.java index 642880d58..617767a4d 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/ConnectionReplaySession.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/ConnectionReplaySession.java @@ -6,8 +6,10 @@ import lombok.Setter; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; +import org.opensearch.migrations.replay.tracing.ChannelKeyContext; import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; import org.opensearch.migrations.replay.util.OnlineRadixSorter; +import org.opensearch.migrations.tracing.SimpleMeteringClosure; /** * This class contains everything that is needed to replay packets to a specific channel. @@ -18,6 +20,7 @@ */ @Slf4j public class ConnectionReplaySession { + /** * We need to store this separately from the channelFuture because the channelFuture itself is * vended by a CompletableFuture (e.g. possibly a rate limiter). If the ChannelFuture hasn't @@ -32,7 +35,7 @@ public class ConnectionReplaySession { @Getter @Setter - private ISourceTrafficChannelKey channelId; + private ChannelKeyContext channelContext; public ConnectionReplaySession(EventLoop eventLoop) { this.eventLoop = eventLoop; diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/netty/BacksideHttpWatcherHandler.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/netty/BacksideHttpWatcherHandler.java index a552fe876..7cbb36864 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/netty/BacksideHttpWatcherHandler.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/netty/BacksideHttpWatcherHandler.java @@ -3,7 +3,6 @@ import io.netty.channel.ChannelHandlerContext; import io.netty.channel.SimpleChannelInboundHandler; import io.netty.handler.codec.http.FullHttpResponse; -import lombok.extern.log4j.Log4j2; import lombok.extern.slf4j.Slf4j; import org.opensearch.migrations.coreutils.MetricsAttributeKey; import org.opensearch.migrations.coreutils.MetricsEvent; diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/ChannelKeyContext.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/ChannelKeyContext.java new file mode 100644 index 000000000..419793eca --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/ChannelKeyContext.java @@ -0,0 +1,36 @@ +package org.opensearch.migrations.replay.tracing; + +import io.opentelemetry.api.trace.Span; +import lombok.Getter; +import org.opensearch.migrations.tracing.ISpanGenerator; +import org.opensearch.migrations.replay.datatypes.ISourceTrafficChannelKey; +import org.opensearch.migrations.tracing.commoncontexts.IConnectionContext; + +import java.util.StringJoiner; + +public class ChannelKeyContext implements IConnectionContext { + @Getter + final ISourceTrafficChannelKey channelKey; + @Getter + final Span currentSpan; + + public ChannelKeyContext(ISourceTrafficChannelKey channelKey, ISpanGenerator spanGenerator) { + this.channelKey = channelKey; + this.currentSpan = spanGenerator.apply(getPopulatedAttributes()); + } + + @Override + public String getConnectionId() { + return channelKey.getConnectionId(); + } + + @Override + public String getNodeId() { + return channelKey.getNodeId(); + } + + @Override + public String toString() { + return channelKey.toString(); + } +} diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/IReplayerRequestContext.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/IReplayerRequestContext.java new file mode 100644 index 000000000..3de2e3dc1 --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/IReplayerRequestContext.java @@ -0,0 +1,17 @@ +package org.opensearch.migrations.replay.tracing; + +import io.opentelemetry.api.common.AttributeKey; +import io.opentelemetry.api.common.AttributesBuilder; +import org.opensearch.migrations.tracing.commoncontexts.IRequestContext; + +public interface IReplayerRequestContext extends IRequestContext { + static final AttributeKey REPLAYER_REQUEST_INDEX_KEY = AttributeKey.longKey("replayerRequestIndex"); + + long replayerRequestIndex(); + + @Override + default AttributesBuilder fillAttributes(AttributesBuilder builder) { + return IRequestContext.super.fillAttributes( + builder.put(REPLAYER_REQUEST_INDEX_KEY, replayerRequestIndex())); + } +} diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/RequestContext.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/RequestContext.java new file mode 100644 index 000000000..e90bfdf14 --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/RequestContext.java @@ -0,0 +1,55 @@ +package org.opensearch.migrations.replay.tracing; + +import io.opentelemetry.api.trace.Span; +import lombok.Getter; +import org.opensearch.migrations.tracing.ISpanWithParentGenerator; +import org.opensearch.migrations.replay.datatypes.ISourceTrafficChannelKey; +import org.opensearch.migrations.replay.datatypes.UniqueReplayerRequestKey; +import org.opensearch.migrations.tracing.EmptyContext; +import org.opensearch.migrations.tracing.commoncontexts.IConnectionContext; +import org.opensearch.migrations.tracing.IWithAttributes; +import org.opensearch.migrations.tracing.IWithStartTime; + +import java.time.Instant; + +public class RequestContext implements IReplayerRequestContext, IWithStartTime { + @Getter + ChannelKeyContext enclosingScope; + @Getter + final UniqueReplayerRequestKey replayerRequestKey; + @Getter + final Instant startTime; + @Getter + final Span currentSpan; + + public RequestContext(ChannelKeyContext enclosingScope, UniqueReplayerRequestKey replayerRequestKey, + ISpanWithParentGenerator spanGenerator) { + this.enclosingScope = enclosingScope; + this.replayerRequestKey = replayerRequestKey; + this.startTime = Instant.now(); + this.currentSpan = spanGenerator.apply(getPopulatedAttributes(), enclosingScope.getCurrentSpan()); + } + + public String getConnectionId() { + return enclosingScope.getConnectionId(); + } + + @Override + public long getSourceRequestIndex() { + return replayerRequestKey.getSourceRequestIndex(); + } + + @Override + public long replayerRequestIndex() { + return replayerRequestKey.getReplayerRequestIndex(); + } + + public ISourceTrafficChannelKey getChannelKey() { + return replayerRequestKey.trafficStreamKey; + } + + @Override + public String toString() { + return replayerRequestKey.toString(); + } +} diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/HeaderTransformerTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/HeaderTransformerTest.java index c84987938..69e8ad8b0 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/HeaderTransformerTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/HeaderTransformerTest.java @@ -15,7 +15,6 @@ import java.util.Random; import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Function; import java.util.function.IntFunction; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -35,7 +34,7 @@ public void testTransformer() throws Exception { var testPacketCapture = new TestCapturePacketToHttpHandler(Duration.ofMillis(100), dummyAggregatedResponse); var transformer = new TransformationLoader().getTransformerFactoryLoader(SILLY_TARGET_CLUSTER_NAME); var transformingHandler = new HttpJsonTransformingConsumer(transformer, null, testPacketCapture, - "TEST", TestRequestKey.getTestConnectionRequestId(0)); + TestRequestKey.getTestConnectionRequestContext(0)); runRandomPayloadWithTransformer(transformingHandler, dummyAggregatedResponse, testPacketCapture, contentLength -> "GET / HTTP/1.1\r\n" + "HoSt: " + SOURCE_CLUSTER_NAME + "\r\n" + @@ -87,8 +86,8 @@ public void testMalformedPayloadIsPassedThrough() throws Exception { var httpBasicAuthTransformer = new StaticAuthTransformerFactory("Basic YWRtaW46YWRtaW4="); var transformingHandler = new HttpJsonTransformingConsumer( new TransformationLoader().getTransformerFactoryLoader(SILLY_TARGET_CLUSTER_NAME), - httpBasicAuthTransformer, testPacketCapture, "TEST", - TestRequestKey.getTestConnectionRequestId(0)); + httpBasicAuthTransformer, testPacketCapture, + TestRequestKey.getTestConnectionRequestContext(0)); runRandomPayloadWithTransformer(transformingHandler, dummyAggregatedResponse, testPacketCapture, contentLength -> "GET / HTTP/1.1\r\n" + @@ -114,7 +113,7 @@ public void testMalformedPayload_andTypeMappingUri_IsPassedThrough() throws Exce var transformingHandler = new HttpJsonTransformingConsumer( new TransformationLoader().getTransformerFactoryLoader(SILLY_TARGET_CLUSTER_NAME, null, "[{\"JsonTransformerForOpenSearch23PlusTargetTransformerProvider\":\"\"}]"), - null, testPacketCapture, "TEST", TestRequestKey.getTestConnectionRequestId(0)); + null, testPacketCapture, TestRequestKey.getTestConnectionRequestContext(0)); Random r = new Random(2); var stringParts = IntStream.range(0, 1).mapToObj(i-> TestUtils.makeRandomString(r, 10)).map(o->(String)o) diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/RequestSenderOrchestratorTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/RequestSenderOrchestratorTest.java index 657254c16..a8701076d 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/RequestSenderOrchestratorTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/RequestSenderOrchestratorTest.java @@ -5,7 +5,6 @@ import io.netty.buffer.Unpooled; import io.netty.handler.codec.http.FullHttpResponse; import lombok.extern.slf4j.Slf4j; -import org.junit.Assume; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; @@ -40,20 +39,21 @@ public void testThatSchedulingWorks() throws Exception { Instant lastEndTime = baseTime; var scheduledItems = new ArrayList>(); for (int i = 0; i directives) throws Exception { var connectionFactory = buildSerializerFactory(bufferSize, ()->{}); - var offloader = connectionFactory.createOffloader("TEST_"+uniqueIdCounter.incrementAndGet()); + var offloader = connectionFactory.createOffloader(new ConnectionContext("n", "test", + new SimpleMeteringClosure("test").makeSpanContinuation("test", null)), + "TEST_"+uniqueIdCounter.incrementAndGet()); for (var directive : directives) { serializeEvent(offloader, interactionOffset++, directive); } @@ -211,12 +218,14 @@ void generateAndTest(String testName, int bufferSize, int skipCount, new CapturedTrafficToHttpTransactionAccumulator(Duration.ofSeconds(30), null, new AccumulationCallbacks() { @Override - public void onRequestReceived(UniqueReplayerRequestKey key, HttpMessageAndTimestamp request) { + public void onRequestReceived(UniqueReplayerRequestKey key, RequestContext ctx, + HttpMessageAndTimestamp request) { requestsReceived.incrementAndGet(); } @Override - public void onFullDataReceived(UniqueReplayerRequestKey requestKey, RequestResponsePacketPair fullPair) { + public void onFullDataReceived(UniqueReplayerRequestKey requestKey, RequestContext ctx, + RequestResponsePacketPair fullPair) { var sourceIdx = requestKey.getSourceRequestIndex(); if (fullPair.completionStatus == RequestResponsePacketPair.ReconstructionStatus.CLOSED_PREMATURELY) { @@ -236,16 +245,19 @@ public void onFullDataReceived(UniqueReplayerRequestKey requestKey, RequestRespo @Override public void onTrafficStreamsExpired(RequestResponsePacketPair.ReconstructionStatus status, + ChannelKeyContext ctx, List trafficStreamKeysBeingHeld) {} @Override public void onConnectionClose(ISourceTrafficChannelKey key, int channelInteractionNumber, + ChannelKeyContext ctx, RequestResponsePacketPair.ReconstructionStatus status, Instant when, List trafficStreamKeysBeingHeld) { } - @Override public void onTrafficStreamIgnored(@NonNull ITrafficStreamKey tsk) { + @Override public void onTrafficStreamIgnored(@NonNull ITrafficStreamKey tsk, + ChannelKeyContext ctx) { tsIndicesReceived.add(tsk.getTrafficStreamIndex()); } }); diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/TrafficReplayerTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/TrafficReplayerTest.java index 089880081..a7fdfda40 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/TrafficReplayerTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/TrafficReplayerTest.java @@ -9,6 +9,8 @@ import org.opensearch.migrations.replay.datatypes.ISourceTrafficChannelKey; import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; import org.opensearch.migrations.replay.datatypes.UniqueReplayerRequestKey; +import org.opensearch.migrations.replay.tracing.ChannelKeyContext; +import org.opensearch.migrations.replay.tracing.RequestContext; import org.opensearch.migrations.replay.traffic.source.InputStreamOfTraffic; import org.opensearch.migrations.testutils.WrapWithNettyLeakDetection; import org.opensearch.migrations.trafficcapture.protos.CloseObservation; @@ -154,30 +156,35 @@ public void testReader() throws Exception { new CapturedTrafficToHttpTransactionAccumulator(Duration.ofSeconds(30), null, new AccumulationCallbacks() { @Override - public void onRequestReceived(UniqueReplayerRequestKey id, HttpMessageAndTimestamp request) { + public void onRequestReceived(UniqueReplayerRequestKey id, RequestContext ctx, + HttpMessageAndTimestamp request) { var bytesList = request.stream().collect(Collectors.toList()); byteArrays.add(bytesList); Assertions.assertEquals(FAKE_READ_PACKET_DATA, collectBytesToUtf8String(bytesList)); } @Override - public void onFullDataReceived(UniqueReplayerRequestKey key, RequestResponsePacketPair fullPair) { + public void onFullDataReceived(UniqueReplayerRequestKey key, RequestContext ctx, + RequestResponsePacketPair fullPair) { var responseBytes = fullPair.responseData.packetBytes.stream().collect(Collectors.toList()); Assertions.assertEquals(FAKE_READ_PACKET_DATA, collectBytesToUtf8String(responseBytes)); } @Override public void onTrafficStreamsExpired(RequestResponsePacketPair.ReconstructionStatus status, + ChannelKeyContext ctx, List trafficStreamKeysBeingHeld) {} @Override public void onConnectionClose(ISourceTrafficChannelKey key, int channelInteractionNumber, + ChannelKeyContext ctx, RequestResponsePacketPair.ReconstructionStatus status, Instant when, List trafficStreamKeysBeingHeld) { } - @Override public void onTrafficStreamIgnored(@NonNull ITrafficStreamKey tsk) {} + @Override public void onTrafficStreamIgnored(@NonNull ITrafficStreamKey tsk, + ChannelKeyContext ctx) {} }); var bytes = synthesizeTrafficStreamsIntoByteArray(Instant.now(), 1); @@ -201,29 +208,33 @@ public void testCapturedReadsAfterCloseAreHandledAsNew() throws Exception { "CapturedTrafficToHttpTransactionAccumulator that's being used in this unit test!", new AccumulationCallbacks() { @Override - public void onRequestReceived(UniqueReplayerRequestKey id, HttpMessageAndTimestamp request) { + public void onRequestReceived(UniqueReplayerRequestKey id, RequestContext ctx, + HttpMessageAndTimestamp request) { var bytesList = request.stream().collect(Collectors.toList()); byteArrays.add(bytesList); Assertions.assertEquals(FAKE_READ_PACKET_DATA, collectBytesToUtf8String(bytesList)); } @Override - public void onFullDataReceived(UniqueReplayerRequestKey key, RequestResponsePacketPair fullPair) { + public void onFullDataReceived(UniqueReplayerRequestKey key, RequestContext ctx, + RequestResponsePacketPair fullPair) { var responseBytes = fullPair.responseData.packetBytes.stream().collect(Collectors.toList()); Assertions.assertEquals(FAKE_READ_PACKET_DATA, collectBytesToUtf8String(responseBytes)); } @Override public void onTrafficStreamsExpired(RequestResponsePacketPair.ReconstructionStatus status, + ChannelKeyContext ctx, List trafficStreamKeysBeingHeld) {} @Override public void onConnectionClose(ISourceTrafficChannelKey key, int channelInteractionNumber, - RequestResponsePacketPair.ReconstructionStatus status, + ChannelKeyContext ctx, RequestResponsePacketPair.ReconstructionStatus status, Instant when, List trafficStreamKeysBeingHeld) { } - @Override public void onTrafficStreamIgnored(@NonNull ITrafficStreamKey tsk) {} + @Override public void onTrafficStreamIgnored(@NonNull ITrafficStreamKey tsk, + ChannelKeyContext ctx) {} } ); byte[] serializedChunks; diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datahandlers/NettyPacketToHttpConsumerTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datahandlers/NettyPacketToHttpConsumerTest.java index e60578199..05d27763c 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datahandlers/NettyPacketToHttpConsumerTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datahandlers/NettyPacketToHttpConsumerTest.java @@ -4,6 +4,7 @@ import io.netty.handler.ssl.SslContextBuilder; import io.netty.handler.ssl.util.InsecureTrustManagerFactory; import io.netty.util.concurrent.DefaultThreadFactory; +import io.opentelemetry.api.GlobalOpenTelemetry; import lombok.Lombok; import lombok.extern.slf4j.Slf4j; import org.junit.jupiter.api.AfterAll; @@ -22,6 +23,7 @@ import org.opensearch.migrations.replay.TransformationLoader; import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamKey; import org.opensearch.migrations.replay.datatypes.UniqueReplayerRequestKey; +import org.opensearch.migrations.replay.tracing.RequestContext; import org.opensearch.migrations.replay.traffic.source.BufferedFlowController; import org.opensearch.migrations.testutils.HttpFirstLine; import org.opensearch.migrations.testutils.PortFinder; @@ -128,9 +130,11 @@ public void testHttpResponseIsSuccessfullyCaptured(boolean useTls) throws Except var testServer = testServers.get(useTls); var sslContext = !testServer.localhostEndpoint().getScheme().toLowerCase().equals("https") ? null : SslContextBuilder.forClient().trustManager(InsecureTrustManagerFactory.INSTANCE).build(); - var nphc = new NettyPacketToHttpConsumer(new NioEventLoopGroup(4, new DefaultThreadFactory("test")), - testServer.localhostEndpoint(), sslContext, "unitTest"+i, - TestRequestKey.getTestConnectionRequestId(0)); + var nphc = new NettyPacketToHttpConsumer( + new NioEventLoopGroup(4, new DefaultThreadFactory("test")), + testServer.localhostEndpoint(), + sslContext, + TestRequestKey.getTestConnectionRequestContext(0)); nphc.consumeBytes((EXPECTED_REQUEST_STRING).getBytes(StandardCharsets.UTF_8)); var aggregatedResponse = nphc.finalizeRequest().get(); var responseBytePackets = aggregatedResponse.getCopyOfPackets(); @@ -160,11 +164,9 @@ public void testThatConnectionsAreKeptAliveAndShared(boolean useTls) new TestFlowController(), timeShifter); for (int j=0; j<2; ++j) { for (int i = 0; i < 2; ++i) { - String connId = "TEST_" + j; - var trafficStreamKey = new PojoTrafficStreamKey("testNodeId", connId, 0); + var ctx = TestRequestKey.getTestConnectionRequestContext("TEST_"+i, j); var requestFinishFuture = TrafficReplayer.transformAndSendRequest(transformingHttpHandlerFactory, - sendingFactory, Instant.now(), Instant.now(), - new UniqueReplayerRequestKey(trafficStreamKey, 0, i), + sendingFactory, ctx, Instant.now(), Instant.now(), ctx.getReplayerRequestKey(), ()->Stream.of(EXPECTED_REQUEST_STRING.getBytes(StandardCharsets.UTF_8))); log.info("requestFinishFuture="+requestFinishFuture); var aggregatedResponse = requestFinishFuture.get(); diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datahandlers/http/HttpJsonTransformingConsumerTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datahandlers/http/HttpJsonTransformingConsumerTest.java index ba6ab87f0..50c2aaa44 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datahandlers/http/HttpJsonTransformingConsumerTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datahandlers/http/HttpJsonTransformingConsumerTest.java @@ -27,8 +27,8 @@ public void testPassThroughSinglePacketPost() throws Exception { var transformingHandler = new HttpJsonTransformingConsumer(new TransformationLoader() .getTransformerFactoryLoader(null), - null, testPacketCapture, "TEST", - TestRequestKey.getTestConnectionRequestId(0)); + null, testPacketCapture, + TestRequestKey.getTestConnectionRequestContext(0)); byte[] testBytes; try (var sampleStream = HttpJsonTransformingConsumer.class.getResourceAsStream( "/requests/raw/post_formUrlEncoded_withFixedLength.txt")) { @@ -48,8 +48,8 @@ public void testPassThroughSinglePacketWithoutBodyTransformationPost() throws Ex var transformingHandler = new HttpJsonTransformingConsumer( new TransformationLoader().getTransformerFactoryLoader("test.domain"), - null, testPacketCapture, "TEST", - TestRequestKey.getTestConnectionRequestId(0)); + null, testPacketCapture, + TestRequestKey.getTestConnectionRequestContext(0)); byte[] testBytes; try (var sampleStream = HttpJsonTransformingConsumer.class.getResourceAsStream( "/requests/raw/post_formUrlEncoded_withFixedLength.txt")) { @@ -73,9 +73,8 @@ public void testRemoveAuthHeadersWorks() throws Exception { var transformingHandler = new HttpJsonTransformingConsumer( new TransformationLoader().getTransformerFactoryLoader("test.domain"), - RemovingAuthTransformerFactory.instance, - testPacketCapture, "TEST", - TestRequestKey.getTestConnectionRequestId(0)); + RemovingAuthTransformerFactory.instance, testPacketCapture, + TestRequestKey.getTestConnectionRequestContext(0)); byte[] testBytes; try (var sampleStream = HttpJsonTransformingConsumer.class.getResourceAsStream( "/requests/raw/get_withAuthHeader.txt")) { @@ -114,7 +113,7 @@ private void walkMaps(Object o) { }); var transformingHandler = new HttpJsonTransformingConsumer(complexTransformer, null, - testPacketCapture, "TEST", TestRequestKey.getTestConnectionRequestId(0)); + testPacketCapture, TestRequestKey.getTestConnectionRequestContext(0)); byte[] testBytes; try (var sampleStream = HttpJsonTransformingConsumer.class.getResourceAsStream( "/requests/raw/post_formUrlEncoded_withFixedLength.txt")) { diff --git a/TrafficCapture/trafficReplayer/src/test/resources/log4j2.properties b/TrafficCapture/trafficReplayer/src/test/resources/log4j2.properties index 9098da413..43e08b306 100644 --- a/TrafficCapture/trafficReplayer/src/test/resources/log4j2.properties +++ b/TrafficCapture/trafficReplayer/src/test/resources/log4j2.properties @@ -18,3 +18,8 @@ logger.OutputTupleJsonLogger.level = OFF logger.KPC.name = org.opensearch.migrations.replay.kafka.KafkaProtobufConsumer logger.KPC.level = DEBUG logger.KPC.appenderRef.stdout.ref = Console + +logger.RSO.name = org.opensearch.migrations.replay.RequestSenderOrchestrator +logger.RSO.level = TRACE +logger.RSO.additivity = false +logger.RSO.appenderRef.RSO.ref = Console diff --git a/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/TestRequestKey.java b/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/TestRequestKey.java index 5751cb4aa..a9b232ae4 100644 --- a/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/TestRequestKey.java +++ b/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/TestRequestKey.java @@ -1,15 +1,28 @@ package org.opensearch.migrations.replay; +import org.opensearch.migrations.tracing.SimpleMeteringClosure; import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamKey; import org.opensearch.migrations.replay.datatypes.UniqueReplayerRequestKey; +import org.opensearch.migrations.replay.tracing.ChannelKeyContext; +import org.opensearch.migrations.replay.tracing.RequestContext; public class TestRequestKey { + public static final String TEST_NODE_ID = "testNodeId"; + public static final String DEFAULT_TEST_CONNECTION = "testConnection"; + private TestRequestKey() {} - public static final UniqueReplayerRequestKey getTestConnectionRequestId(int replayerIdx) { - return new UniqueReplayerRequestKey( - new PojoTrafficStreamKey("testNodeId", "testConnectionId", 0), + public static final RequestContext getTestConnectionRequestContext(int replayerIdx) { + return getTestConnectionRequestContext(DEFAULT_TEST_CONNECTION, replayerIdx); + } + + public static final RequestContext getTestConnectionRequestContext(String connectionId, int replayerIdx) { + var rk = new UniqueReplayerRequestKey( + new PojoTrafficStreamKey(TEST_NODE_ID, connectionId, 0), 0, replayerIdx); + var smc = new SimpleMeteringClosure("test"); + var channelKeyContext = new ChannelKeyContext(rk.trafficStreamKey, smc.makeSpanContinuation("test", null)); + return new RequestContext(channelKeyContext, rk, smc.makeSpanContinuation("test2")); } } diff --git a/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/TestUtils.java b/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/TestUtils.java index 77322ffa6..3439bbf8f 100644 --- a/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/TestUtils.java +++ b/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/TestUtils.java @@ -141,7 +141,7 @@ static void runPipelineAndValidate(IJsonTransformer transformer, var testPacketCapture = new TestCapturePacketToHttpHandler(Duration.ofMillis(100), new AggregatedRawResponse(-1, Duration.ZERO, new ArrayList<>(), null)); var transformingHandler = new HttpJsonTransformingConsumer<>(transformer, authTransformer, testPacketCapture, - "TEST", TestRequestKey.getTestConnectionRequestId(0)); + TestRequestKey.getTestConnectionRequestContext("TEST_CONNECTION", 0)); var contentLength = stringParts.stream().mapToInt(String::length).sum(); var headerString = "GET / HTTP/1.1\r\n" +