diff --git a/TrafficCapture/captureKafkaOffloader/build.gradle b/TrafficCapture/captureKafkaOffloader/build.gradle index 11f4e3a06..82af8c0c0 100644 --- a/TrafficCapture/captureKafkaOffloader/build.gradle +++ b/TrafficCapture/captureKafkaOffloader/build.gradle @@ -9,20 +9,22 @@ repositories { } dependencies { - api 'io.netty:netty-buffer:4.1.100.Final' 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: '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 testFixtures(project(path: ':coreUtilities')) 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..a207a01ba 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 @@ -7,17 +7,22 @@ import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; import org.opensearch.migrations.coreutils.MetricsAttributeKey; import org.opensearch.migrations.coreutils.MetricsEvent; -import org.apache.kafka.clients.producer.RecordMetadata; +import org.opensearch.migrations.coreutils.MetricsLogger; +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.IRootKafkaOffloaderContext; +import org.opensearch.migrations.trafficcapture.kafkaoffloader.tracing.KafkaRecordContext; +import java.io.IOException; import java.nio.ByteBuffer; +import java.time.Instant; import java.util.Arrays; import java.util.concurrent.CompletableFuture; @@ -32,27 +37,31 @@ public class KafkaCaptureFactory implements IConnectionCaptureFactory producer; private final String topicNameForTraffic; private final int bufferSize; - public KafkaCaptureFactory(String nodeId, Producer producer, + public KafkaCaptureFactory(IRootKafkaOffloaderContext rootScope, String nodeId, Producer producer, String topicNameForTraffic, int messageSize) { + this.rootScope = rootScope; this.nodeId = nodeId; this.producer = producer; this.topicNameForTraffic = topicNameForTraffic; this.bufferSize = messageSize - KAFKA_MESSAGE_OVERHEAD_BYTES; } - public KafkaCaptureFactory(String nodeId, Producer producer, int messageSize) { - this(nodeId, producer, DEFAULT_TOPIC_NAME_FOR_TRAFFIC, messageSize); + public KafkaCaptureFactory(IRootKafkaOffloaderContext rootScope, String nodeId, Producer producer, int messageSize) { + this(rootScope, nodeId, producer, DEFAULT_TOPIC_NAME_FOR_TRAFFIC, messageSize); } @Override - public IChannelConnectionCaptureSerializer createOffloader(String connectionId) { - return new StreamChannelConnectionCaptureSerializer<>(nodeId, connectionId, new StreamManager(connectionId)); + public IChannelConnectionCaptureSerializer + createOffloader(IConnectionContext ctx) { + return new StreamChannelConnectionCaptureSerializer<>(nodeId, ctx.getConnectionId(), + new StreamManager(rootScope, ctx)); } @AllArgsConstructor @@ -65,12 +74,22 @@ static class CodedOutputStreamWrapper implements CodedOutputStreamHolder { } } - @AllArgsConstructor class StreamManager extends OrderedStreamLifecyleManager { - String connectionId; + IConnectionContext telemetryContext; + IRootKafkaOffloaderContext rootScope; + Instant startTime; + + public StreamManager(IRootKafkaOffloaderContext rootScope, IConnectionContext ctx) { + // TODO - add https://opentelemetry.io/blog/2022/instrument-kafka-clients/ + this.rootScope = rootScope; + this.telemetryContext = ctx; + this.startTime = Instant.now(); + } @Override public CodedOutputStreamWrapper createStream() { + telemetryContext.getCurrentSpan().addEvent("streamCreated"); + ByteBuffer bb = ByteBuffer.allocate(bufferSize); return new CodedOutputStreamWrapper(CodedOutputStream.newInstance(bb), bb); } @@ -84,7 +103,7 @@ public CodedOutputStreamWrapper createStream() { } var osh = (CodedOutputStreamWrapper) outputStreamHolder; - // Structured context for MetricsLogger + final var connectionId = telemetryContext.getConnectionId(); try { String recordId = String.format("%s.%d", connectionId, index); var byteBuffer = osh.byteBuffer; @@ -93,8 +112,12 @@ 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 = rootScope.createKafkaRecordContext(telemetryContext, + topicNameForTraffic, recordId, kafkaRecord.value().length); + // 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 +131,35 @@ 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(); + if (exception != null) { + flushContext.addException(exception); + 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); + } + flushContext.close(); + }; + } } diff --git a/TrafficCapture/captureKafkaOffloader/src/main/java/org/opensearch/migrations/trafficcapture/kafkaoffloader/tracing/IRootKafkaOffloaderContext.java b/TrafficCapture/captureKafkaOffloader/src/main/java/org/opensearch/migrations/trafficcapture/kafkaoffloader/tracing/IRootKafkaOffloaderContext.java new file mode 100644 index 000000000..b9221b901 --- /dev/null +++ b/TrafficCapture/captureKafkaOffloader/src/main/java/org/opensearch/migrations/trafficcapture/kafkaoffloader/tracing/IRootKafkaOffloaderContext.java @@ -0,0 +1,15 @@ +package org.opensearch.migrations.trafficcapture.kafkaoffloader.tracing; + +import org.opensearch.migrations.tracing.IInstrumentConstructor; +import org.opensearch.migrations.tracing.commoncontexts.IConnectionContext; + +public interface IRootKafkaOffloaderContext extends IInstrumentConstructor { + KafkaRecordContext.MetricInstruments getKafkaOffloadingInstruments(); + + default KafkaRecordContext createKafkaRecordContext(IConnectionContext telemetryContext, + String topicNameForTraffic, + String recordId, + int length) { + return new KafkaRecordContext(this, telemetryContext, topicNameForTraffic, recordId, length); + } +} 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..98233036f --- /dev/null +++ b/TrafficCapture/captureKafkaOffloader/src/main/java/org/opensearch/migrations/trafficcapture/kafkaoffloader/tracing/KafkaRecordContext.java @@ -0,0 +1,60 @@ +package org.opensearch.migrations.trafficcapture.kafkaoffloader.tracing; + +import io.opentelemetry.api.common.AttributeKey; +import io.opentelemetry.api.common.AttributesBuilder; +import io.opentelemetry.api.metrics.Meter; +import lombok.Getter; +import lombok.NonNull; +import org.opensearch.migrations.tracing.BaseNestedSpanContext; +import org.opensearch.migrations.tracing.CommonScopedMetricInstruments; +import org.opensearch.migrations.tracing.IScopedInstrumentationAttributes; +import org.opensearch.migrations.tracing.commoncontexts.IConnectionContext; + +public class KafkaRecordContext extends + BaseNestedSpanContext + implements IScopedInstrumentationAttributes { + public static final String ACTIVITY_NAME = "kafkaCommit"; + + 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 String topic; + @Getter + public final String recordId; + + public KafkaRecordContext(IRootKafkaOffloaderContext rootScope, IConnectionContext enclosingScope, + String topic, String recordId, int recordSize) { + super(rootScope, enclosingScope); + this.topic = topic; + this.recordId = recordId; + initializeSpan(); + getCurrentSpan().setAttribute(RECORD_SIZE_ATTR, recordSize); + } + + public static class MetricInstruments extends CommonScopedMetricInstruments { + private MetricInstruments(Meter meter, String activityName) { + super(meter, activityName); + } + } + + public static @NonNull MetricInstruments makeMetrics(Meter meter) { + return new MetricInstruments(meter, ACTIVITY_NAME); + } + + @Override + public @NonNull MetricInstruments getMetrics() { + return getRootInstrumentationScope().getKafkaOffloadingInstruments(); + } + + @Override + public String getActivityName() { return "stream_flush_called"; } + + @Override + public AttributesBuilder fillAttributesForSpansBelow(AttributesBuilder builder) { + return super.fillAttributesForSpansBelow(builder) + .put(TOPIC_ATTR, getTopic()) + .put(RECORD_ID_ATTR, getRecordId()); + } +} 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..2c5e2c0c2 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 @@ -18,7 +18,8 @@ import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; -import org.opensearch.migrations.trafficcapture.IChannelConnectionCaptureSerializer; +import org.opensearch.migrations.trafficcapture.kafkaoffloader.tracing.TestRootKafkaOffloaderContext; +import org.opensearch.migrations.trafficcapture.tracing.ConnectionContext; import java.io.IOException; import java.nio.charset.StandardCharsets; @@ -26,7 +27,6 @@ import java.time.Instant; import java.util.ArrayList; import java.util.List; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import static org.mockito.ArgumentMatchers.any; @@ -42,7 +42,6 @@ public class KafkaCaptureFactoryTest { private String connectionId = "0242c0fffea82008-0000000a-00000003-62993a3207f92af6-9093ce33"; private String topic = "test_topic"; - @Test public void testLargeRequestIsWithinKafkaMessageSizeLimit() throws IOException, ExecutionException, InterruptedException { final var referenceTimestamp = Instant.now(Clock.systemUTC()); @@ -50,18 +49,17 @@ public void testLargeRequestIsWithinKafkaMessageSizeLimit() throws IOException, int maxAllowableMessageSize = 1024*1024; MockProducer producer = new MockProducer<>(true, new StringSerializer(), new ByteArraySerializer()); KafkaCaptureFactory kafkaCaptureFactory = - new KafkaCaptureFactory(TEST_NODE_ID_STRING, producer, maxAllowableMessageSize); - IChannelConnectionCaptureSerializer serializer = kafkaCaptureFactory.createOffloader(connectionId); - - StringBuilder sb = new StringBuilder(); - for (int i = 0; i < 15000; i++) { - sb.append("{ \"create\": { \"_index\": \"office-index\" } }\n{ \"title\": \"Malone's Cones\", \"year\": 2013 }\n"); - } - Assertions.assertTrue(sb.toString().getBytes().length > 1024*1024); - byte[] fakeDataBytes = sb.toString().getBytes(StandardCharsets.UTF_8); + new KafkaCaptureFactory(TestRootKafkaOffloaderContext.noTracking(), + TEST_NODE_ID_STRING, producer, maxAllowableMessageSize); + var serializer = kafkaCaptureFactory.createOffloader(createCtx()); + + var testStr = "{ \"create\": { \"_index\": \"office-index\" } }\n{ \"title\": \"Malone's Cones\", \"year\": 2013 }\n" + .repeat(15000); + var fakeDataBytes = testStr.getBytes(StandardCharsets.UTF_8); + Assertions.assertTrue(fakeDataBytes.length > 1024*1024); var bb = Unpooled.wrappedBuffer(fakeDataBytes); serializer.addReadEvent(referenceTimestamp, bb); - CompletableFuture future = serializer.flushCommitAndResetStream(true); + var future = serializer.flushCommitAndResetStream(true); future.get(); for (ProducerRecord record : producer.history()) { int recordSize = calculateRecordSize(record, null); @@ -73,6 +71,10 @@ public void testLargeRequestIsWithinKafkaMessageSizeLimit() throws IOException, producer.close(); } + private static ConnectionContext createCtx() { + return new ConnectionContext(new TestRootKafkaOffloaderContext(), "test", "test"); + } + /** * 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 @@ -96,8 +98,9 @@ private int calculateRecordSize(ProducerRecord record, String re @Test public void testLinearOffloadingIsSuccessful() throws IOException { KafkaCaptureFactory kafkaCaptureFactory = - new KafkaCaptureFactory(TEST_NODE_ID_STRING, mockProducer, 1024*1024); - IChannelConnectionCaptureSerializer offloader = kafkaCaptureFactory.createOffloader(connectionId); + new KafkaCaptureFactory(TestRootKafkaOffloaderContext.noTracking(), + TEST_NODE_ID_STRING, mockProducer, 1024*1024); + var offloader = kafkaCaptureFactory.createOffloader(createCtx()); List recordSentCallbacks = new ArrayList<>(3); when(mockProducer.send(any(), any())).thenAnswer(invocation -> { @@ -112,11 +115,11 @@ public void testLinearOffloadingIsSuccessful() throws IOException { byte[] fakeDataBytes = "FakeData".getBytes(StandardCharsets.UTF_8); var bb = Unpooled.wrappedBuffer(fakeDataBytes); offloader.addReadEvent(ts, bb); - CompletableFuture cf1 = offloader.flushCommitAndResetStream(false); + var cf1 = offloader.flushCommitAndResetStream(false); offloader.addReadEvent(ts, bb); - CompletableFuture cf2 = offloader.flushCommitAndResetStream(false); + var cf2 = offloader.flushCommitAndResetStream(false); offloader.addReadEvent(ts, bb); - CompletableFuture cf3 = offloader.flushCommitAndResetStream(false); + var cf3 = offloader.flushCommitAndResetStream(false); bb.release(); Assertions.assertEquals(false, cf1.isDone()); diff --git a/TrafficCapture/captureKafkaOffloader/src/test/java/org/opensearch/migrations/trafficcapture/kafkaoffloader/tracing/TestRootKafkaOffloaderContext.java b/TrafficCapture/captureKafkaOffloader/src/test/java/org/opensearch/migrations/trafficcapture/kafkaoffloader/tracing/TestRootKafkaOffloaderContext.java new file mode 100644 index 000000000..00029d717 --- /dev/null +++ b/TrafficCapture/captureKafkaOffloader/src/test/java/org/opensearch/migrations/trafficcapture/kafkaoffloader/tracing/TestRootKafkaOffloaderContext.java @@ -0,0 +1,39 @@ +package org.opensearch.migrations.trafficcapture.kafkaoffloader.tracing; + +import io.opentelemetry.sdk.testing.exporter.InMemoryMetricExporter; +import io.opentelemetry.sdk.testing.exporter.InMemorySpanExporter; +import lombok.Getter; +import org.opensearch.migrations.tracing.InMemoryInstrumentationBundle; +import org.opensearch.migrations.tracing.RootOtelContext; +import org.opensearch.migrations.trafficcapture.tracing.ConnectionContext; +import org.opensearch.migrations.trafficcapture.tracing.IRootOffloaderContext; + +public class TestRootKafkaOffloaderContext extends RootOtelContext implements IRootOffloaderContext, IRootKafkaOffloaderContext { + @Getter + public final KafkaRecordContext.MetricInstruments kafkaOffloadingInstruments; + @Getter + public final ConnectionContext.MetricInstruments connectionInstruments; + + private final InMemoryInstrumentationBundle inMemoryInstrumentationBundle; + + public static TestRootKafkaOffloaderContext withTracking() { + return new TestRootKafkaOffloaderContext(new InMemoryInstrumentationBundle(true, true)); + } + + public static TestRootKafkaOffloaderContext noTracking() { + return new TestRootKafkaOffloaderContext(); + } + + public TestRootKafkaOffloaderContext() { + this(new InMemoryInstrumentationBundle(null, null)); + } + + public TestRootKafkaOffloaderContext(InMemoryInstrumentationBundle inMemoryInstrumentationBundle) { + super("tests", inMemoryInstrumentationBundle == null ? null : + inMemoryInstrumentationBundle.openTelemetrySdk); + this.inMemoryInstrumentationBundle = inMemoryInstrumentationBundle; + final var meter = getMeterProvider().get("test"); + this.kafkaOffloadingInstruments = KafkaRecordContext.makeMetrics(meter); + this.connectionInstruments = ConnectionContext.makeMetrics(meter); + } +} diff --git a/TrafficCapture/captureOffloader/build.gradle b/TrafficCapture/captureOffloader/build.gradle index c375d39a2..cdc0350c6 100644 --- a/TrafficCapture/captureOffloader/build.gradle +++ b/TrafficCapture/captureOffloader/build.gradle @@ -23,16 +23,18 @@ dependencies { 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: '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 project(':coreUtilities') 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' - 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/CodedOutputStreamAndByteBufferWrapper.java b/TrafficCapture/captureOffloader/src/main/java/org/opensearch/migrations/trafficcapture/CodedOutputStreamAndByteBufferWrapper.java index 46927afe6..7cbd0bf52 100644 --- a/TrafficCapture/captureOffloader/src/main/java/org/opensearch/migrations/trafficcapture/CodedOutputStreamAndByteBufferWrapper.java +++ b/TrafficCapture/captureOffloader/src/main/java/org/opensearch/migrations/trafficcapture/CodedOutputStreamAndByteBufferWrapper.java @@ -7,9 +7,11 @@ import java.nio.ByteBuffer; public class CodedOutputStreamAndByteBufferWrapper implements CodedOutputStreamHolder { - @NonNull @Getter + @NonNull + @Getter private final CodedOutputStream outputStream; - @NonNull @Getter + @NonNull + @Getter private final ByteBuffer byteBuffer; public CodedOutputStreamAndByteBufferWrapper(int bufferSize) { 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..38a0a3bf7 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,7 @@ import lombok.AllArgsConstructor; import lombok.Lombok; import lombok.extern.slf4j.Slf4j; +import org.opensearch.migrations.tracing.commoncontexts.IConnectionContext; import java.io.FileNotFoundException; import java.io.FileOutputStream; @@ -51,6 +52,7 @@ public FileConnectionCaptureFactory(String nodeId, String path, int bufferSize) @AllArgsConstructor class StreamManager extends OrderedStreamLifecyleManager { String connectionId; + @Override public CodedOutputStreamAndByteBufferWrapper createStream() { return new CodedOutputStreamAndByteBufferWrapper(bufferSize); @@ -80,7 +82,8 @@ public CodedOutputStreamAndByteBufferWrapper createStream() { } @Override - public IChannelConnectionCaptureSerializer createOffloader(String connectionId) { - return new StreamChannelConnectionCaptureSerializer(nodeId, connectionId, new StreamManager(connectionId)); + public IChannelConnectionCaptureSerializer createOffloader(IConnectionContext ctx) { + final var connectionId = ctx.getConnectionId(); + 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..f6d417c05 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,9 @@ package org.opensearch.migrations.trafficcapture; +import org.opensearch.migrations.tracing.commoncontexts.IConnectionContext; + import java.io.IOException; public interface IConnectionCaptureFactory { - IChannelConnectionCaptureSerializer createOffloader(String connectionId) throws IOException; + IChannelConnectionCaptureSerializer createOffloader(IConnectionContext ctx) throws IOException; } diff --git a/TrafficCapture/captureOffloader/src/main/java/org/opensearch/migrations/trafficcapture/OrderedStreamLifecyleManager.java b/TrafficCapture/captureOffloader/src/main/java/org/opensearch/migrations/trafficcapture/OrderedStreamLifecyleManager.java index d585c1ff4..b3dc72e75 100644 --- a/TrafficCapture/captureOffloader/src/main/java/org/opensearch/migrations/trafficcapture/OrderedStreamLifecyleManager.java +++ b/TrafficCapture/captureOffloader/src/main/java/org/opensearch/migrations/trafficcapture/OrderedStreamLifecyleManager.java @@ -11,5 +11,5 @@ public CompletableFuture closeStream(CodedOutputStreamHolder outputStreamHold } protected abstract CompletableFuture kickoffCloseStream(CodedOutputStreamHolder outputStreamHolder, - int index); + int index); } 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 c30cae6fc..ee06b4443 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 @@ -6,7 +6,6 @@ import io.netty.buffer.ByteBuf; import lombok.NonNull; import lombok.extern.slf4j.Slf4j; - import org.opensearch.migrations.trafficcapture.protos.CloseObservation; import org.opensearch.migrations.trafficcapture.protos.ConnectionExceptionObservation; import org.opensearch.migrations.trafficcapture.protos.EndOfMessageIndication; @@ -193,20 +192,22 @@ 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; + } } - return future; } @Override @@ -237,7 +238,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 @@ -246,7 +248,7 @@ public void addDeregisterEvent(Instant timestamp) throws IOException { } private void addStringMessage(int captureFieldNumber, int dataFieldNumber, - Instant timestamp, String str) throws IOException { + Instant timestamp, @NonNull String str) throws IOException { int dataSize = 0; int lengthSize = 1; if (str.length() > 0) { 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..74de0277a 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,5 +1,6 @@ package org.opensearch.migrations.trafficcapture; +import java.io.IOException; import java.util.concurrent.CompletableFuture; public interface StreamLifecycleManager { 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..d4ce9906c --- /dev/null +++ b/TrafficCapture/captureOffloader/src/main/java/org/opensearch/migrations/trafficcapture/tracing/ConnectionContext.java @@ -0,0 +1,57 @@ +package org.opensearch.migrations.trafficcapture.tracing; + +import io.opentelemetry.api.metrics.LongUpDownCounter; +import io.opentelemetry.api.metrics.Meter; +import lombok.Getter; +import lombok.NonNull; +import org.opensearch.migrations.tracing.BaseSpanContext; +import org.opensearch.migrations.tracing.CommonScopedMetricInstruments; +import org.opensearch.migrations.tracing.IHasRootInstrumentationScope; +import org.opensearch.migrations.tracing.commoncontexts.IConnectionContext; + +public class ConnectionContext extends BaseSpanContext + implements IConnectionContext, IHasRootInstrumentationScope { + + public static final String ACTIVE_CONNECTION = "activeConnection"; + public static final String ACTIVITY_NAME = "captureConnection"; + + @Getter + public final String connectionId; + @Getter + public final String nodeId; + + @Override + public String getActivityName() { return ACTIVITY_NAME; } + + public ConnectionContext(IRootOffloaderContext rootInstrumentationScope, String connectionId, String nodeId) { + super(rootInstrumentationScope); + this.connectionId = connectionId; + this.nodeId = nodeId; + initializeSpan(); + meterDeltaEvent(getMetrics().activeConnectionsCounter, 1); + } + + public static class MetricInstruments extends CommonScopedMetricInstruments { + private final LongUpDownCounter activeConnectionsCounter; + + protected MetricInstruments(Meter meter, String activityName) { + super(meter, activityName); + activeConnectionsCounter = meter.upDownCounterBuilder(ConnectionContext.ACTIVE_CONNECTION) + .setUnit("count").build(); + } + } + + public static @NonNull MetricInstruments makeMetrics(Meter meter) { + return new MetricInstruments(meter, ACTIVITY_NAME); + } + + public @NonNull MetricInstruments getMetrics() { + return getRootInstrumentationScope().getConnectionInstruments(); + } + + @Override + public void sendMeterEventsForEnd() { + super.sendMeterEventsForEnd(); + meterDeltaEvent(getMetrics().activeConnectionsCounter, -1); + } +} diff --git a/TrafficCapture/captureOffloader/src/main/java/org/opensearch/migrations/trafficcapture/tracing/IRootOffloaderContext.java b/TrafficCapture/captureOffloader/src/main/java/org/opensearch/migrations/trafficcapture/tracing/IRootOffloaderContext.java new file mode 100644 index 000000000..2b0e6bab0 --- /dev/null +++ b/TrafficCapture/captureOffloader/src/main/java/org/opensearch/migrations/trafficcapture/tracing/IRootOffloaderContext.java @@ -0,0 +1,18 @@ +package org.opensearch.migrations.trafficcapture.tracing; + +import org.opensearch.migrations.tracing.IRootOtelContext; + +public interface IRootOffloaderContext extends IRootOtelContext { + //public static final String OFFLOADER_SCOPE_NAME = "Offloader"; + ConnectionContext.MetricInstruments getConnectionInstruments(); + +// public RootOffloaderContext(OpenTelemetry openTelemetry) { +// this(openTelemetry, OFFLOADER_SCOPE_NAME); +// } +// +// public RootOffloaderContext(OpenTelemetry openTelemetry, String scopeName) { +// super(scopeName, openTelemetry); +// var meter = openTelemetry.getMeterProvider().get(scopeName); +// connectionInstruments = new ConnectionContext.MetricInstruments(meter, scopeName); +// } +} 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 f39395782..88ee5171d 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 @@ -28,6 +28,7 @@ import org.opensearch.migrations.trafficcapture.protos.ReadObservation; import org.opensearch.migrations.trafficcapture.protos.TrafficObservation; import org.opensearch.migrations.trafficcapture.protos.TrafficStream; + import org.opensearch.migrations.trafficcapture.protos.WriteObservation; @Slf4j 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..09b9b01c4 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,12 @@ 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 java.io.IOException; -import java.nio.ByteBuffer; import java.util.Arrays; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentLinkedQueue; @@ -36,6 +34,7 @@ public InMemoryConnectionCaptureFactory(String nodeId, int bufferSize, Runnable @AllArgsConstructor class StreamManager extends OrderedStreamLifecyleManager { + @Override public CodedOutputStreamHolder createStream() { return new CodedOutputStreamAndByteBufferWrapper(bufferSize); @@ -59,9 +58,9 @@ protected CompletableFuture kickoffCloseStream(CodedOutputStreamHolder out } @Override - public IChannelConnectionCaptureSerializer createOffloader(String connectionId) throws IOException { + public IChannelConnectionCaptureSerializer createOffloader(IConnectionContext ctx) 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()); + return new StreamChannelConnectionCaptureSerializer<>(nodeId, ctx.getConnectionId(), new StreamManager()); } public Stream getRecordedTrafficStreamsStream() { diff --git a/TrafficCapture/coreUtilities/README.md b/TrafficCapture/coreUtilities/README.md new file mode 100644 index 000000000..3581db355 --- /dev/null +++ b/TrafficCapture/coreUtilities/README.md @@ -0,0 +1,89 @@ +# Traffic Capture/Replay Instrumentation + +This "coreUtilities" package contains common classes and interfaces used to facilitate instrumentation for metrics and +traces. + +## Approach + +The package takes a hard dependency on OpenTelemetry ('otel'). OpenTelemetry provides a unified interface to a +variety of tracing and metering systems. From that unified interface, metric instruments and traces (or "spans") can +be sent to a variety of metric and tracing platforms, including Prometheus, Jaeger, and cloud native solutions like +Amazon CloudWatch and AWS X-Ray. +[RootOtelContext](src/main/java/org/opensearch/migrations/tracing/RootOtelContext.java) acts as a factory for metric +instruments and trace spans. As it is currently implemented, both metrics and spans are exported via +[OTLP](https://opentelemetry.io/docs/specs/otel/protocol/) to an +[OTEL Collector](https://opentelemetry.io/docs/collector/) that proxies instrumentations through further processing +and into downstream systems via exporters. + +It would be redundant to try to make another generic library, so the goal of this package is not to make it easier to +switch instrumentation platforms, but to make instrumentation fit with the TrafficCapture packages more naturally. As +a gradle project, this dependency is exposed as an "api" dependency so that other consumer packages will automatically +pick up the dependency as if it were their own dependency. + +Some of the OpenTelemetry patterns don't work naturally for this asynchronous code with dependency injection. +Specifically, try-with-resources and the use of stack frames to determine contexts are more trouble than they're worth. +Similarly, using statics to store meter values make the code more rigid and can make testing in parallel more difficult. + +This library adapts the parts of OpenTelemetry to make it more natural and more foolproof throughout the rest of the +TrafficCapture packages. This package introduces the concept of "Contexts" to build manage all tracing and metering +instrumentation. + +Just as the otel metering and tracing can be efficiently disabled by not configuring them, this library provides some +future-proofing by defining interfaces to track attributes, activities, exceptions, etc - but through descriptive +interfaces where callers describe which actions they're performing, preventing the code from becoming overly complex. + +The goals of the instrumentation package are to + +1. make instrumentation classes easy to use. +2. make it easy to create new safe and easy to use instrumentation classes. +3. be efficient enough to use in most cases and flexible enough to tune in cases where the cost is too high. + +The third point is still a work in progress as the exact performance penalty isn't understood yet. However, work for +point #2 dovetails into #3. As context creations are chained together, a no-op uber-context can be created with zero +memory footprint and minimal CPU penalty. The first couple points are accomplished by putting contextual information +alongside other data as first class parameters and fields. For example, where a method might require an identifier, +a context might be passed instead so that the function can retrieve identifying information via the context AND have +the ability to instrument activity within the appropriate context. + +## Class Structure Contexts + +All metering and tracing activity within the TrafficCapture packages occurs via "Contexts" which are implementations of +either [IInstrumentationAttributes](src/main/java/org/opensearch/migrations/tracing/IInstrumentationAttributes.java) or +its extension, +[IScopedInstrumentationAttributes](src/main/java/org/opensearch/migrations/tracing/IScopedInstrumentationAttributes.java). +IInstrumentationAttributes allows callers to meter activities into counters and histograms via +[otel instruments](https://opentelemetry.io/docs/concepts/signals/metrics/#metric-instruments). Callers need not know +any specific metric structures in order to add activities. Instead, contexts expose progress APIs that fit the +components that they're designed to work with. + +For example, the +[RequestTransformationContext](../trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/ReplayContexts.java) +class tracks network activity and performance of the http transformation code. That class manages how those +interactions and values are converted to otel instrumentation. That frees application code from implementation details, +makes the application cleaner, and allows all instrumentation concerns to be consolidated. + +IScopedInstrumentationAttributes extensions also provide (auto) instrumentation to indicate when the activities that +they represent began and ended. That information includes the duration of the activity represented by the context, +along with a count of the occurrences of the activity. In addition to those metrics, spans are also created and +emitted as the context is closed. + +The base Attributes interfaces (IInstrumentationAttributes and IScopedInstrumentationAttributes) provide functions to +fill in attributes that are specific to metrics and, independently, specific to spans. Metric values are aggregated +and the more unique attribute combinations possible for each time bucket, the larger the stress on the time-series +database. However, varied attributes can, in some circumstances, be worth the price of extra space and processing time. +Consider the metrics to show status code differences between the source and target clusters. + +In addition to those baseline features, some Context classes (that extend the Attributes interfaces) are capable of +creating child contexts that have a parent relationship with the creating Context. + +## OpenTelemetry Specifics + +While metric instruments can be emitted without any span context, after all the two systems receiving those values are +unrelated, emitting metrics from within a [Scope](https://opentelemetry.io/docs/concepts/instrumentation-scope/) allows +metrics to be linked to [exemplar](https://opentelemetry.io/docs/specs/otel/metrics/data-model/#exemplars) +spans. When Prometheus is used as a metrics data sink, as it is configured in the dockerSolution +(with '--enable-feature=exemplar-storage'), exemplars can be rendered in the same graph as the general data points. + +Since exact values can't be stored within a metrics data store, but we still have a need to render percentiles of +those results, OpenTelemetry uses bucketed histograms. The Contexts will automatically convert a numerical value (or +will calculate the number of milliseconds from the time that the Context was created) into a histogram. \ No newline at end of file diff --git a/TrafficCapture/coreUtilities/build.gradle b/TrafficCapture/coreUtilities/build.gradle index 1c4404477..17ad01040 100644 --- a/TrafficCapture/coreUtilities/build.gradle +++ b/TrafficCapture/coreUtilities/build.gradle @@ -22,6 +22,7 @@ plugins { // id 'checkstyle' id "io.freefair.lombok" version "8.0.1" id 'java' + id 'java-test-fixtures' } //spotbugs { @@ -40,6 +41,8 @@ repositories { } dependencies { + api platform("io.opentelemetry:opentelemetry-bom:1.34.1") + implementation project(':captureProtobufs') implementation "com.google.protobuf:protobuf-java:3.22.2" @@ -51,17 +54,27 @@ 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") + api group: 'io.opentelemetry', name: 'opentelemetry-api' + api group: 'io.opentelemetry', name: 'opentelemetry-sdk' + implementation group: 'io.opentelemetry', name: 'opentelemetry-exporter-otlp' + implementation group: 'io.opentelemetry.instrumentation', name: 'opentelemetry-log4j-appender-2.17', version: '1.30.0-alpha' + implementation group: 'io.opentelemetry.semconv', name: 'opentelemetry-semconv', version: '1.23.1-alpha' // OpenTelemetry log4j appender implementation("io.opentelemetry.instrumentation:opentelemetry-log4j-appender-2.17:1.30.0-alpha") + + testFixturesApi group: 'io.opentelemetry', name: 'opentelemetry-api' + testFixturesApi group: 'io.opentelemetry', name: 'opentelemetry-sdk' + testFixturesApi group: 'io.opentelemetry', name: 'opentelemetry-sdk-testing' + testFixturesImplementation platform("io.opentelemetry:opentelemetry-bom:1.34.1") + testFixturesImplementation group: 'io.opentelemetry', name: 'opentelemetry-api' + testFixturesImplementation group: 'io.opentelemetry', name: 'opentelemetry-sdk-testing' + testFixturesImplementation group: 'org.slf4j', name: 'slf4j-api', version: '2.0.7' } configurations.all { diff --git a/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/Utils.java b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/Utils.java new file mode 100644 index 000000000..bb181ff09 --- /dev/null +++ b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/Utils.java @@ -0,0 +1,22 @@ +package org.opensearch.migrations; + +import java.util.function.BiFunction; +import java.util.function.Function; +import java.util.stream.Collector; +import java.util.stream.Collectors; + +public class Utils { + /** + * See https://en.wikipedia.org/wiki/Fold_(higher-order_function) + */ + public static Collector + foldLeft(final B seedValue, final BiFunction f) { + return Collectors.collectingAndThen( + Collectors.reducing( + Function.identity(), + a -> b -> f.apply(b, a), + Function::andThen), + finisherArg -> finisherArg.apply(seedValue) + ); + } +} 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..2a13d0f37 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 @@ -1,14 +1,15 @@ package org.opensearch.migrations.coreutils; import lombok.extern.slf4j.Slf4j; -import org.slf4j.event.Level; import org.slf4j.Logger; +import org.slf4j.event.Level; 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..0bbc8b714 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,15 +1,8 @@ package org.opensearch.migrations.coreutils; -import io.opentelemetry.api.GlobalOpenTelemetry; -import io.opentelemetry.exporter.otlp.logs.OtlpGrpcLogRecordExporter; -import io.opentelemetry.instrumentation.log4j.appender.v2_17.OpenTelemetryAppender; -import io.opentelemetry.sdk.OpenTelemetrySdk; -import io.opentelemetry.sdk.logs.SdkLoggerProvider; -import io.opentelemetry.sdk.logs.export.BatchLogRecordProcessor; -import io.opentelemetry.sdk.resources.Resource; -import io.opentelemetry.semconv.resource.attributes.ResourceAttributes; -import org.slf4j.Logger; + import lombok.extern.slf4j.Slf4j; +import org.slf4j.Logger; import org.slf4j.LoggerFactory; @Slf4j @@ -28,31 +21,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 +52,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/BaseNestedSpanContext.java b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/BaseNestedSpanContext.java new file mode 100644 index 000000000..a7f35dc36 --- /dev/null +++ b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/BaseNestedSpanContext.java @@ -0,0 +1,33 @@ +package org.opensearch.migrations.tracing; + +import io.opentelemetry.api.common.AttributeKey; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.common.AttributesBuilder; +import io.opentelemetry.api.trace.Span; +import lombok.Getter; +import lombok.NonNull; + +import java.util.Optional; +import java.util.stream.Stream; + +public abstract class BaseNestedSpanContext + + extends BaseSpanContext +{ + final T enclosingScope; + + protected BaseNestedSpanContext(S rootScope, T enclosingScope) { + super(rootScope); + this.enclosingScope = enclosingScope; + } + + @Override + public IScopedInstrumentationAttributes getEnclosingScope() { + return enclosingScope; + } + + public T getImmediateEnclosingScope() { + return enclosingScope; + } + +} diff --git a/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/BaseSpanContext.java b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/BaseSpanContext.java new file mode 100644 index 000000000..5885cf774 --- /dev/null +++ b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/BaseSpanContext.java @@ -0,0 +1,74 @@ +package org.opensearch.migrations.tracing; + +import io.opentelemetry.api.common.AttributeKey; +import io.opentelemetry.api.common.AttributesBuilder; +import io.opentelemetry.api.trace.Span; +import lombok.Getter; +import lombok.NonNull; + +import java.util.Optional; +import java.util.stream.Stream; + +public abstract class BaseSpanContext + implements IScopedInstrumentationAttributes, IWithStartTimeAndAttributes, IHasRootInstrumentationScope, AutoCloseable { + @Getter + protected final S rootInstrumentationScope; + @Getter + final long startNanoTime; + @Getter + Throwable observedExceptionToIncludeInMetrics; + @Getter + private Span currentSpan; + + public BaseSpanContext(S rootScope) { + this.startNanoTime = System.nanoTime(); + this.rootInstrumentationScope = rootScope; + rootScope.onContextCreated(this); + } + + protected static AttributesBuilder addAttributeIfPresent(AttributesBuilder attributesBuilder, + AttributeKey key, Optional value) { + return value.map(v -> attributesBuilder.put(key, v)).orElse(attributesBuilder); + } + + @Override + public void endSpan() { + IScopedInstrumentationAttributes.super.endSpan(); + rootInstrumentationScope.onContextClosed(this); + } + + protected void initializeSpan() { + initializeSpanWithLinkedSpans(null); + } + + protected void initializeSpanWithLinkedSpans(Stream linkedSpans) { + initializeSpan(rootInstrumentationScope.buildSpan(this, getActivityName(), linkedSpans)); + } + + public void initializeSpan(@NonNull Span s) { + assert currentSpan == null : "only expect to set the current span once"; + currentSpan = s; + } + + @Override + public void addException(Throwable e) { + IScopedInstrumentationAttributes.super.addException(e); + observedExceptionToIncludeInMetrics = e; + } + + public long getStartNanoTime() { + return this.startNanoTime; + } + + public @NonNull Span getCurrentSpan() { + return this.currentSpan; + } + + public S getRootInstrumentationScope() { + return this.rootInstrumentationScope; + } + + public Throwable getObservedExceptionToIncludeInMetrics() { + return this.observedExceptionToIncludeInMetrics; + } +} diff --git a/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/CommonMetricInstruments.java b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/CommonMetricInstruments.java new file mode 100644 index 000000000..a93777a9b --- /dev/null +++ b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/CommonMetricInstruments.java @@ -0,0 +1,13 @@ +package org.opensearch.migrations.tracing; + +import io.opentelemetry.api.metrics.LongCounter; +import io.opentelemetry.api.metrics.Meter; + +public class CommonMetricInstruments { + final LongCounter exceptionCounter; + + public CommonMetricInstruments(Meter meter, String activityName) { + exceptionCounter = meter + .counterBuilder(activityName + "ExceptionCount").build(); + } +} diff --git a/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/CommonScopedMetricInstruments.java b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/CommonScopedMetricInstruments.java new file mode 100644 index 000000000..836362b86 --- /dev/null +++ b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/CommonScopedMetricInstruments.java @@ -0,0 +1,64 @@ +package org.opensearch.migrations.tracing; + +import io.opentelemetry.api.metrics.DoubleHistogram; +import io.opentelemetry.api.metrics.LongCounter; +import io.opentelemetry.api.metrics.Meter; +import lombok.extern.slf4j.Slf4j; + +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.DoubleStream; + + +@Slf4j +public class CommonScopedMetricInstruments extends CommonMetricInstruments { + final LongCounter contextCounter; + final DoubleHistogram contextDuration; + + public CommonScopedMetricInstruments(Meter meter, String activityName) { + this(meter, activityName, null); + } + + public CommonScopedMetricInstruments(Meter meter, String activityName, + double firstBucketSize, double lastBucketCeiling) { + this(meter, activityName, getBuckets(activityName, firstBucketSize, lastBucketCeiling)); + } + + private static List getBuckets(String activityName, + double firstBucketSize, double lastBucketCeiling) { + var buckets = getExponentialBucketsBetween(firstBucketSize, lastBucketCeiling, 2.0); + log.atInfo().setMessage(() -> "Setting buckets for " + activityName + " to " + + buckets.stream().map(x -> "" + x).collect(Collectors.joining(",", "[", "]"))).log(); + return buckets; + } + + private static List getExponentialBucketsBetween(double firstBucketSize, double lastBucketCeiling) { + return getExponentialBucketsBetween(firstBucketSize, lastBucketCeiling, 2.0); + } + + private static List getExponentialBucketsBetween(double firstBucketSize, double lastBucketCeiling, + double rate) { + if (firstBucketSize <= 0) { + throw new IllegalArgumentException("firstBucketSize value " + firstBucketSize + " must be > 0"); + } + double[] bucketBoundary = new double[]{firstBucketSize}; + return DoubleStream.generate(() -> { + var tmp = bucketBoundary[0]; + bucketBoundary[0] *= rate; + return tmp; + }).takeWhile(v -> v <= lastBucketCeiling).boxed().collect(Collectors.toList()); + } + + public CommonScopedMetricInstruments(Meter meter, String activityName, List buckets) { + super(meter, activityName); + contextCounter = meter + .counterBuilder(activityName + "Count").build(); + var durationBuilder = meter + .histogramBuilder(activityName + "Duration") + .setUnit("ms"); + if (buckets != null) { + durationBuilder = durationBuilder.setExplicitBucketBoundariesAdvice(buckets); + } + contextDuration = durationBuilder.build(); + } +} diff --git a/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/DirectNestedSpanContext.java b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/DirectNestedSpanContext.java new file mode 100644 index 000000000..5d6c59a6e --- /dev/null +++ b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/DirectNestedSpanContext.java @@ -0,0 +1,17 @@ +package org.opensearch.migrations.tracing; + +public abstract class DirectNestedSpanContext, + L> + extends BaseNestedSpanContext + implements IWithTypedEnclosingScope +{ + protected DirectNestedSpanContext(T parent) { + super(parent.getRootInstrumentationScope(), parent); + } + + @Override + public L getLogicalEnclosingScope() { + return (L) getEnclosingScope(); + } +} diff --git a/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/IHasRootInstrumentationScope.java b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/IHasRootInstrumentationScope.java new file mode 100644 index 000000000..80a56cbb0 --- /dev/null +++ b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/IHasRootInstrumentationScope.java @@ -0,0 +1,9 @@ +package org.opensearch.migrations.tracing; + +/** + * This exists as helper glue to make pattern matching in the generics + * work to allow for more simplified constructors. + */ +public interface IHasRootInstrumentationScope { + S getRootInstrumentationScope(); +} diff --git a/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/IInstrumentConstructor.java b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/IInstrumentConstructor.java new file mode 100644 index 000000000..4be0a3b65 --- /dev/null +++ b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/IInstrumentConstructor.java @@ -0,0 +1,22 @@ +package org.opensearch.migrations.tracing; + +import io.opentelemetry.api.common.AttributesBuilder; +import io.opentelemetry.api.trace.Span; +import lombok.NonNull; + +import java.util.stream.Stream; + +public interface IInstrumentConstructor { + @NonNull Span buildSpan(IScopedInstrumentationAttributes forScope, String spanName, Stream linkedSpans); + + /** + * For debugging, this will be overridden to track creation and termination of spans + */ + default void onContextCreated(IScopedInstrumentationAttributes newScopedContext) {} + + /** + * For debugging, this will be overridden to track creation and termination of spans + */ + default void onContextClosed(IScopedInstrumentationAttributes newScopedContext) {} + +} diff --git a/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/IInstrumentationAttributes.java b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/IInstrumentationAttributes.java new file mode 100644 index 000000000..e89570036 --- /dev/null +++ b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/IInstrumentationAttributes.java @@ -0,0 +1,80 @@ +package org.opensearch.migrations.tracing; + +import io.opentelemetry.api.common.AttributeKey; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.common.AttributesBuilder; +import io.opentelemetry.api.metrics.DoubleHistogram; +import io.opentelemetry.api.metrics.LongCounter; +import io.opentelemetry.api.metrics.LongHistogram; +import io.opentelemetry.api.metrics.LongUpDownCounter; +import io.opentelemetry.api.trace.Span; +import lombok.NonNull; +import org.opensearch.migrations.Utils; + +import java.time.Duration; +import java.util.ArrayDeque; + +public interface IInstrumentationAttributes { + AttributeKey HAD_EXCEPTION_KEY = AttributeKey.booleanKey("hadException"); + + IInstrumentationAttributes getEnclosingScope(); + + CommonMetricInstruments getMetrics(); + + Throwable getObservedExceptionToIncludeInMetrics(); + + default @NonNull Attributes getPopulatedMetricAttributes(AttributesBuilder attributesBuilder) { + final var e = getObservedExceptionToIncludeInMetrics(); + return e == null ? attributesBuilder.build() : attributesBuilder.put(HAD_EXCEPTION_KEY, true).build(); + } + + default void addException(Throwable e) { + meterIncrementEvent(getMetrics().exceptionCounter); + } + + default void meterIncrementEvent(LongCounter c) { + meterIncrementEvent(c, 1); + } + + default void meterIncrementEvent(LongCounter c, long increment) { + meterIncrementEvent(c, increment, Attributes.builder()); + } + + default void meterIncrementEvent(LongCounter c, long increment, AttributesBuilder attributesBuilder) { + c.add(increment, getPopulatedMetricAttributes(attributesBuilder)); + } + + default void meterDeltaEvent(LongUpDownCounter c, long delta) { + meterDeltaEvent(c, delta, Attributes.builder()); + } + + default void meterDeltaEvent(LongUpDownCounter c, long delta, AttributesBuilder attributesBuilder) { + var attributes = getPopulatedMetricAttributes(attributesBuilder); + c.add(delta, attributes); + } + + default void meterHistogramMillis(DoubleHistogram histogram, Duration value) { + meterHistogram(histogram, value.toNanos() / 1_000_000.0); + } + + default void meterHistogramMillis(DoubleHistogram histogram, Duration value, AttributesBuilder attributesBuilder) { + meterHistogram(histogram, value.toNanos() / 1_000_000.0, attributesBuilder); + } + + default void meterHistogram(DoubleHistogram histogram, double value) { + meterHistogram(histogram, value, Attributes.builder()); + } + + default void meterHistogram(DoubleHistogram histogram, double value, AttributesBuilder attributesBuilder) { + histogram.record(value, getPopulatedMetricAttributes(attributesBuilder)); + } + + default void meterHistogram(LongHistogram histogram, long value) { + meterHistogram(histogram, value, Attributes.builder()); + } + + default void meterHistogram(LongHistogram histogram, long value, AttributesBuilder attributesBuilder) { + histogram.record(value, getPopulatedMetricAttributes(attributesBuilder)); + } + +} diff --git a/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/IRootOtelContext.java b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/IRootOtelContext.java new file mode 100644 index 000000000..4fb34ece9 --- /dev/null +++ b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/IRootOtelContext.java @@ -0,0 +1,7 @@ +package org.opensearch.migrations.tracing; + +import io.opentelemetry.api.metrics.MeterProvider; + +public interface IRootOtelContext extends IInstrumentationAttributes, IInstrumentConstructor { + MeterProvider getMeterProvider(); +} diff --git a/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/IScopedInstrumentationAttributes.java b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/IScopedInstrumentationAttributes.java new file mode 100644 index 000000000..44f33423e --- /dev/null +++ b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/IScopedInstrumentationAttributes.java @@ -0,0 +1,112 @@ +package org.opensearch.migrations.tracing; + +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.common.AttributesBuilder; +import io.opentelemetry.api.metrics.DoubleHistogram; +import io.opentelemetry.api.metrics.LongCounter; +import io.opentelemetry.api.metrics.LongHistogram; +import io.opentelemetry.api.metrics.LongUpDownCounter; +import io.opentelemetry.api.trace.Span; +import lombok.NonNull; +import org.opensearch.migrations.Utils; + +import java.util.ArrayDeque; + +public interface IScopedInstrumentationAttributes + extends IWithStartTimeAndAttributes, AutoCloseable { + + String getActivityName(); + + @Override + IScopedInstrumentationAttributes getEnclosingScope(); + + @Override + CommonScopedMetricInstruments getMetrics(); + + @NonNull Span getCurrentSpan(); + + default Attributes getPopulatedSpanAttributes() { + return getPopulatedSpanAttributesBuilder().build(); + } + + default AttributesBuilder getPopulatedSpanAttributesBuilder() { + IInstrumentationAttributes currentObj = this; + // reverse the order so that the lowest attribute scopes will overwrite the upper ones if there were conflicts + var stack = new ArrayDeque(); + while (currentObj != null) { + if (currentObj instanceof IScopedInstrumentationAttributes) { + stack.addFirst((IScopedInstrumentationAttributes) currentObj); + } + currentObj = currentObj.getEnclosingScope(); + } + var builder = stack.stream() + .collect(Utils.foldLeft(Attributes.builder(), (b, iia)->iia.fillAttributesForSpansBelow(b))); + return fillExtraAttributesForThisSpan(builder); + } + + default AttributesBuilder fillAttributesForSpansBelow(AttributesBuilder builder) { + return builder; + } + + default AttributesBuilder fillExtraAttributesForThisSpan(AttributesBuilder builder) { + return builder; + } + + default LongCounter getEndOfScopeCountMetric() { + return getMetrics().contextCounter; + } + + default DoubleHistogram getEndOfScopeDurationMetric() { + return getMetrics().contextDuration; + } + + default void endSpan() { + var span = getCurrentSpan(); + span.setAllAttributes(getPopulatedSpanAttributes()); + span.end(); + } + + default void sendMeterEventsForEnd() { + meterIncrementEvent(getEndOfScopeCountMetric()); + meterHistogramMillis(getEndOfScopeDurationMetric()); + } + + default void close() { + endSpan(); + sendMeterEventsForEnd(); + } + + @Override + default void addException(Throwable e) { + IWithStartTimeAndAttributes.super.addException(e); + getCurrentSpan().recordException(e); + } + + @Override + default void meterIncrementEvent(LongCounter c, long increment, AttributesBuilder attributesBuilder) { + try (var scope = new NullableExemplarScope(getCurrentSpan())) { + IWithStartTimeAndAttributes.super.meterIncrementEvent(c, increment, attributesBuilder); + } + } + + @Override + default void meterDeltaEvent(LongUpDownCounter c, long delta, AttributesBuilder attributesBuilder) { + try (var scope = new NullableExemplarScope(getCurrentSpan())) { + IWithStartTimeAndAttributes.super.meterDeltaEvent(c, delta, attributesBuilder); + } + } + + @Override + default void meterHistogram(DoubleHistogram histogram, double value, AttributesBuilder attributesBuilder) { + try (var scope = new NullableExemplarScope(getCurrentSpan())) { + IWithStartTimeAndAttributes.super.meterHistogram(histogram, value, attributesBuilder); + } + } + + @Override + default void meterHistogram(LongHistogram histogram, long value, AttributesBuilder attributesBuilder) { + try (var scope = new NullableExemplarScope(getCurrentSpan())) { + IWithStartTimeAndAttributes.super.meterHistogram(histogram, value, attributesBuilder); + } + } +} 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..d844aebaf --- /dev/null +++ b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/IWithStartTimeAndAttributes.java @@ -0,0 +1,22 @@ +package org.opensearch.migrations.tracing; + +import io.opentelemetry.api.common.AttributesBuilder; +import io.opentelemetry.api.metrics.DoubleHistogram; + +import java.time.Duration; + +public interface IWithStartTimeAndAttributes extends IInstrumentationAttributes { + long getStartNanoTime(); + + default Duration getSpanDuration() { + return Duration.ofNanos(System.nanoTime() - getStartNanoTime()); + } + + default void meterHistogramMillis(DoubleHistogram histogram) { + meterHistogramMillis(histogram, getSpanDuration()); + } + + default void meterHistogramMillis(DoubleHistogram histogram, AttributesBuilder attributesBuilder) { + meterHistogramMillis(histogram, getSpanDuration(), attributesBuilder); + } +} diff --git a/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/IWithTypedEnclosingScope.java b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/IWithTypedEnclosingScope.java new file mode 100644 index 000000000..1802e9649 --- /dev/null +++ b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/IWithTypedEnclosingScope.java @@ -0,0 +1,5 @@ +package org.opensearch.migrations.tracing; + +public interface IWithTypedEnclosingScope extends IInstrumentationAttributes { + T getLogicalEnclosingScope(); +} diff --git a/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/NullableExemplarScope.java b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/NullableExemplarScope.java new file mode 100644 index 000000000..f84c3f802 --- /dev/null +++ b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/NullableExemplarScope.java @@ -0,0 +1,20 @@ +package org.opensearch.migrations.tracing; + +import io.opentelemetry.api.trace.Span; +import io.opentelemetry.context.Context; +import io.opentelemetry.context.Scope; + +public class NullableExemplarScope implements Scope { + final Scope underlyingScope; + + public NullableExemplarScope(Span span) { + underlyingScope = span == null ? null : Context.current().with(span).makeCurrent(); + } + + @Override + public void close() { + if (underlyingScope != null) { + underlyingScope.close(); + } + } +} diff --git a/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/RootOtelContext.java b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/RootOtelContext.java new file mode 100644 index 000000000..320693fe3 --- /dev/null +++ b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/RootOtelContext.java @@ -0,0 +1,161 @@ +package org.opensearch.migrations.tracing; + +import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.api.metrics.Meter; +import io.opentelemetry.api.metrics.MeterProvider; +import io.opentelemetry.api.trace.Span; +import io.opentelemetry.api.trace.SpanBuilder; +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.ResourceAttributes; +import lombok.Getter; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.opensearch.migrations.Utils; + +import java.time.Duration; +import java.util.Optional; +import java.util.concurrent.TimeUnit; +import java.util.stream.Stream; + +@Slf4j +public class RootOtelContext implements IRootOtelContext { + private final OpenTelemetry openTelemetryImpl; + private final String scopeName; + @Getter + private final MetricInstruments metrics; + + public static OpenTelemetry initializeOpenTelemetryForCollector(@NonNull String collectorEndpoint, + @NonNull String serviceName) { + var serviceResource = Resource.getDefault().toBuilder() + .put(ResourceAttributes.SERVICE_NAME, serviceName) + .build(); + + final var spanProcessor = BatchSpanProcessor.builder(OtlpGrpcSpanExporter.builder() + .setEndpoint(collectorEndpoint) + .setTimeout(2, TimeUnit.SECONDS) + .build()) + .build(); + final var metricReader = PeriodicMetricReader.builder(OtlpGrpcMetricExporter.builder() + .setEndpoint(collectorEndpoint) + // see https://opentelemetry.io/docs/specs/otel/metrics/sdk_exporters/prometheus/ + // "A Prometheus Exporter MUST only support Cumulative Temporality." + //.setAggregationTemporalitySelector(AggregationTemporalitySelector.deltaPreferred()) + .build()) + .setInterval(Duration.ofMillis(1000)) + .build(); + final var logProcessor = BatchLogRecordProcessor.builder(OtlpGrpcLogRecordExporter.builder() + .setEndpoint(collectorEndpoint) + .build()) + .build(); + + var openTelemetrySdk = OpenTelemetrySdk.builder() + .setTracerProvider(SdkTracerProvider.builder().setResource(serviceResource) + .addSpanProcessor(spanProcessor).build()) + .setMeterProvider(SdkMeterProvider.builder().setResource(serviceResource) + .registerMetricReader(metricReader).build()) + .setLoggerProvider(SdkLoggerProvider.builder().setResource(serviceResource) + .addLogRecordProcessor(logProcessor).build()) + .build(); + + // Add hook to close SDK, which flushes logs + Runtime.getRuntime().addShutdownHook(new Thread(openTelemetrySdk::close)); + return openTelemetrySdk; + } + + public static OpenTelemetry initializeNoopOpenTelemetry() { + return OpenTelemetrySdk.builder().build(); + } + + /** + * Initialize the Otel SDK for a collector if collectorEndpoint != null or setup an empty, + * do-nothing SDK when it is null. + * @param collectorEndpoint - URL of the otel-collector + * @param serviceName - name of this service that is sending data to the collector + * @return a fully initialize OpenTelemetry object capable of producing MeterProviders and TraceProviders + */ + public static OpenTelemetry + initializeOpenTelemetryWithCollectorOrAsNoop(String collectorEndpoint, String serviceName) { + return Optional.ofNullable(collectorEndpoint) + .map(endpoint -> initializeOpenTelemetryForCollector(endpoint, serviceName)) + .orElseGet(() -> { + if (serviceName != null) { + log.atWarn().setMessage("Collector endpoint=null, so serviceName parameter '" + serviceName + + "' is being ignored since a no-op OpenTelemetry object is being created").log(); + } + return initializeNoopOpenTelemetry(); + }); + } + + public static class MetricInstruments extends CommonMetricInstruments { + public MetricInstruments(Meter meter, String activityName) { + super(meter, activityName); + } + } + + public RootOtelContext(String scopeName) { + this(scopeName, null); + } + + public RootOtelContext(String scopeName, String collectorEndpoint, String serviceName) { + this(scopeName, initializeOpenTelemetryWithCollectorOrAsNoop(collectorEndpoint, serviceName)); + } + + public RootOtelContext(String scopeName, OpenTelemetry sdk) { + openTelemetryImpl = sdk != null ? sdk : initializeOpenTelemetryWithCollectorOrAsNoop(null, null); + this.scopeName = scopeName; + metrics = new MetricInstruments(this.getMeterProvider().get(scopeName), "root"); + } + + @Override + public Exception getObservedExceptionToIncludeInMetrics() { + return null; + } + + @Override + public RootOtelContext getEnclosingScope() { + return null; + } + + OpenTelemetry getOpenTelemetry() { + return openTelemetryImpl; + } + + @Override + public MeterProvider getMeterProvider() { + return getOpenTelemetry().getMeterProvider(); + } + + private static SpanBuilder addLinkedToBuilder(Stream linkedSpanContexts, SpanBuilder spanBuilder) { + return Optional.ofNullable(linkedSpanContexts) + .map(ss -> ss.collect(Utils.foldLeft(spanBuilder, (b, s) -> b.addLink(s.getSpanContext())))) + .orElse(spanBuilder); + } + + private static Span buildSpanWithParent(SpanBuilder builder, Span parentSpan, Stream linkedSpanContexts) { + return addLinkedToBuilder(linkedSpanContexts, + Optional.ofNullable(parentSpan) + .map(p -> builder.setParent(Context.current().with(p))) + .orElseGet(builder::setNoParent)) + .startSpan(); + } + + @Override + public @NonNull Span buildSpan(IScopedInstrumentationAttributes forScope, String spanName, Stream linkedSpans) { + assert forScope.getCurrentSpan() == null; + var forEnclosingScope = forScope.getEnclosingScope(); + var parentSpan = forEnclosingScope == null ? null : forEnclosingScope.getCurrentSpan(); + var spanBuilder = getOpenTelemetry().getTracer(scopeName).spanBuilder(spanName); + return buildSpanWithParent(spanBuilder, parentSpan, linkedSpans); + } +} 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..48a02873a --- /dev/null +++ b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/commoncontexts/IConnectionContext.java @@ -0,0 +1,24 @@ +package org.opensearch.migrations.tracing.commoncontexts; + +import io.opentelemetry.api.common.AttributeKey; +import io.opentelemetry.api.common.AttributesBuilder; +import org.opensearch.migrations.tracing.IInstrumentationAttributes; +import org.opensearch.migrations.tracing.IScopedInstrumentationAttributes; + +public interface IConnectionContext extends IScopedInstrumentationAttributes { + static final AttributeKey CONNECTION_ID_ATTR = AttributeKey.stringKey("connectionId"); + static final AttributeKey NODE_ID_ATTR = AttributeKey.stringKey("nodeId"); + + String getConnectionId(); + String getNodeId(); + + @Override + default IScopedInstrumentationAttributes getEnclosingScope() { return null; } + + @Override + default AttributesBuilder fillAttributesForSpansBelow(AttributesBuilder builder) { + return IScopedInstrumentationAttributes.super.fillAttributesForSpansBelow(builder) + .put(CONNECTION_ID_ATTR, getConnectionId()) + .put(NODE_ID_ATTR, getNodeId()); + } +} diff --git a/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/commoncontexts/IHttpTransactionContext.java b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/commoncontexts/IHttpTransactionContext.java new file mode 100644 index 000000000..13acb7136 --- /dev/null +++ b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/tracing/commoncontexts/IHttpTransactionContext.java @@ -0,0 +1,17 @@ +package org.opensearch.migrations.tracing.commoncontexts; + +import io.opentelemetry.api.common.AttributeKey; +import io.opentelemetry.api.common.AttributesBuilder; +import org.opensearch.migrations.tracing.IScopedInstrumentationAttributes; + +public interface IHttpTransactionContext extends IScopedInstrumentationAttributes { + static final AttributeKey SOURCE_REQUEST_INDEX_KEY = AttributeKey.longKey("sourceRequestIndex"); + + long getSourceRequestIndex(); + + @Override + default AttributesBuilder fillAttributesForSpansBelow(AttributesBuilder builder) { + return IScopedInstrumentationAttributes.super.fillAttributesForSpansBelow(builder) + .put(SOURCE_REQUEST_INDEX_KEY, getSourceRequestIndex()); + } +} diff --git a/TrafficCapture/coreUtilities/src/test/java/org/opensearch/migrations/tracing/CommonScopedMetricInstrumentsTest.java b/TrafficCapture/coreUtilities/src/test/java/org/opensearch/migrations/tracing/CommonScopedMetricInstrumentsTest.java new file mode 100644 index 000000000..3f375b2c7 --- /dev/null +++ b/TrafficCapture/coreUtilities/src/test/java/org/opensearch/migrations/tracing/CommonScopedMetricInstrumentsTest.java @@ -0,0 +1,20 @@ +package org.opensearch.migrations.tracing; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.*; + +class CommonScopedMetricInstrumentsTest { + @Test + public void testThatBadSizeThrowsException() { + Assertions.assertThrows(IllegalArgumentException.class, () -> + new CommonScopedMetricInstruments(null, "testActivity", 0, 2)); + Assertions.assertThrows(IllegalArgumentException.class, () -> + new CommonScopedMetricInstruments(null, "testActivity", -2, 2)); + var otelSdkBundle = new InMemoryInstrumentationBundle(false, false); + Assertions.assertDoesNotThrow(() -> + new CommonScopedMetricInstruments(otelSdkBundle.getOpenTelemetrySdk().getMeter(""), + "testActivity", 1, 8)); + } +} \ No newline at end of file diff --git a/TrafficCapture/coreUtilities/src/test/java/org/opensearch/migrations/tracing/IInstrumentationAttributesTest.java b/TrafficCapture/coreUtilities/src/test/java/org/opensearch/migrations/tracing/IInstrumentationAttributesTest.java new file mode 100644 index 000000000..45afa2769 --- /dev/null +++ b/TrafficCapture/coreUtilities/src/test/java/org/opensearch/migrations/tracing/IInstrumentationAttributesTest.java @@ -0,0 +1,80 @@ +package org.opensearch.migrations.tracing; + +import io.opentelemetry.api.common.AttributeKey; +import io.opentelemetry.api.common.AttributesBuilder; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.Optional; + +class IInstrumentationAttributesTest { + + private static final AttributeKey OVERRIDE_KEY = AttributeKey.stringKey("overrideKey"); + private static final AttributeKey UNIQUE_KEY = AttributeKey.stringKey("uniqueKey"); + + private static class AContext extends BaseSpanContext { + protected AContext(RootOtelContext rootScope) { + super(rootScope); + } + + @Override + public String getActivityName() { + return "A"; + } + + @Override + public IScopedInstrumentationAttributes getEnclosingScope() { + return null; + } + + @Override + public CommonScopedMetricInstruments getMetrics() { + return null; + } + + @Override + public AttributesBuilder fillAttributesForSpansBelow(AttributesBuilder builder) { + return super.fillAttributesForSpansBelow(builder) + .put(OVERRIDE_KEY, "a-toBeOverridden") + .put(UNIQUE_KEY, "a-toStay"); + } + } + + private static class BContext extends BaseNestedSpanContext{ + protected BContext(RootOtelContext rootScope, AContext enclosingScope) { + super(rootScope, enclosingScope); + } + + @Override + public String getActivityName() { + return "B"; + } + + @Override + public CommonScopedMetricInstruments getMetrics() { + return null; + } + + @Override + public AttributesBuilder fillAttributesForSpansBelow(AttributesBuilder builder) { + return super.fillAttributesForSpansBelow(builder) + .put(OVERRIDE_KEY, "b"); + } + } + + @Test + public void getPopulatedAttributesAreOverrideCorrectly() { + var rootCtx = new RootOtelContext("test"); + var aCtx = new AContext(rootCtx); + var bCtx = new BContext(rootCtx, aCtx); + + Optional.ofNullable(aCtx.getPopulatedSpanAttributes()).ifPresent(attrs-> { + Assertions.assertEquals("a-toBeOverridden", attrs.get(OVERRIDE_KEY)); + Assertions.assertEquals("a-toStay", attrs.get(UNIQUE_KEY)); + }); + Optional.ofNullable(bCtx.getPopulatedSpanAttributes()).ifPresent(attrs-> { + Assertions.assertEquals("b", attrs.get(OVERRIDE_KEY)); + Assertions.assertEquals("a-toStay", attrs.get(UNIQUE_KEY)); + }); + } +} \ No newline at end of file diff --git a/TrafficCapture/coreUtilities/src/testFixtures/java/org/opensearch/migrations/tracing/ContextTracker.java b/TrafficCapture/coreUtilities/src/testFixtures/java/org/opensearch/migrations/tracing/ContextTracker.java new file mode 100644 index 000000000..a3214ce3c --- /dev/null +++ b/TrafficCapture/coreUtilities/src/testFixtures/java/org/opensearch/migrations/tracing/ContextTracker.java @@ -0,0 +1,74 @@ +package org.opensearch.migrations.tracing; + +import lombok.Getter; +import lombok.extern.slf4j.Slf4j; + +import java.util.Map; +import java.util.WeakHashMap; +import java.util.stream.Collectors; + +@Slf4j +public class ContextTracker implements AutoCloseable { + private static class ExceptionForStackTracingOnly extends Exception { + } + + @Getter + public static class CallDetails { + private final ExceptionForStackTracingOnly createStackException; + private ExceptionForStackTracingOnly closeStackException; + + public CallDetails() { + createStackException = new ExceptionForStackTracingOnly(); + } + } + + private final Map scopedContextToCallDetails = + new WeakHashMap<>(); + private final Object lockObject = new Object(); + private boolean isClosed; + + public void onCreated(IScopedInstrumentationAttributes ctx) { + synchronized (lockObject) { + if (isClosed) { + return; + } + var oldItem = scopedContextToCallDetails.putIfAbsent(ctx, new CallDetails()); + assert oldItem == null; + } + } + + public void onClosed(IScopedInstrumentationAttributes ctx) { + synchronized (lockObject) { + if (isClosed) { + return; + } + var newExceptionStack = new ExceptionForStackTracingOnly(); + var oldCallDetails = scopedContextToCallDetails.get(ctx); + assert oldCallDetails != null; + final var oldE = oldCallDetails.closeStackException; + if (oldE != null) { + log.atError().setCause(newExceptionStack).setMessage(() -> "Close is being called here").log(); + log.atError().setCause(oldE).setMessage(() -> "... but close was already called here").log(); + assert oldE == null; + } + oldCallDetails.closeStackException = new ExceptionForStackTracingOnly(); + } + } + + public Map getAllRemainingActiveScopes() { + synchronized (lockObject) { + return scopedContextToCallDetails.entrySet().stream() + // filter away items that were closed but not cleared yet (since it's a weak map) + .filter(kvp -> kvp.getValue().closeStackException == null) + // make a copy since we're in a synchronized block + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + } + } + + public void close() { + synchronized (lockObject) { + scopedContextToCallDetails.clear(); + isClosed = true; + } + } +} diff --git a/TrafficCapture/coreUtilities/src/testFixtures/java/org/opensearch/migrations/tracing/InMemoryInstrumentationBundle.java b/TrafficCapture/coreUtilities/src/testFixtures/java/org/opensearch/migrations/tracing/InMemoryInstrumentationBundle.java new file mode 100644 index 000000000..3d1ea5930 --- /dev/null +++ b/TrafficCapture/coreUtilities/src/testFixtures/java/org/opensearch/migrations/tracing/InMemoryInstrumentationBundle.java @@ -0,0 +1,104 @@ +package org.opensearch.migrations.tracing; + +import io.opentelemetry.sdk.OpenTelemetrySdk; +import io.opentelemetry.sdk.common.CompletableResultCode; +import io.opentelemetry.sdk.metrics.InstrumentType; +import io.opentelemetry.sdk.metrics.SdkMeterProvider; +import io.opentelemetry.sdk.metrics.data.AggregationTemporality; +import io.opentelemetry.sdk.metrics.data.MetricData; +import io.opentelemetry.sdk.metrics.export.MetricExporter; +import io.opentelemetry.sdk.metrics.export.PeriodicMetricReader; +import io.opentelemetry.sdk.testing.exporter.InMemoryMetricExporter; +import io.opentelemetry.sdk.testing.exporter.InMemorySpanExporter; +import io.opentelemetry.sdk.trace.SdkTracerProvider; +import io.opentelemetry.sdk.trace.export.SimpleSpanProcessor; +import lombok.Getter; +import lombok.NonNull; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; + +@Getter +public class InMemoryInstrumentationBundle implements AutoCloseable { + + public static class LastMetricsExporter implements MetricExporter { + private final Queue finishedMetricItems = new ConcurrentLinkedQueue<>(); + boolean isStopped; + + public List getFinishedMetricItems() { + return Collections.unmodifiableList(new ArrayList<>(finishedMetricItems)); + } + + @Override + public CompletableResultCode export(@NonNull Collection metrics) { + if (isStopped) { + return CompletableResultCode.ofFailure(); + } + finishedMetricItems.clear(); + finishedMetricItems.addAll(metrics); + return CompletableResultCode.ofSuccess(); + } + + @Override + public CompletableResultCode flush() { + return CompletableResultCode.ofSuccess(); + } + + @Override + public CompletableResultCode shutdown() { + isStopped = true; + return CompletableResultCode.ofSuccess(); + } + + @Override + public AggregationTemporality getAggregationTemporality(@NonNull InstrumentType instrumentType) { + return AggregationTemporality.CUMULATIVE; + } + } + + public final OpenTelemetrySdk openTelemetrySdk; + public final InMemorySpanExporter testSpanExporter; + public final LastMetricsExporter testMetricExporter; + + public InMemoryInstrumentationBundle(boolean collectTraces, + boolean collectMetrics) { + this(collectTraces ? InMemorySpanExporter.create() : null, + collectMetrics ? new LastMetricsExporter() : null); + } + + public InMemoryInstrumentationBundle(InMemorySpanExporter testSpanExporter, + LastMetricsExporter testMetricExporter) { + this.testSpanExporter = testSpanExporter; + this.testMetricExporter = testMetricExporter; + + var otelBuilder = OpenTelemetrySdk.builder(); + if (testSpanExporter != null) { + otelBuilder = otelBuilder.setTracerProvider(SdkTracerProvider.builder() + .addSpanProcessor(SimpleSpanProcessor.create(testSpanExporter)).build()); + } + if (testMetricExporter != null) { + otelBuilder = otelBuilder.setMeterProvider(SdkMeterProvider.builder() + .registerMetricReader(PeriodicMetricReader.builder(testMetricExporter) + .setInterval(Duration.ofMillis(100)) + .build()) + .build()); + } + openTelemetrySdk = otelBuilder.build(); + } + + @Override + public void close() { + Optional.ofNullable(testMetricExporter).ifPresent(MetricExporter::close); + Optional.ofNullable(testSpanExporter).ifPresent(te -> { + te.close(); + te.reset(); + }); + } + +} diff --git a/TrafficCapture/dockerSolution/README.md b/TrafficCapture/dockerSolution/README.md index 6d2dc1967..4d31eb2c4 100644 --- a/TrafficCapture/dockerSolution/README.md +++ b/TrafficCapture/dockerSolution/README.md @@ -8,13 +8,61 @@ down again. Notice that most of the Dockerfiles are dynamically constructed in the build hierarchy. Some efforts have been made to ensure that changes will make it into containers to be launched. -### Running the Docker Solution +## Running the Docker Solution While in the TrafficCapture directory, run the following command: `./gradlew :dockerSolution:composeUp` -### Compatibility +### Running with different telemetry flavors. + +By default, composeUp will run an otel-collector that exports instrumentation to other local containers within the +migrations network. However, the collector's export configuration can be overridden via the otel-collector property: +`TrafficCapture % ./gradlew :dockerSolution:composeUp -Potel-collector=otel-aws.yml` + +The [otel-aws.yml](src/main/docker/composeExtensions/otel-aws.yml) will use that docker-compose extension. +That extension uses the collector configurations (from the container's base image) and wires the ~/.aws/credentials +file into the container to provide the collector credentials that it needs to push metrics and traces to CloudWatch +and X-Ray. In addition to the [default configuration](src/main/docker/composeExtensions/otel-prometheus-jaeger-opensearch.yml) +to use local containers, a [third option](src/main/docker/composeExtensions/otel-everything.yml) will use _BOTH_ local +containers AND the AWS services. + +## Maintaining configurations + +### Otel-Collector configurations + +The migrations components use OpenTelemetry for instrumentation so that different systems can be utilized across +(Prometheus, CloudWatch, Zipkin, etc) and across different types of infrastructure. The docker solutions vended +in this directory try to provide flexibility and consistency between different environments. Base images may +change in the future as otel matures and configurations will also need to be updated over time due to external +changes (e.g. debug/logging exporter) or internal ones (buffering parameters). To manage the 5 starting +configurations that we produce for one purpose or another, support code is within [otelConfigs](otelConfigs). + +The otel-collector configuration is contained within one yaml file. That single file configures the collector +to interface with many different systems. Maintaining consistency of configurations even as they're +copy-pasted between each other isn't scalable. Complicating matters more is that the base +otel-collector from otel and the AWS distro both lack a posix base system. That makes cons-ing any +configurations within the container challenging. The compromise struck here is to do the construction of +configuration files as a preprocessing step BEFORE docker builds. That preprocessing logic is within +dockerSolution/otelConfigs. + +A python script creates individual (but complete) otel-collector configurations +([consConfigSnippets.py](otelConfigs/consConfigSnippets.py)). +A shell script ([makeConfigFiles.sh](otelConfigs/makeConfigFiles.sh)) runs 5 configuration sets +to output the otel-config-*.yaml files that are used by the +[otelCollector image](src/main/docker/otelCollector/Dockerfile) and the +[compose configurations](src/main/docker/composeExtensions/). The compose configurations override +the original otel collector configurations with new ones. Those compose files also vary by specifying +different ports depending upon which services have been configured (Prometheus, zpages, etc ports). + +Those configurations are created by merging a set of YAML snippets into a final file. Within the +dependencies.yml file, parents (and their respective snippets) are dependencies of their children. +As consConfigSnippets.py is invoked, all ancestors' snippets are included before the children that +were specified. To further simplify management of dependencies, snippets may have multiple dependencies. +Upon hitting each dependency for the first time, all of its dependencies are also found and included +within the final yaml configuration that is being output. + +## Compatibility The tools in this directory can only be built if you have Java version 11 installed. diff --git a/TrafficCapture/dockerSolution/build.gradle b/TrafficCapture/dockerSolution/build.gradle index 5d6258042..f65a60d20 100644 --- a/TrafficCapture/dockerSolution/build.gradle +++ b/TrafficCapture/dockerSolution/build.gradle @@ -5,9 +5,7 @@ plugins { } import org.opensearch.migrations.common.CommonUtils -import java.security.MessageDigest import com.bmuschko.gradle.docker.tasks.image.DockerBuildImage -import org.apache.tools.ant.taskdefs.condition.Os def calculateDockerHash = { projectName -> CommonUtils.calculateDockerHash(projectName, project) @@ -22,6 +20,7 @@ def dockerFilesForExternalServices = [ "elasticsearchWithSearchGuard": "elasticsearch_searchguard", "migrationConsole": "migration_console", "opensearchDashboards": "opensearch_dashboards", + "otelCollector": "otel_collector" ] // Create the static docker files that aren't hosting migrations java code from this repo dockerFilesForExternalServices.each { projectName, dockerImageName -> @@ -59,15 +58,21 @@ javaContainerServices.forEach { projectName, dockerImageName -> } dockerCompose { - useComposeFiles = project.hasProperty('multiProxy') ? - ['src/main/docker/docker-compose.yml', 'src/main/docker/docker-compose-multi.yml'] : - ['src/main/docker/docker-compose.yml', 'src/main/docker/docker-compose-single.yml'] + def extensionsDir = "src/main/docker/composeExtensions/" + useComposeFiles = + ['src/main/docker/docker-compose.yml', + "${extensionsDir}/" + (project.hasProperty("otel-collector") ? + "${project.getProperty('otel-collector')}" : + "otel-prometheus-jaeger-opensearch.yml"), + "${extensionsDir}" + (project.hasProperty("multiProxy") ? "proxy-multi.yml" : "proxy-single.yml") + ] } task buildDockerImages { dependsOn buildDockerImage_elasticsearchWithSearchGuard dependsOn buildDockerImage_migrationConsole dependsOn buildDockerImage_opensearchDashboards + dependsOn buildDockerImage_otelCollector dependsOn buildDockerImage_trafficCaptureProxyServer dependsOn buildDockerImage_trafficReplayer diff --git a/TrafficCapture/dockerSolution/otelConfigs/README.md b/TrafficCapture/dockerSolution/otelConfigs/README.md new file mode 100644 index 000000000..401432f88 --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/README.md @@ -0,0 +1,31 @@ +# PRE-REQUISITE + +* Python3 and venv + +## Activate your Python virtual environment + +To isolate the Python environment for the project from your local machine, create virtual environment like so: +``` +python3 -m venv .venv +source .venv/bin/activate +pip install -r requirements.txt +``` + +You can exit the Python virtual environment and remove its resources like so: +``` +deactivate +rm -rf .venv +``` + +Learn more about venv [here](https://docs.python.org/3/library/venv.html). + +## Create otel-collector config files + +Run `consConfigSnippets.py` with the snippet components (without the .yaml extension) +that you want to include in the output (stdout). Common dependencies (as determined +by dependencies.yaml) will only be included in the final output once. + +For example +``` +python3 consConfigSnippets.py base +``` \ No newline at end of file diff --git a/TrafficCapture/dockerSolution/otelConfigs/configSnippets/awsCloudWatch.yaml b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/awsCloudWatch.yaml new file mode 100644 index 000000000..6f6b8c657 --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/awsCloudWatch.yaml @@ -0,0 +1,8 @@ +exporters: + awsemf: + namespace: 'TrafficCaptureReplay' + +service: + pipelines: + metrics: + exporters: [ awsemf ] diff --git a/TrafficCapture/dockerSolution/otelConfigs/configSnippets/awsXRay.yaml b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/awsXRay.yaml new file mode 100644 index 000000000..37d4a0474 --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/awsXRay.yaml @@ -0,0 +1,8 @@ +exporters: + awsxray: + index_all_attributes: true + +service: + pipelines: + traces: + exporters: [ awsxray ] diff --git a/TrafficCapture/dockerSolution/otelConfigs/configSnippets/base.yaml b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/base.yaml new file mode 100644 index 000000000..20b605dfe --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/base.yaml @@ -0,0 +1,14 @@ +receivers: + otlp: + protocols: + grpc: + +processors: + +extensions: + +exporters: + +service: + extensions: + pipelines: diff --git a/TrafficCapture/dockerSolution/otelConfigs/configSnippets/basicAuthClient.yaml b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/basicAuthClient.yaml new file mode 100644 index 000000000..4e81c6d90 --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/basicAuthClient.yaml @@ -0,0 +1,5 @@ +extensions: + basicauth/client: + client_auth: + username: "admin" + password: "admin" diff --git a/TrafficCapture/dockerSolution/otelConfigs/configSnippets/batch.yaml b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/batch.yaml new file mode 100644 index 000000000..c5b44dc1e --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/batch.yaml @@ -0,0 +1,5 @@ +processors: + batch: + timeout: 10s + send_batch_size: 8192 + send_batch_max_size: 10000 \ No newline at end of file diff --git a/TrafficCapture/dockerSolution/otelConfigs/configSnippets/batchMetrics.yaml b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/batchMetrics.yaml new file mode 100644 index 000000000..c755351ad --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/batchMetrics.yaml @@ -0,0 +1,4 @@ +service: + pipelines: + metrics: + processors: [ batch ] diff --git a/TrafficCapture/dockerSolution/otelConfigs/configSnippets/batchTraces.yaml b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/batchTraces.yaml new file mode 100644 index 000000000..a32aff59d --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/batchTraces.yaml @@ -0,0 +1,4 @@ +service: + pipelines: + traces: + processors: [ batch ] diff --git a/TrafficCapture/dockerSolution/otelConfigs/configSnippets/debugDetailed.yaml b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/debugDetailed.yaml new file mode 100644 index 000000000..1568fb276 --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/debugDetailed.yaml @@ -0,0 +1,5 @@ +exporters: + logging: + verbosity: detailed + sampling_initial: 5 + sampling_thereafter: 200 \ No newline at end of file diff --git a/TrafficCapture/dockerSolution/otelConfigs/configSnippets/debugInfo.yaml b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/debugInfo.yaml new file mode 100644 index 000000000..760aabf66 --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/debugInfo.yaml @@ -0,0 +1,5 @@ +exporters: + logging: + verbosity: info + sampling_initial: 5 + sampling_thereafter: 200 \ No newline at end of file diff --git a/TrafficCapture/dockerSolution/otelConfigs/configSnippets/debugLogsDetailed.yaml b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/debugLogsDetailed.yaml new file mode 100644 index 000000000..04c30b4a9 --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/debugLogsDetailed.yaml @@ -0,0 +1,4 @@ +service: + pipelines: + logs: + exporters: [logging] \ No newline at end of file diff --git a/TrafficCapture/dockerSolution/otelConfigs/configSnippets/debugLogsInfo.yaml b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/debugLogsInfo.yaml new file mode 100644 index 000000000..04c30b4a9 --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/debugLogsInfo.yaml @@ -0,0 +1,4 @@ +service: + pipelines: + logs: + exporters: [logging] \ No newline at end of file diff --git a/TrafficCapture/dockerSolution/otelConfigs/configSnippets/debugMetricsDetailed.yaml b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/debugMetricsDetailed.yaml new file mode 100644 index 000000000..d7b4f497c --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/debugMetricsDetailed.yaml @@ -0,0 +1,4 @@ +service: + pipelines: + metrics: + exporters: [logging] \ No newline at end of file diff --git a/TrafficCapture/dockerSolution/otelConfigs/configSnippets/debugMetricsInfo.yaml b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/debugMetricsInfo.yaml new file mode 100644 index 000000000..d7b4f497c --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/debugMetricsInfo.yaml @@ -0,0 +1,4 @@ +service: + pipelines: + metrics: + exporters: [logging] \ No newline at end of file diff --git a/TrafficCapture/dockerSolution/otelConfigs/configSnippets/debugTracesDetailed.yaml b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/debugTracesDetailed.yaml new file mode 100644 index 000000000..c1e7f9541 --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/debugTracesDetailed.yaml @@ -0,0 +1,4 @@ +service: + pipelines: + traces: + exporters: [logging] \ No newline at end of file diff --git a/TrafficCapture/dockerSolution/otelConfigs/configSnippets/debugTracesInfo.yaml b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/debugTracesInfo.yaml new file mode 100644 index 000000000..c1e7f9541 --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/debugTracesInfo.yaml @@ -0,0 +1,4 @@ +service: + pipelines: + traces: + exporters: [logging] \ No newline at end of file diff --git a/TrafficCapture/dockerSolution/otelConfigs/configSnippets/healthCheck.yaml b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/healthCheck.yaml new file mode 100644 index 000000000..5e20967f1 --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/healthCheck.yaml @@ -0,0 +1,5 @@ +extensions: + health_check: + +service: + extensions: [ health_check ] \ No newline at end of file diff --git a/TrafficCapture/dockerSolution/otelConfigs/configSnippets/jaeger.yaml b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/jaeger.yaml new file mode 100644 index 000000000..127d7d56e --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/jaeger.yaml @@ -0,0 +1,10 @@ +exporters: + otlp/jaeger: # Jaeger supports OTLP directly. The default port for OTLP/gRPC is 4317 + endpoint: jaeger:4317 + tls: + insecure: true + +service: + pipelines: + traces: + exporters: [ otlp/jaeger ] \ No newline at end of file diff --git a/TrafficCapture/dockerSolution/otelConfigs/configSnippets/logs.yaml b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/logs.yaml new file mode 100644 index 000000000..cb93ed1e6 --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/logs.yaml @@ -0,0 +1,6 @@ +service: + pipelines: + logs: + receivers: [ otlp ] + processors: + exporters: \ No newline at end of file diff --git a/TrafficCapture/dockerSolution/otelConfigs/configSnippets/metrics.yaml b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/metrics.yaml new file mode 100644 index 000000000..6f9ea4b5d --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/metrics.yaml @@ -0,0 +1,6 @@ +service: + pipelines: + metrics: + receivers: [ otlp ] + processors: + exporters: \ No newline at end of file diff --git a/TrafficCapture/dockerSolution/otelConfigs/configSnippets/openSearch.yaml b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/openSearch.yaml new file mode 100644 index 000000000..60a70e82e --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/openSearch.yaml @@ -0,0 +1,39 @@ +processors: + attributes: + # This processor is currently renaming two attributes + # that are prefixed with `log4j.context_data.` to the base attribute name + # to make queries within OpenSearch clearer. Both the `insert from_attribute` + # and the `delete` actions will fail silently if the attribute is not present, + # which means that these are safe for events that both do and don't have these + # attributes. This pattern should be extended to all of our standard attributes. + actions: + - key: event + from_attribute: log4j.context_data.event + action: insert + - key: log4j.context_data.event + action: delete + - key: channel_id + from_attribute: log4j.context_data.channel_id + action: insert + - key: log4j.context_data.channel_id + action: delete + # The following actions convert various should-be-int strings to ints + - key: log4j.context_data.source_http_status + action: convert + converted_type: int + - key: log4j.context_data.target_http_status + action: convert + converted_type: int + - key: log4j.context_data.http_status_match + action: convert + converted_type: int + +exporters: + opensearch: + namespace: migrations + +service: + pipelines: + logs: + processors: [ attributes ] + exporters: [ opensearch ] diff --git a/TrafficCapture/dockerSolution/otelConfigs/configSnippets/openSearchAws.yaml b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/openSearchAws.yaml new file mode 100644 index 000000000..21e6b60de --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/openSearchAws.yaml @@ -0,0 +1,4 @@ +exporters: + opensearch: + http: + endpoint: "${ANALYTICS_DOMAIN_ENDPOINT}" \ No newline at end of file diff --git a/TrafficCapture/dockerSolution/otelConfigs/configSnippets/openSearchLocal.yaml b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/openSearchLocal.yaml new file mode 100644 index 000000000..ef1e17432 --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/openSearchLocal.yaml @@ -0,0 +1,11 @@ +exporters: + opensearch: + http: + endpoint: "https://opensearchanalytics:9200" + auth: + authenticator: basicauth/client + tls: + insecure_skip_verify: true + +service: + extensions: [ basicauth/client ] \ No newline at end of file diff --git a/TrafficCapture/dockerSolution/otelConfigs/configSnippets/pprof.yaml b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/pprof.yaml new file mode 100644 index 000000000..f14fe1283 --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/pprof.yaml @@ -0,0 +1,6 @@ +extensions: + pprof: + endpoint: :1888 + +service: + extensions: [ pprof ] \ No newline at end of file diff --git a/TrafficCapture/dockerSolution/otelConfigs/configSnippets/prometheus.yaml b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/prometheus.yaml new file mode 100644 index 000000000..a2d885d64 --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/prometheus.yaml @@ -0,0 +1,11 @@ +exporters: + prometheus: + endpoint: "0.0.0.0:8889" + send_timestamps: true + metric_expiration: 5m + enable_open_metrics: true + +service: + pipelines: + metrics: + exporters: [ prometheus ] \ No newline at end of file diff --git a/TrafficCapture/dockerSolution/otelConfigs/configSnippets/traces.yaml b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/traces.yaml new file mode 100644 index 000000000..6fcd87eac --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/traces.yaml @@ -0,0 +1,6 @@ +service: + pipelines: + traces: + receivers: [ otlp ] + processors: + exporters: \ No newline at end of file diff --git a/TrafficCapture/dockerSolution/otelConfigs/configSnippets/zpages.yaml b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/zpages.yaml new file mode 100644 index 000000000..2f7d48dc0 --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/configSnippets/zpages.yaml @@ -0,0 +1,6 @@ +extensions: + zpages: + endpoint: :55679 + +service: + extensions: [ zpages] \ No newline at end of file diff --git a/TrafficCapture/dockerSolution/otelConfigs/consConfigSnippets.py b/TrafficCapture/dockerSolution/otelConfigs/consConfigSnippets.py new file mode 100644 index 000000000..77ad1c4ad --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/consConfigSnippets.py @@ -0,0 +1,52 @@ +import sys +import subprocess +import yaml + + +def collect_snippet_dependencies(key, original_dict, + depth, + ground_truth_dict, already_collected_set, + found_at_depth_map): + + already_collected_set.add(key) + if ground_truth_dict is None: + return False + found_match = False + for parent, innerMap in ground_truth_dict.items(): + if parent == key or collect_snippet_dependencies(key, original_dict, depth + 1, + innerMap, already_collected_set, + found_at_depth_map): + if parent not in already_collected_set: + collect_snippet_dependencies(parent, original_dict, 0, + original_dict, already_collected_set, found_at_depth_map) + found_at_depth_map[parent] = depth + found_match = True + return found_match + + +def construct_command(selected_keys, deps): + dependency_depth_dict = dict() + for key in selected_keys: + foundKey = collect_snippet_dependencies(key, deps, 0, deps, set(), dependency_depth_dict) + assert foundKey, f"key={key}" + ordered_snippets = sorted(dependency_depth_dict, key=lambda k: dependency_depth_dict[k]) + + files = ' '.join([f"configSnippets/{dep}.yaml" for dep in ordered_snippets]) + return f"yq eval-all '. as $item ireduce ({{}}; . *+ $item )' {files}" + + +def run_command(command): + subprocess.run(command, shell=True, text=True) + + +def main(selected_keys): + with open('dependencies.yml', 'r') as file: + deps = yaml.safe_load(file) + + command = construct_command(selected_keys, deps) + run_command(command) + + +if __name__ == "__main__": + args = sys.argv[1:] # Arguments from command line + main(args) diff --git a/TrafficCapture/dockerSolution/otelConfigs/dependencies.yml b/TrafficCapture/dockerSolution/otelConfigs/dependencies.yml new file mode 100644 index 000000000..a98a4cf9a --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/dependencies.yml @@ -0,0 +1,35 @@ +base: + metrics: + batchMetrics: + prometheus: + awsCloudWatch: + debugMetricsDetailed: + infoMetricsDetailed: + traces: + batchTraces: + awsXRay: + jaeger: + debugTracesDetailed: + infoTracessDetailed: + logs: + openSearch: + openSearchAws: + openSearchLocal: + debugLogsDetailed: + infoLogsDetailed: + basicAuthClient: + openSearchLocal: + batch: + batchMetrics: + batchTraces: + debugDetailed: + debugMetricsDetailed: + debugLogsDetailed: + debugTracesDetailed: + debugInfo: + debugMetricsInfo: + debugLogsInfo: + debugTracesInfo: + healthCheck: + pprof: + zpages: \ No newline at end of file diff --git a/TrafficCapture/dockerSolution/otelConfigs/makeConfigFiles.sh b/TrafficCapture/dockerSolution/otelConfigs/makeConfigFiles.sh new file mode 100755 index 000000000..c6b96f61e --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/makeConfigFiles.sh @@ -0,0 +1,20 @@ +#!/bin/sh + +python3 -m venv .venv +source .venv/bin/activate +pip install -r requirements.txt + +DOCKER_IMAGE_BUILTINS=../src/main/docker/otelCollector + +python3 consConfigSnippets.py awsCloudWatch awsXRay openSearchAws healthCheck > "${DOCKER_IMAGE_BUILTINS}/otel-config-aws.yaml" +python3 consConfigSnippets.py awsCloudWatch awsXRay openSearchAws healthCheck debugMetricsDetailed debugTracesDetailed debugLogsDetailed > "${DOCKER_IMAGE_BUILTINS}/otel-config-aws-debug.yaml" +python3 consConfigSnippets.py zpages pprof healthCheck debugTracesDetailed debugMetricsDetailed debugTracesDetailed debugLogsDetailed > "${DOCKER_IMAGE_BUILTINS}/otel-config-debug-only.yaml" + +COMPOSE_EXTENSIONS=../src/main/docker/composeExtensions/configs +python3 consConfigSnippets.py awsCloudWatch awsXRay prometheus jaeger openSearchLocal zpages pprof healthCheck debugMetricsDetailed debugTracesDetailed debugLogsDetailed > "${COMPOSE_EXTENSIONS}/otel-config-everything.yaml" +python3 consConfigSnippets.py prometheus jaeger openSearchLocal zpages pprof healthCheck debugMetricsDetailed debugTracesDetailed debugLogsDetailed > "${COMPOSE_EXTENSIONS}/otel-config-prometheus-jaeger-opensearch.yaml" + +echo Done making files + +deactivate +rm -rf .venv \ No newline at end of file diff --git a/TrafficCapture/dockerSolution/otelConfigs/requirements.txt b/TrafficCapture/dockerSolution/otelConfigs/requirements.txt new file mode 100644 index 000000000..dbfc7099c --- /dev/null +++ b/TrafficCapture/dockerSolution/otelConfigs/requirements.txt @@ -0,0 +1 @@ +PyYAML \ No newline at end of file diff --git a/TrafficCapture/dockerSolution/src/main/docker/composeExtensions/configs/otel-config-everything.yaml b/TrafficCapture/dockerSolution/src/main/docker/composeExtensions/configs/otel-config-everything.yaml new file mode 100644 index 000000000..f9ba3eedf --- /dev/null +++ b/TrafficCapture/dockerSolution/src/main/docker/composeExtensions/configs/otel-config-everything.yaml @@ -0,0 +1,88 @@ +receivers: + otlp: + protocols: + grpc: +processors: + batch: + timeout: 10s + send_batch_size: 8192 + send_batch_max_size: 10000 + attributes: + # This processor is currently renaming two attributes + # that are prefixed with `log4j.context_data.` to the base attribute name + # to make queries within OpenSearch clearer. Both the `insert from_attribute` + # and the `delete` actions will fail silently if the attribute is not present, + # which means that these are safe for events that both do and don't have these + # attributes. This pattern should be extended to all of our standard attributes. + actions: + - key: event + from_attribute: log4j.context_data.event + action: insert + - key: log4j.context_data.event + action: delete + - key: channel_id + from_attribute: log4j.context_data.channel_id + action: insert + - key: log4j.context_data.channel_id + action: delete + # The following actions convert various should-be-int strings to ints + - key: log4j.context_data.source_http_status + action: convert + converted_type: int + - key: log4j.context_data.target_http_status + action: convert + converted_type: int + - key: log4j.context_data.http_status_match + action: convert + converted_type: int +extensions: + basicauth/client: + client_auth: + username: "admin" + password: "admin" + zpages: + endpoint: :55679 + pprof: + endpoint: :1888 + health_check: +exporters: + logging: + verbosity: detailed + sampling_initial: 5 + sampling_thereafter: 200 + opensearch: + http: + endpoint: "https://opensearchanalytics:9200" + auth: + authenticator: basicauth/client + tls: + insecure_skip_verify: true + namespace: migrations + awsemf: + namespace: 'TrafficCaptureReplay' + awsxray: + index_all_attributes: true + prometheus: + endpoint: "0.0.0.0:8889" + send_timestamps: true + metric_expiration: 5m + enable_open_metrics: true + otlp/jaeger: # Jaeger supports OTLP directly. The default port for OTLP/gRPC is 4317 + endpoint: jaeger:4317 + tls: + insecure: true +service: + extensions: [zpages, pprof, health_check, basicauth/client] + pipelines: + metrics: + receivers: [otlp] + processors: [batch] + exporters: [logging, awsemf, prometheus] + traces: + receivers: [otlp] + processors: [batch] + exporters: [logging, awsxray, otlp/jaeger] + logs: + receivers: [otlp] + processors: [attributes] + exporters: [opensearch, logging] diff --git a/TrafficCapture/dockerSolution/src/main/docker/composeExtensions/configs/otel-config-prometheus-jaeger-opensearch.yaml b/TrafficCapture/dockerSolution/src/main/docker/composeExtensions/configs/otel-config-prometheus-jaeger-opensearch.yaml new file mode 100644 index 000000000..7e418eba8 --- /dev/null +++ b/TrafficCapture/dockerSolution/src/main/docker/composeExtensions/configs/otel-config-prometheus-jaeger-opensearch.yaml @@ -0,0 +1,84 @@ +receivers: + otlp: + protocols: + grpc: +processors: + batch: + timeout: 10s + send_batch_size: 8192 + send_batch_max_size: 10000 + attributes: + # This processor is currently renaming two attributes + # that are prefixed with `log4j.context_data.` to the base attribute name + # to make queries within OpenSearch clearer. Both the `insert from_attribute` + # and the `delete` actions will fail silently if the attribute is not present, + # which means that these are safe for events that both do and don't have these + # attributes. This pattern should be extended to all of our standard attributes. + actions: + - key: event + from_attribute: log4j.context_data.event + action: insert + - key: log4j.context_data.event + action: delete + - key: channel_id + from_attribute: log4j.context_data.channel_id + action: insert + - key: log4j.context_data.channel_id + action: delete + # The following actions convert various should-be-int strings to ints + - key: log4j.context_data.source_http_status + action: convert + converted_type: int + - key: log4j.context_data.target_http_status + action: convert + converted_type: int + - key: log4j.context_data.http_status_match + action: convert + converted_type: int +extensions: + basicauth/client: + client_auth: + username: "admin" + password: "admin" + zpages: + endpoint: :55679 + pprof: + endpoint: :1888 + health_check: +exporters: + logging: + verbosity: detailed + sampling_initial: 5 + sampling_thereafter: 200 + opensearch: + http: + endpoint: "https://opensearchanalytics:9200" + auth: + authenticator: basicauth/client + tls: + insecure_skip_verify: true + namespace: migrations + prometheus: + endpoint: "0.0.0.0:8889" + send_timestamps: true + metric_expiration: 5m + enable_open_metrics: true + otlp/jaeger: # Jaeger supports OTLP directly. The default port for OTLP/gRPC is 4317 + endpoint: jaeger:4317 + tls: + insecure: true +service: + extensions: [zpages, pprof, health_check, basicauth/client] + pipelines: + metrics: + receivers: [otlp] + processors: [batch] + exporters: [logging, prometheus] + traces: + receivers: [otlp] + processors: [batch] + exporters: [logging, otlp/jaeger] + logs: + receivers: [otlp] + processors: [attributes] + exporters: [opensearch, logging] diff --git a/TrafficCapture/dockerSolution/src/main/docker/composeExtensions/otel-aws-debug.yml b/TrafficCapture/dockerSolution/src/main/docker/composeExtensions/otel-aws-debug.yml new file mode 100644 index 000000000..c1035379a --- /dev/null +++ b/TrafficCapture/dockerSolution/src/main/docker/composeExtensions/otel-aws-debug.yml @@ -0,0 +1,18 @@ +version: '3.7' +services: + + otel-collector: + image: migrations/otel_collector:latest + command: ["--config=/etc/otel-config-aws-debug.yaml", "${OTELCOL_ARGS}"] + networks: + - migrations + ports: + - "13133:13133" # health_check extension + - "4317:4317" # otlp receiver + volumes: + - ~/.aws:/home/aoc/.aws + environment: + - ANALYTICS_DOMAIN_ENDPOINT=opensearchanalytics # use the local container for compose here + - AWS_REGION=us-east-2 + - AWS_DEFAULT_REGION=us-east-2 + - AWS_PROFILE=default \ No newline at end of file diff --git a/TrafficCapture/dockerSolution/src/main/docker/composeExtensions/otel-aws.yml b/TrafficCapture/dockerSolution/src/main/docker/composeExtensions/otel-aws.yml new file mode 100644 index 000000000..40f776eca --- /dev/null +++ b/TrafficCapture/dockerSolution/src/main/docker/composeExtensions/otel-aws.yml @@ -0,0 +1,18 @@ +version: '3.7' +services: + + otel-collector: + image: migrations/otel_collector:latest + command: ["--config=/etc/otel-config-aws.yaml", "${OTELCOL_ARGS}"] + networks: + - migrations + ports: + - "13133:13133" # health_check extension + - "4317:4317" # otlp receiver + volumes: + - ~/.aws:/home/aoc/.aws + environment: + - ANALYTICS_DOMAIN_ENDPOINT=opensearchanalytics # use the local container for compose here + - AWS_REGION=us-east-2 + - AWS_DEFAULT_REGION=us-east-2 + - AWS_PROFILE=default \ No newline at end of file diff --git a/TrafficCapture/dockerSolution/src/main/docker/composeExtensions/otel-everything.yml b/TrafficCapture/dockerSolution/src/main/docker/composeExtensions/otel-everything.yml new file mode 100644 index 000000000..6c3cbe32e --- /dev/null +++ b/TrafficCapture/dockerSolution/src/main/docker/composeExtensions/otel-everything.yml @@ -0,0 +1,23 @@ +version: '3.7' +services: + + otel-collector: + image: migrations/otel_collector:latest + command: ["--config=/etc/otel-config-everything.yaml", "${OTELCOL_ARGS}"] + networks: + - migrations + volumes: + - ./composeExtensions/configs/otel-config-everything.yaml:/etc/otel-config-everything.yaml + - ~/.aws:/home/aoc/.aws + ports: + - "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 + environment: + - AWS_REGION=us-east-2 + - AWS_DEFAULT_REGION=us-east-2 + - AWS_PROFILE=default \ No newline at end of file diff --git a/TrafficCapture/dockerSolution/src/main/docker/composeExtensions/otel-image-default-config.yml b/TrafficCapture/dockerSolution/src/main/docker/composeExtensions/otel-image-default-config.yml new file mode 100644 index 000000000..58706136c --- /dev/null +++ b/TrafficCapture/dockerSolution/src/main/docker/composeExtensions/otel-image-default-config.yml @@ -0,0 +1,14 @@ +version: '3.7' +services: + + otel-collector: + image: migrations/otel_collector:latest + networks: + - migrations + volumes: + - ./composeExtensions/configs/otel-config-prometheus-jaeger-opensearch.yaml:/etc/otel-config-prometheus-jaeger-opensearch.yaml + ports: + - "1888:1888" # pprof extension + - "13133:13133" # health_check extension + - "55679:55679" # zpages extension + - "4317:4317" # otlp receiver diff --git a/TrafficCapture/dockerSolution/src/main/docker/composeExtensions/otel-prometheus-jaeger-opensearch.yml b/TrafficCapture/dockerSolution/src/main/docker/composeExtensions/otel-prometheus-jaeger-opensearch.yml new file mode 100644 index 000000000..719995c01 --- /dev/null +++ b/TrafficCapture/dockerSolution/src/main/docker/composeExtensions/otel-prometheus-jaeger-opensearch.yml @@ -0,0 +1,19 @@ +version: '3.7' +services: + + otel-collector: + image: migrations/otel_collector:latest + command: ["--config=/etc/otel-config-prometheus-jaeger-opensearch.yaml", "${OTELCOL_ARGS}"] + networks: + - migrations + volumes: + - ./composeExtensions/configs/otel-config-prometheus-jaeger-opensearch.yaml:/etc/otel-config-prometheus-jaeger-opensearch.yaml + 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 \ No newline at end of file diff --git a/TrafficCapture/dockerSolution/src/main/docker/docker-compose-multi.yml b/TrafficCapture/dockerSolution/src/main/docker/composeExtensions/proxy-multi.yml similarity index 100% rename from TrafficCapture/dockerSolution/src/main/docker/docker-compose-multi.yml rename to TrafficCapture/dockerSolution/src/main/docker/composeExtensions/proxy-multi.yml diff --git a/TrafficCapture/dockerSolution/src/main/docker/docker-compose-single.yml b/TrafficCapture/dockerSolution/src/main/docker/composeExtensions/proxy-single.yml similarity index 87% rename from TrafficCapture/dockerSolution/src/main/docker/docker-compose-single.yml rename to TrafficCapture/dockerSolution/src/main/docker/composeExtensions/proxy-single.yml index febda3d42..0ae8ffbcb 100644 --- a/TrafficCapture/dockerSolution/src/main/docker/docker-compose-single.yml +++ b/TrafficCapture/dockerSolution/src/main/docker/composeExtensions/proxy-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 798ca1071..bff5b3c42 100644 --- a/TrafficCapture/dockerSolution/src/main/docker/docker-compose.yml +++ b/TrafficCapture/dockerSolution/src/main/docker/docker-compose.yml @@ -1,39 +1,44 @@ 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: + - "9090:9090" + command: + - '--config.file=/etc/prometheus/prometheus.yml' + - '--enable-feature=exemplar-storage' + + # Jaeger + jaeger: + image: jaegertracing/all-in-one:latest networks: - migrations ports: - - "9200:9200" - - "19200:19200" + - "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' + grafana: + image: grafana/grafana:latest + networks: + - migrations + ports: + - "3000:3000" + volumes: + - ./grafana_data:/var/lib/grafana + environment: + - GF_SECURITY_ADMIN_PASSWORD=admin + depends_on: + - prometheus zookeeper: image: docker.io/bitnami/zookeeper:3.8 @@ -111,18 +116,19 @@ 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-debug-only.yaml:/etc/otel-config-debug-only.yaml +# networks: +# - migrations +# depends_on: +# - opensearchanalytics +# command: tail -f /dev/null migration-console: image: 'migrations/migration_console:latest' @@ -132,6 +138,7 @@ services: - sharedReplayerOutput:/shared-replayer-output environment: - MIGRATION_KAFKA_BROKER_ENDPOINTS=kafka:9092 +# command: ./runTestBenchmarks.sh volumes: zookeeper_data: @@ -142,6 +149,8 @@ volumes: driver: local sharedReplayerOutput: driver: local + grafana_data: + driver: local networks: migrations: diff --git a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/Dockerfile b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/Dockerfile index 69186c587..26e9c7c4d 100644 --- a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/Dockerfile +++ b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/Dockerfile @@ -11,13 +11,16 @@ RUN mkdir /root/kafka-tools RUN mkdir /root/kafka-tools/aws COPY runTestBenchmarks.sh /root/ COPY humanReadableLogs.py /root/ +COPY simpleDocumentGenerator.py /root/ COPY catIndices.sh /root/ COPY showFetchMigrationCommand.sh /root/ COPY setupIntegTests.sh /root/ COPY msk-iam-auth.properties /root/kafka-tools/aws COPY kafkaCmdRef.md /root/kafka-tools +RUN ln -s /usr/bin/python3 /usr/bin/python RUN chmod ug+x /root/runTestBenchmarks.sh RUN chmod ug+x /root/humanReadableLogs.py +RUN chmod ug+x /root/simpleDocumentGenerator.py RUN chmod ug+x /root/catIndices.sh RUN chmod ug+x /root/showFetchMigrationCommand.sh WORKDIR /root/kafka-tools diff --git a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/runTestBenchmarks.sh b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/runTestBenchmarks.sh index 9057c2889..558df039a 100644 --- a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/runTestBenchmarks.sh +++ b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/runTestBenchmarks.sh @@ -5,7 +5,6 @@ endpoint="https://capture-proxy-es:9200" auth_user="admin" auth_pass="admin" no_auth=false -no_ssl=false usage() { echo "" @@ -47,10 +46,6 @@ while [[ $# -gt 0 ]]; do no_auth=true shift ;; - --no-ssl) - no_ssl=true - shift - ;; -h|--h|--help) usage ;; @@ -64,21 +59,24 @@ while [[ $# -gt 0 ]]; do esac done -# Populate auth string -if [ "$no_auth" = true ]; then - auth_string="" -else - auth_string=",basic_auth_user:${auth_user},basic_auth_password:${auth_pass}" + +# Initialize an empty array to hold non-empty values +options=() + +if [[ "$endpoint" == https:* ]]; then + options+=("use_ssl:true,verify_certs:false") fi -if [ "$no_ssl" = true ]; then - base_options_string="" -else - base_options_string="use_ssl:true,verify_certs:false" + +# Populate auth string +if [ "$no_auth" = false ]; then + options+=("basic_auth_user:${auth_user},basic_auth_password:${auth_pass}") fi -# Construct the final client options string -client_options="${base_options_string}${auth_string}" +# Join the non-empty values using a comma +client_options=$(IFS=,; echo "${options[*]}") + +set -o xtrace echo "Running opensearch-benchmark workloads against ${endpoint}" echo "Running opensearch-benchmark w/ 'geonames' workload..." && diff --git a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/simpleDocumentGenerator.py b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/simpleDocumentGenerator.py new file mode 100644 index 000000000..e22ceece5 --- /dev/null +++ b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/simpleDocumentGenerator.py @@ -0,0 +1,74 @@ +#!/usr/bin/env python +import requests +import time +import argparse +from datetime import datetime + +# url_base="http://test.elb.us-west-2.amazonaws.com:9200" +username = 'admin' +password = 'admin' + +session = requests.Session() +keep_alive_headers = { + 'Connection': 'keep-alive' +} + + +# Function to get current date in a specific format for indexing +def get_current_date_index(): + return datetime.now().strftime("%Y-%m-%d") + + +# Function to send a request +def send_request(index, counter, url_base): + url = f"{url_base}/{index}/_doc/{counter}" + timestamp = datetime.now().isoformat() + # Basic Authentication + auth = (username, password) + payload = { + "timestamp": timestamp, + "new_field": "apple" + } + + try: + # a new connection for every request + #response = requests.put(url, json=payload, auth=auth) + response = session.put(url, json=payload, auth=auth, headers=keep_alive_headers, verify=False) + print(response.text) + print(f"Request sent at {timestamp}: {response.status_code}") + return response.status_code + except requests.RequestException as e: + print(f"Error sending request: {e}") + return None + + +def parse_args(): + parser = argparse.ArgumentParser() + parser.add_argument("--endpoint", help="Source cluster endpoint e.g. http://test.elb.us-west-2.amazonaws.com:9200.") + return parser.parse_args() + + +args = parse_args() +# Main loop +counter = 1 +total2xxCount = 0 +total4xxCount = 0 +total5xxCount = 0 +totalErrorCount = 0 +while True: + current_index = get_current_date_index() + response_code = send_request(current_index, counter, args.endpoint) + if (response_code is not None): + first_digit = int(str(response_code)[:1]) + if (first_digit == 2): + total2xxCount += 1 + elif (first_digit == 4): + total4xxCount += 1 + elif (first_digit == 5): + total5xxCount += 1 + else: + totalErrorCount += 1 + print(f"Summary: 2xx responses = {total2xxCount}, 4xx responses = {total4xxCount}, " + f"5xx responses = {total5xxCount}, Error requests = {totalErrorCount}") + counter += 1 + time.sleep(0.1) diff --git a/TrafficCapture/dockerSolution/src/main/docker/otelCollector/Dockerfile b/TrafficCapture/dockerSolution/src/main/docker/otelCollector/Dockerfile new file mode 100644 index 000000000..e5083c6bf --- /dev/null +++ b/TrafficCapture/dockerSolution/src/main/docker/otelCollector/Dockerfile @@ -0,0 +1,14 @@ +FROM public.ecr.aws/a0w2c5q7/otelcol-with-opensearch:amd-latest +#FROM public.ecr.aws/aws-observability/aws-otel-collector:v0.37.0 + +COPY otel-config*.yaml /etc/ +COPY exportCredsAndRun.sh / +RUN chmod ugo+x exportCredsAndRun.sh + +# Make this image consistent with the AWS Distro for OpenTelemetry. That +# is a leaner (and official) image that still has what we need, except for +# the OpenSearch logs exporter. +RUN useradd -m aoc +USER aoc +ENTRYPOINT ["./exportCredsAndRun.sh", "./otelcontribcol"] +CMD ["--config", "/etc/otel-config-debug-only.yaml"] diff --git a/TrafficCapture/dockerSolution/src/main/docker/otelcol/README.md b/TrafficCapture/dockerSolution/src/main/docker/otelCollector/README.md similarity index 100% rename from TrafficCapture/dockerSolution/src/main/docker/otelcol/README.md rename to TrafficCapture/dockerSolution/src/main/docker/otelCollector/README.md diff --git a/TrafficCapture/dockerSolution/src/main/docker/otelCollector/exportCredsAndRun.sh b/TrafficCapture/dockerSolution/src/main/docker/otelCollector/exportCredsAndRun.sh new file mode 100644 index 000000000..ce7ad9eaf --- /dev/null +++ b/TrafficCapture/dockerSolution/src/main/docker/otelCollector/exportCredsAndRun.sh @@ -0,0 +1,17 @@ +#!/bin/sh + +PROFILE_NAME=${AWS_PROFILE:-"default"} +echo "Using profile " $PROFILE_NAME + +CREDENTIALS_FILE="$HOME/.aws/credentials" + +# Check if the AWS credentials file exists +if [ -f "$CREDENTIALS_FILE" ]; then + if grep -q "^\[$PROFILE_NAME\]" "$CREDENTIALS_FILE"; then + export AWS_ACCESS_KEY_ID=$(awk -F "=" "/^\[$PROFILE_NAME\]/ {f=1} f==1 && /aws_access_key_id/ {print \$2; exit}" $CREDENTIALS_FILE) + export AWS_SECRET_ACCESS_KEY=$(awk -F "=" "/^\[$PROFILE_NAME\]/ {f=1} f==1 && /aws_secret_access_key/ {print \$2; exit}" $CREDENTIALS_FILE) + export AWS_SESSION_TOKEN=$(awk -F "=" "/^\[$PROFILE_NAME\]/ {f=1} f==1 && /aws_session_token/ {print \$2; exit}" $CREDENTIALS_FILE) + fi +fi + +"$@" diff --git a/TrafficCapture/dockerSolution/src/main/docker/otelcol/otel-config.yml b/TrafficCapture/dockerSolution/src/main/docker/otelCollector/otel-config-aws-debug.yaml similarity index 73% rename from TrafficCapture/dockerSolution/src/main/docker/otelcol/otel-config.yml rename to TrafficCapture/dockerSolution/src/main/docker/otelCollector/otel-config-aws-debug.yaml index f6ccc70e6..b400f2393 100644 --- a/TrafficCapture/dockerSolution/src/main/docker/otelcol/otel-config.yml +++ b/TrafficCapture/dockerSolution/src/main/docker/otelCollector/otel-config-aws-debug.yaml @@ -2,9 +2,11 @@ receivers: otlp: protocols: grpc: - processors: batch: + timeout: 10s + send_batch_size: 8192 + send_batch_max_size: 10000 attributes: # This processor is currently renaming two attributes # that are prefixed with `log4j.context_data.` to the base attribute name @@ -33,34 +35,33 @@ processors: - key: log4j.context_data.http_status_match action: convert converted_type: int - extensions: - basicauth/client: - client_auth: - username: "admin" - password: "admin" health_check: - exporters: + logging: + verbosity: detailed + sampling_initial: 5 + sampling_thereafter: 200 opensearch: namespace: migrations http: - endpoint: "https://opensearchanalytics:9200" - auth: - authenticator: basicauth/client - tls: - insecure_skip_verify: true - logging: - verbosity: detailed - debug: - + endpoint: "${ANALYTICS_DOMAIN_ENDPOINT}" + awsemf: + namespace: 'TrafficCaptureReplay' + awsxray: + index_all_attributes: true service: - extensions: [health_check, basicauth/client] - telemetry: - logs: - level: "debug" + extensions: [health_check] pipelines: + metrics: + receivers: [otlp] + processors: [batch] + exporters: [logging, awsemf] + traces: + receivers: [otlp] + processors: [batch] + exporters: [logging, awsxray] logs: receivers: [otlp] processors: [attributes] - exporters: [logging, debug, opensearch] \ No newline at end of file + exporters: [opensearch, logging] diff --git a/TrafficCapture/dockerSolution/src/main/docker/otelcol/otel-config-cdk.yml b/TrafficCapture/dockerSolution/src/main/docker/otelCollector/otel-config-aws.yaml similarity index 80% rename from TrafficCapture/dockerSolution/src/main/docker/otelcol/otel-config-cdk.yml rename to TrafficCapture/dockerSolution/src/main/docker/otelCollector/otel-config-aws.yaml index 093c6360f..aad185e1a 100644 --- a/TrafficCapture/dockerSolution/src/main/docker/otelcol/otel-config-cdk.yml +++ b/TrafficCapture/dockerSolution/src/main/docker/otelCollector/otel-config-aws.yaml @@ -2,9 +2,11 @@ receivers: otlp: protocols: grpc: - processors: batch: + timeout: 10s + send_batch_size: 8192 + send_batch_max_size: 10000 attributes: # This processor is currently renaming two attributes # that are prefixed with `log4j.context_data.` to the base attribute name @@ -33,26 +35,29 @@ processors: - key: log4j.context_data.http_status_match action: convert converted_type: int - extensions: health_check: - exporters: opensearch: namespace: migrations http: endpoint: "${ANALYTICS_DOMAIN_ENDPOINT}" - logging: - verbosity: detailed - debug: - + awsemf: + namespace: 'TrafficCaptureReplay' + awsxray: + index_all_attributes: true service: extensions: [health_check] - telemetry: - logs: - level: "debug" pipelines: + metrics: + receivers: [otlp] + processors: [batch] + exporters: [awsemf] + traces: + receivers: [otlp] + processors: [batch] + exporters: [awsxray] logs: receivers: [otlp] processors: [attributes] - exporters: [logging, debug, opensearch] \ No newline at end of file + exporters: [opensearch] diff --git a/TrafficCapture/dockerSolution/src/main/docker/otelCollector/otel-config-debug-only.yaml b/TrafficCapture/dockerSolution/src/main/docker/otelCollector/otel-config-debug-only.yaml new file mode 100644 index 000000000..baea39abc --- /dev/null +++ b/TrafficCapture/dockerSolution/src/main/docker/otelCollector/otel-config-debug-only.yaml @@ -0,0 +1,31 @@ +receivers: + otlp: + protocols: + grpc: +processors: +extensions: + zpages: + endpoint: :55679 + pprof: + endpoint: :1888 + health_check: +exporters: + logging: + verbosity: detailed + sampling_initial: 5 + sampling_thereafter: 200 +service: + extensions: [zpages, pprof, health_check] + pipelines: + traces: + receivers: [otlp] + processors: + exporters: [logging] + metrics: + receivers: [otlp] + processors: + exporters: [logging] + logs: + receivers: [otlp] + processors: + exporters: [logging] diff --git a/TrafficCapture/dockerSolution/src/main/docker/otelcol/Dockerfile b/TrafficCapture/dockerSolution/src/main/docker/otelcol/Dockerfile deleted file mode 100644 index 687d8e910..000000000 --- a/TrafficCapture/dockerSolution/src/main/docker/otelcol/Dockerfile +++ /dev/null @@ -1,4 +0,0 @@ -FROM public.ecr.aws/a0w2c5q7/otelcol-with-opensearch:amd-latest - -COPY ./otel-config-cdk.yml /etc/otel-config.yml -ENTRYPOINT ["./otelcontribcol", "--config", "/etc/otel-config.yml"] \ 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..5b112abd0 --- /dev/null +++ b/TrafficCapture/dockerSolution/src/main/docker/prometheus.yaml @@ -0,0 +1,7 @@ +scrape_configs: + - job_name: 'otel-collector' + scrape_interval: 1s + honor_timestamps: true + static_configs: + - targets: [ 'otel-collector:8889' ] + - targets: [ 'otel-collector:8888' ] diff --git a/TrafficCapture/nettyWireLogging/build.gradle b/TrafficCapture/nettyWireLogging/build.gradle index ade3744c9..cf08434fd 100644 --- a/TrafficCapture/nettyWireLogging/build.gradle +++ b/TrafficCapture/nettyWireLogging/build.gradle @@ -8,20 +8,25 @@ plugins { } dependencies { + 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: '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-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/ConditionallyReliableLoggingHttpHandler.java b/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/ConditionallyReliableLoggingHttpHandler.java new file mode 100644 index 000000000..1513ae131 --- /dev/null +++ b/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/ConditionallyReliableLoggingHttpHandler.java @@ -0,0 +1,60 @@ +package org.opensearch.migrations.trafficcapture.netty; + +import io.netty.channel.ChannelHandlerContext; +import io.netty.handler.codec.http.HttpRequest; +import io.netty.util.ReferenceCountUtil; +import lombok.Lombok; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.opensearch.migrations.trafficcapture.IConnectionCaptureFactory; +import org.opensearch.migrations.trafficcapture.netty.tracing.IRootWireLoggingContext; +import org.opensearch.migrations.trafficcapture.netty.tracing.IWireCaptureContexts; + +import java.io.IOException; +import java.util.function.Predicate; + +@Slf4j +public class ConditionallyReliableLoggingHttpHandler extends LoggingHttpHandler { + private final Predicate shouldBlockPredicate; + + public ConditionallyReliableLoggingHttpHandler(@NonNull IRootWireLoggingContext rootContext, + @NonNull String nodeId, String connectionId, + @NonNull IConnectionCaptureFactory trafficOffloaderFactory, + @NonNull RequestCapturePredicate requestCapturePredicate, + @NonNull Predicate headerPredicateForWhenToBlock) + throws IOException { + super(rootContext, nodeId, connectionId, trafficOffloaderFactory, requestCapturePredicate); + this.shouldBlockPredicate = headerPredicateForWhenToBlock; + } + + @Override + protected void channelFinishedReadingAnHttpMessage(ChannelHandlerContext ctx, Object msg, + boolean shouldCapture, HttpRequest httpRequest) + throws Exception { + if (shouldCapture && shouldBlockPredicate.test(httpRequest)) { + ((IWireCaptureContexts.IRequestContext)messageContext).onBlockingRequest(); + messageContext = messageContext.createBlockingContext(); + trafficOffloader.flushCommitAndResetStream(false).whenComplete((result, t) -> { + log.atInfo().setMessage(()->"Done flushing").log(); + + 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("Error offloading the request, but forwarding it to the service anyway").log(); + ReferenceCountUtil.release(msg); + messageContext.addException(t); + } + try { + super.channelFinishedReadingAnHttpMessage(ctx, msg, shouldCapture, httpRequest); + } catch (Exception e) { + throw Lombok.sneakyThrow(e); + } + }); + } else { + assert messageContext instanceof IWireCaptureContexts.IRequestContext; + super.channelFinishedReadingAnHttpMessage(ctx, msg, shouldCapture, httpRequest); + } + } +} 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 deleted file mode 100644 index 441dafbbf..000000000 --- a/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/ConditionallyReliableLoggingHttpRequestHandler.java +++ /dev/null @@ -1,48 +0,0 @@ -package org.opensearch.migrations.trafficcapture.netty; - -import io.netty.channel.ChannelHandlerContext; -import io.netty.handler.codec.http.HttpRequest; -import io.netty.util.ReferenceCountUtil; -import lombok.Lombok; -import lombok.NonNull; -import lombok.extern.slf4j.Slf4j; -import org.opensearch.migrations.trafficcapture.IChannelConnectionCaptureSerializer; - -import java.util.function.Predicate; - -@Slf4j -public class ConditionallyReliableLoggingHttpRequestHandler extends LoggingHttpRequestHandler { - private final Predicate shouldBlockPredicate; - - public ConditionallyReliableLoggingHttpRequestHandler(@NonNull IChannelConnectionCaptureSerializer trafficOffloader, - @NonNull RequestCapturePredicate requestCapturePredicate, - @NonNull Predicate headerPredicateForWhenToBlock) { - super(trafficOffloader, requestCapturePredicate); - this.shouldBlockPredicate = headerPredicateForWhenToBlock; - } - - @Override - protected void channelFinishedReadingAnHttpMessage(ChannelHandlerContext ctx, Object msg, - boolean shouldCapture, HttpRequest httpRequest) - throws Exception { - if (shouldCapture && shouldBlockPredicate.test(httpRequest)) { - trafficOffloader.flushCommitAndResetStream(false).whenComplete((result, t) -> { - 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(); - ReferenceCountUtil.release(msg); - } else { - try { - super.channelFinishedReadingAnHttpMessage(ctx, msg, shouldCapture, httpRequest); - } catch (Exception e) { - throw Lombok.sneakyThrow(e); - } - } - }); - } else { - 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/LoggingHttpHandler.java similarity index 56% rename from TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/LoggingHttpRequestHandler.java rename to TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/LoggingHttpHandler.java index ca3e5075f..8a5f7b5b0 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/LoggingHttpHandler.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,20 +21,53 @@ import lombok.extern.slf4j.Slf4j; import org.opensearch.migrations.coreutils.MetricsAttributeKey; import org.opensearch.migrations.coreutils.MetricsEvent; -import org.opensearch.migrations.trafficcapture.IChannelConnectionCaptureSerializer; import org.opensearch.migrations.coreutils.MetricsLogger; +import org.opensearch.migrations.trafficcapture.IChannelConnectionCaptureSerializer; +import org.opensearch.migrations.trafficcapture.IConnectionCaptureFactory; +import org.opensearch.migrations.trafficcapture.netty.tracing.IRootWireLoggingContext; +import org.opensearch.migrations.trafficcapture.netty.tracing.IWireCaptureContexts; +import java.io.IOException; import java.time.Instant; @Slf4j -public class LoggingHttpRequestHandler extends ChannelInboundHandlerAdapter { +public class LoggingHttpHandler extends ChannelDuplexHandler { private static final MetricsLogger metricsLogger = new MetricsLogger("LoggingHttpRequestHandler"); + static class CaptureIgnoreState { + static final byte CAPTURE = 0; + static final byte IGNORE_REQUEST = 1; + static final byte IGNORE_RESPONSE = 2; + private CaptureIgnoreState() {} + } + + static class CaptureState { + byte captureIgnoreState = CaptureIgnoreState.CAPTURE; + boolean liveReadObservationsInOffloader = false; + + boolean shouldCapture() { + return captureIgnoreState == CaptureIgnoreState.CAPTURE; + } + + public void setShouldCaptureForRequest(boolean b) { + captureIgnoreState = b ? CaptureIgnoreState.CAPTURE : CaptureIgnoreState.IGNORE_REQUEST; + } + + public void advanceStateModelIntoResponseGather() { + if (CaptureIgnoreState.CAPTURE != captureIgnoreState) { + captureIgnoreState = CaptureIgnoreState.IGNORE_RESPONSE; + } + } + } + static class SimpleHttpRequestDecoder extends HttpRequestDecoder { private final PassThruHttpHeaders.HttpHeadersToPreserve headersToPreserve; + private final CaptureState captureState; - public SimpleHttpRequestDecoder(@NonNull PassThruHttpHeaders.HttpHeadersToPreserve headersToPreserve) { + public SimpleHttpRequestDecoder(@NonNull PassThruHttpHeaders.HttpHeadersToPreserve headersToPreserve, + CaptureState captureState) { this.headersToPreserve = headersToPreserve; + this.captureState = captureState; } /** @@ -47,34 +82,40 @@ public HttpMessage createMessage(String[] initialLine) throws Exception { , new PassThruHttpHeaders(headersToPreserve) ); } - } + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + if (captureState.captureIgnoreState == CaptureIgnoreState.IGNORE_RESPONSE) { + captureState.captureIgnoreState = CaptureIgnoreState.CAPTURE; + } + super.channelRead(ctx, msg); + } + } + static class SimpleDecodedHttpRequestHandler extends ChannelInboundHandlerAdapter { @Getter private HttpRequest currentRequest; final RequestCapturePredicate requestCapturePredicate; - boolean isDone; - boolean shouldCapture; - boolean liveReadObservationsInOffloader; - - SimpleDecodedHttpRequestHandler(RequestCapturePredicate requestCapturePredicate) { + boolean haveParsedFullRequest; + final CaptureState captureState; + + SimpleDecodedHttpRequestHandler(RequestCapturePredicate requestCapturePredicate, CaptureState captureState) { this.requestCapturePredicate = requestCapturePredicate; this.currentRequest = null; - this.isDone = false; - this.shouldCapture = true; - liveReadObservationsInOffloader = false; + this.haveParsedFullRequest = false; + this.captureState = captureState; } @Override public void channelRead(@NonNull ChannelHandlerContext ctx, @NonNull Object msg) throws Exception { if (msg instanceof HttpRequest) { currentRequest = (HttpRequest) msg; - shouldCapture = RequestCapturePredicate.CaptureDirective.CAPTURE == - requestCapturePredicate.apply((HttpRequest) msg); + captureState.setShouldCaptureForRequest(RequestCapturePredicate.CaptureDirective.CAPTURE == + requestCapturePredicate.apply((HttpRequest) msg)); } else if (msg instanceof HttpContent) { ((HttpContent)msg).release(); if (msg instanceof LastHttpContent) { - isDone = true; + haveParsedFullRequest = true; } } else { super.channelRead(ctx, msg); @@ -82,11 +123,9 @@ public void channelRead(@NonNull ChannelHandlerContext ctx, @NonNull Object msg) } public HttpRequest resetCurrentRequest() { - this.shouldCapture = true; - this.isDone = false; + this.haveParsedFullRequest = false; var old = currentRequest; this.currentRequest = null; - this.liveReadObservationsInOffloader = false; return old; } } @@ -95,15 +134,27 @@ public HttpRequest resetCurrentRequest() { protected final EmbeddedChannel httpDecoderChannel; - public LoggingHttpRequestHandler(IChannelConnectionCaptureSerializer trafficOffloader, - @NonNull RequestCapturePredicate httpHeadersCapturePredicate) { - this.trafficOffloader = trafficOffloader; + protected IWireCaptureContexts.IHttpMessageContext messageContext; + + public LoggingHttpHandler(@NonNull IRootWireLoggingContext rootContext, String nodeId, String channelKey, + @NonNull IConnectionCaptureFactory trafficOffloaderFactory, + @NonNull RequestCapturePredicate httpHeadersCapturePredicate) + throws IOException { + var parentContext = rootContext.createConnectionContext(channelKey, nodeId); + this.messageContext = parentContext.createInitialRequestContext(); + + this.trafficOffloader = trafficOffloaderFactory.createOffloader(parentContext); + var captureState = new CaptureState(); httpDecoderChannel = new EmbeddedChannel( - new SimpleHttpRequestDecoder(httpHeadersCapturePredicate.getHeadersRequiredForMatcher()), - new SimpleDecodedHttpRequestHandler(httpHeadersCapturePredicate) + new SimpleHttpRequestDecoder(httpHeadersCapturePredicate.getHeadersRequiredForMatcher(), captureState), + new SimpleDecodedHttpRequestHandler(httpHeadersCapturePredicate, captureState) ); } + private IWireCaptureContexts.ICapturingConnectionContext getConnectionContext() { + return messageContext.getLogicalEnclosingScope(); + } + private SimpleDecodedHttpRequestHandler getHandlerThatHoldsParsedHttpRequest() { return (SimpleDecodedHttpRequestHandler) httpDecoderChannel.pipeline().last(); } @@ -111,6 +162,7 @@ private SimpleDecodedHttpRequestHandler getHandlerThatHoldsParsedHttpRequest() { @Override public void channelUnregistered(ChannelHandlerContext ctx) throws Exception { trafficOffloader.addCloseEvent(Instant.now()); + getConnectionContext().onUnregistered(); trafficOffloader.flushCommitAndResetStream(true).whenComplete((result, t) -> { if (t != null) { log.warn("Got error: " + t.getMessage()); @@ -127,6 +179,10 @@ public void channelUnregistered(ChannelHandlerContext ctx) throws Exception { @Override public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { + getConnectionContext().onRemoved(); + messageContext.close(); + messageContext.getLogicalEnclosingScope().close(); + trafficOffloader.flushCommitAndResetStream(true).whenComplete((result, t) -> { if (t != null) { log.warn("Got error: " + t.getMessage()); @@ -142,7 +198,9 @@ public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { protected void channelFinishedReadingAnHttpMessage(ChannelHandlerContext ctx, Object msg, boolean shouldCapture, HttpRequest httpRequest) throws Exception { + messageContext = messageContext.createWaitingForResponseContext(); super.channelRead(ctx, msg); + metricsLogger.atSuccess(MetricsEvent.RECEIVED_FULL_HTTP_REQUEST) .setAttribute(MetricsAttributeKey.CHANNEL_ID, ctx.channel().id().asLongText()) .setAttribute(MetricsAttributeKey.HTTP_METHOD, httpRequest.method().toString()) @@ -151,23 +209,40 @@ protected void channelFinishedReadingAnHttpMessage(ChannelHandlerContext ctx, Ob @Override public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + IWireCaptureContexts.IRequestContext requestContext; + if (!(messageContext instanceof IWireCaptureContexts.IRequestContext)) { + messageContext = requestContext = messageContext.createNextRequestContext(); + } else { + requestContext = (IWireCaptureContexts.IRequestContext) messageContext; + } + var timestamp = Instant.now(); var requestParsingHandler = getHandlerThatHoldsParsedHttpRequest(); var bb = ((ByteBuf) msg); httpDecoderChannel.writeInbound(bb.retainedDuplicate()); // the ByteBuf is consumed/release by this method - var shouldCapture = requestParsingHandler.shouldCapture; + + var captureState = requestParsingHandler.captureState; + var shouldCapture = captureState.shouldCapture(); if (shouldCapture) { - requestParsingHandler.liveReadObservationsInOffloader = true; + captureState.liveReadObservationsInOffloader = true; trafficOffloader.addReadEvent(timestamp, bb); - } else if (requestParsingHandler.liveReadObservationsInOffloader) { + } else if (captureState.liveReadObservationsInOffloader) { + requestContext.onCaptureSuppressed(); trafficOffloader.cancelCaptureForCurrentRequest(timestamp); - requestParsingHandler.liveReadObservationsInOffloader = false; + captureState.liveReadObservationsInOffloader = false; } + metricsLogger.atSuccess(MetricsEvent.RECEIVED_REQUEST_COMPONENT) .setAttribute(MetricsAttributeKey.CHANNEL_ID, ctx.channel().id().asLongText()).emit(); + requestContext.onBytesRead(bb.readableBytes()); + - if (requestParsingHandler.isDone) { + if (requestParsingHandler.haveParsedFullRequest) { + requestContext.onFullyParsedRequest(); var httpRequest = requestParsingHandler.resetCurrentRequest(); + captureState.liveReadObservationsInOffloader = false; + captureState.advanceStateModelIntoResponseGather(); + if (shouldCapture) { var decoderResultLoose = httpRequest.decoderResult(); if (decoderResultLoose instanceof HttpMessageDecoderResult) { @@ -183,9 +258,30 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception } } + @Override + public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) throws Exception { + IWireCaptureContexts.IResponseContext responseContext; + if (!(messageContext instanceof IWireCaptureContexts.IResponseContext)) { + messageContext = responseContext = messageContext.createResponseContext(); + } else { + responseContext = (IWireCaptureContexts.IResponseContext) messageContext; + } + + var bb = (ByteBuf) msg; + if (getHandlerThatHoldsParsedHttpRequest().captureState.shouldCapture()) { + trafficOffloader.addWriteEvent(Instant.now(), bb); + } + metricsLogger.atSuccess(MetricsEvent.RECEIVED_RESPONSE_COMPONENT) + .setAttribute(MetricsAttributeKey.CHANNEL_ID, ctx.channel().id().asLongText()).emit(); + responseContext.onBytesWritten(bb.readableBytes()); + + super.write(ctx, msg, promise); + } + @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { trafficOffloader.addExceptionCaughtEvent(Instant.now(), cause); + messageContext.addException(cause); 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 deleted file mode 100644 index 66d8912a4..000000000 --- a/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/LoggingHttpResponseHandler.java +++ /dev/null @@ -1,77 +0,0 @@ -package org.opensearch.migrations.trafficcapture.netty; - -import io.netty.buffer.ByteBuf; -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.ChannelOutboundHandlerAdapter; -import io.netty.channel.ChannelPromise; -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.trafficcapture.IChannelConnectionCaptureSerializer; - -import java.net.SocketAddress; -import java.time.Instant; - -@Slf4j -public class LoggingHttpResponseHandler extends ChannelOutboundHandlerAdapter { - - private final IChannelConnectionCaptureSerializer trafficOffloader; - private static final MetricsLogger metricsLogger = new MetricsLogger("LoggingHttpResponseHandler"); - - - public LoggingHttpResponseHandler(IChannelConnectionCaptureSerializer trafficOffloader) { - this.trafficOffloader = trafficOffloader; - } - - @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()); - super.disconnect(ctx, promise); - } - - @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()); - 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); - } - -} diff --git a/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/PassThruHttpHeaders.java b/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/PassThruHttpHeaders.java index 33953b55c..a45f85d30 100644 --- a/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/PassThruHttpHeaders.java +++ b/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/PassThruHttpHeaders.java @@ -6,7 +6,6 @@ import lombok.NonNull; import java.util.Arrays; -import java.util.List; import java.util.stream.Stream; public class PassThruHttpHeaders extends DefaultHttpHeaders { diff --git a/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/RequestCapturePredicate.java b/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/RequestCapturePredicate.java index 45d4241fa..cc38aebfd 100644 --- a/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/RequestCapturePredicate.java +++ b/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/RequestCapturePredicate.java @@ -1,6 +1,5 @@ package org.opensearch.migrations.trafficcapture.netty; -import io.netty.handler.codec.http.HttpHeaders; import io.netty.handler.codec.http.HttpRequest; import lombok.Getter; 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..b02272208 --- /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.WireCaptureContexts; +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 + WireCaptureContexts.HttpMessageContext currentRequestContext; + + public RequestContextStateMachine(ConnectionContext incoming) { + connectionContext = incoming; + } +} diff --git a/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/tracing/IRootWireLoggingContext.java b/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/tracing/IRootWireLoggingContext.java new file mode 100644 index 000000000..1b409c200 --- /dev/null +++ b/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/tracing/IRootWireLoggingContext.java @@ -0,0 +1,19 @@ +package org.opensearch.migrations.trafficcapture.netty.tracing; + +import org.opensearch.migrations.trafficcapture.tracing.IRootOffloaderContext; + +public interface IRootWireLoggingContext extends IRootOffloaderContext { + + WireCaptureContexts.ConnectionContext.MetricInstruments getConnectionInstruments(); + + WireCaptureContexts.RequestContext.MetricInstruments getRequestInstruments(); + + WireCaptureContexts.BlockingContext.MetricInstruments getBlockingInstruments(); + + WireCaptureContexts.WaitingForResponseContext.MetricInstruments getWaitingForResponseInstruments(); + + WireCaptureContexts.ResponseContext.MetricInstruments getResponseInstruments(); + + IWireCaptureContexts.ICapturingConnectionContext createConnectionContext(String channelKey, String nodeId); + +} diff --git a/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/tracing/IWireCaptureContexts.java b/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/tracing/IWireCaptureContexts.java new file mode 100644 index 000000000..8ac05baa6 --- /dev/null +++ b/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/tracing/IWireCaptureContexts.java @@ -0,0 +1,93 @@ +package org.opensearch.migrations.trafficcapture.netty.tracing; + +import org.opensearch.migrations.tracing.IWithStartTimeAndAttributes; +import org.opensearch.migrations.tracing.IWithTypedEnclosingScope; +import org.opensearch.migrations.tracing.commoncontexts.IHttpTransactionContext; + +public abstract class IWireCaptureContexts { + + public static class ActivityNames { + private ActivityNames() {} + public static final String BLOCKED = "blocked"; + public static final String GATHERING_REQUEST = "gatheringRequest"; + public static final String WAITING_FOR_RESPONSE = "waitingForResponse"; + public static final String GATHERING_RESPONSE = "gatheringResponse"; + } + + public static class MetricNames { + private MetricNames() {} + public static final String UNREGISTERED = "unregistered"; + public static final String REMOVED = "removed"; + + public static final String BLOCKING_REQUEST = "blockingRequest"; + public static final String CAPTURE_SUPPRESSED = "captureSuppressed"; + public static final String FULL_REQUEST = "fullRequest"; + public static final String BYTES_READ = "bytesRead"; + public static final String BYTES_WRITTEN = "bytesWritten"; + } + + public interface ICapturingConnectionContext + extends org.opensearch.migrations.tracing.commoncontexts.IConnectionContext { + IHttpMessageContext createInitialRequestContext(); + + void onUnregistered(); + + void onRemoved(); + } + + public interface IHttpMessageContext + extends IHttpTransactionContext, + IWithStartTimeAndAttributes, + IWithTypedEnclosingScope + { + IBlockingContext createBlockingContext(); + + IWaitingForResponseContext createWaitingForResponseContext(); + + IResponseContext createResponseContext(); + + IRequestContext createNextRequestContext(); + } + + public interface IRequestContext extends IHttpMessageContext { + String ACTIVITY_NAME = ActivityNames.GATHERING_REQUEST; + + default String getActivityName() { + return ACTIVITY_NAME; + } + + void onBlockingRequest(); + + void onCaptureSuppressed(); + + void onFullyParsedRequest(); + + void onBytesRead(int size); + } + + public interface IBlockingContext extends IHttpMessageContext { + String ACTIVITY_NAME = ActivityNames.BLOCKED; + + default String getActivityName() { + return ACTIVITY_NAME; + } + } + + public interface IWaitingForResponseContext extends IHttpMessageContext { + String ACTIVITY_NAME = ActivityNames.WAITING_FOR_RESPONSE; + + default String getActivityName() { + return ACTIVITY_NAME; + } + } + + public interface IResponseContext extends IHttpMessageContext { + String ACTIVITY_NAME = ActivityNames.GATHERING_RESPONSE; + + default String getActivityName() { + return ACTIVITY_NAME; + } + + void onBytesWritten(int size); + } +} diff --git a/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/tracing/RootWireLoggingContext.java b/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/tracing/RootWireLoggingContext.java new file mode 100644 index 000000000..503861b1f --- /dev/null +++ b/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/tracing/RootWireLoggingContext.java @@ -0,0 +1,35 @@ +package org.opensearch.migrations.trafficcapture.netty.tracing; + +import io.opentelemetry.api.OpenTelemetry; +import lombok.Getter; +import org.opensearch.migrations.tracing.RootOtelContext; + +@Getter +public class RootWireLoggingContext extends RootOtelContext implements IRootWireLoggingContext { + public static final String SCOPE_NAME = "NettyCapture"; + + public final WireCaptureContexts.ConnectionContext.MetricInstruments connectionInstruments; + public final WireCaptureContexts.RequestContext.MetricInstruments requestInstruments; + public final WireCaptureContexts.BlockingContext.MetricInstruments blockingInstruments; + public final WireCaptureContexts.WaitingForResponseContext.MetricInstruments waitingForResponseInstruments; + public final WireCaptureContexts.ResponseContext.MetricInstruments responseInstruments; + + public RootWireLoggingContext(OpenTelemetry openTelemetry) { + this(openTelemetry, SCOPE_NAME); + } + + public RootWireLoggingContext(OpenTelemetry openTelemetry, String scopeName) { + super(scopeName, openTelemetry); + var meter = this.getMeterProvider().get(scopeName); + connectionInstruments = WireCaptureContexts.ConnectionContext.makeMetrics(meter); + requestInstruments = WireCaptureContexts.RequestContext.makeMetrics(meter); + blockingInstruments = WireCaptureContexts.BlockingContext.makeMetrics(meter); + waitingForResponseInstruments = WireCaptureContexts.WaitingForResponseContext.makeMetrics(meter); + responseInstruments = WireCaptureContexts.ResponseContext.makeMetrics(meter); + } + + @Override + public IWireCaptureContexts.ICapturingConnectionContext createConnectionContext(String channelKey, String nodeId) { + return new WireCaptureContexts.ConnectionContext(this, channelKey, nodeId); + } +} diff --git a/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/tracing/WireCaptureContexts.java b/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/tracing/WireCaptureContexts.java new file mode 100644 index 000000000..0e452ba79 --- /dev/null +++ b/TrafficCapture/nettyWireLogging/src/main/java/org/opensearch/migrations/trafficcapture/netty/tracing/WireCaptureContexts.java @@ -0,0 +1,278 @@ +package org.opensearch.migrations.trafficcapture.netty.tracing; + +import io.opentelemetry.api.metrics.LongCounter; +import io.opentelemetry.api.metrics.Meter; +import lombok.Getter; +import lombok.NonNull; +import org.opensearch.migrations.tracing.BaseNestedSpanContext; +import org.opensearch.migrations.tracing.CommonScopedMetricInstruments; +import org.opensearch.migrations.tracing.commoncontexts.IConnectionContext; + +public class WireCaptureContexts extends IWireCaptureContexts { + public static final String COUNT_UNITS = "count"; + public static final String BYTES_UNIT = "bytes"; + + + public static class ConnectionContext + extends org.opensearch.migrations.trafficcapture.tracing.ConnectionContext + implements IWireCaptureContexts.ICapturingConnectionContext + { + public ConnectionContext(IRootWireLoggingContext rootInstrumentationScope, String connectionId, String nodeId) { + super(rootInstrumentationScope, connectionId, nodeId); + } + + @Override + public IRootWireLoggingContext getRootInstrumentationScope() { + return (IRootWireLoggingContext) super.getRootInstrumentationScope(); + } + + public static class MetricInstruments + extends org.opensearch.migrations.trafficcapture.tracing.ConnectionContext.MetricInstruments { + public final LongCounter unregisteredCounter; + public final LongCounter removedCounter; + + public MetricInstruments(Meter meter, String activityMeter) { + super(meter, activityMeter); + unregisteredCounter = meter + .counterBuilder(MetricNames.UNREGISTERED).setUnit(COUNT_UNITS).build(); + removedCounter = meter + .counterBuilder(MetricNames.REMOVED).setUnit(COUNT_UNITS).build(); + } + } + + public static @NonNull MetricInstruments makeMetrics(Meter meter) { + return new MetricInstruments(meter, ACTIVITY_NAME); + } + + @Override + public @NonNull MetricInstruments getMetrics() { + return getRootInstrumentationScope().getConnectionInstruments(); + } + + @Override + public IWireCaptureContexts.IHttpMessageContext createInitialRequestContext() { + return new RequestContext((RootWireLoggingContext) getRootInstrumentationScope(), + this, 0); + } + + @Override + public void onUnregistered() { + meterIncrementEvent(getMetrics().unregisteredCounter); + } + + @Override + public void onRemoved() { + meterIncrementEvent(getMetrics().removedCounter); + } + } + + @Getter + public abstract static class HttpMessageContext extends + BaseNestedSpanContext + implements IWireCaptureContexts.IHttpMessageContext { + + final long sourceRequestIndex; + + protected HttpMessageContext(RootWireLoggingContext rootWireLoggingContext, IConnectionContext enclosingScope, + long sourceRequestIndex) { + super(rootWireLoggingContext, enclosingScope); + this.sourceRequestIndex = sourceRequestIndex; + initializeSpan(); + } + + @Override + public IWireCaptureContexts.ICapturingConnectionContext getLogicalEnclosingScope() { + return (IWireCaptureContexts.ICapturingConnectionContext) getEnclosingScope(); + } + + @Override + public IWireCaptureContexts.IBlockingContext createBlockingContext() { + close(); + return new BlockingContext(getRootInstrumentationScope(), getImmediateEnclosingScope(), sourceRequestIndex); + } + + @Override + public IWireCaptureContexts.IWaitingForResponseContext createWaitingForResponseContext() { + close(); + return new WaitingForResponseContext(getRootInstrumentationScope(), getImmediateEnclosingScope(), + sourceRequestIndex); + } + + @Override + public IWireCaptureContexts.IResponseContext createResponseContext() { + close(); + return new ResponseContext(getRootInstrumentationScope(), getImmediateEnclosingScope(), sourceRequestIndex); + } + + @Override + public IWireCaptureContexts.IRequestContext createNextRequestContext() { + close(); + return new RequestContext(getRootInstrumentationScope(), getImmediateEnclosingScope(), + sourceRequestIndex + 1); + } + } + + public static class RequestContext + extends HttpMessageContext + implements IWireCaptureContexts.IRequestContext { + public RequestContext(RootWireLoggingContext rootWireLoggingContext, + IConnectionContext enclosingScope, + long sourceRequestIndex) { + super(rootWireLoggingContext, enclosingScope, sourceRequestIndex); + } + + @Override + public IWireCaptureContexts.IWaitingForResponseContext createWaitingForResponseContext() { + return new WaitingForResponseContext(getRootInstrumentationScope(), getImmediateEnclosingScope(), + sourceRequestIndex); + } + + public static class MetricInstruments extends CommonScopedMetricInstruments { + public final LongCounter blockingRequestCounter; + public final LongCounter requestsNotOffloadedCounter; + public final LongCounter fullyParsedRequestCounter; + public final LongCounter bytesReadCounter; + + public MetricInstruments(Meter meter, String activityName) { + super(meter, activityName); + blockingRequestCounter = meter + .counterBuilder(MetricNames.BLOCKING_REQUEST).setUnit(COUNT_UNITS).build(); + requestsNotOffloadedCounter = meter + .counterBuilder(MetricNames.CAPTURE_SUPPRESSED).setUnit(COUNT_UNITS).build(); + fullyParsedRequestCounter = meter + .counterBuilder(MetricNames.FULL_REQUEST).setUnit(COUNT_UNITS).build(); + bytesReadCounter = meter + .counterBuilder(MetricNames.BYTES_READ).setUnit(BYTES_UNIT).build(); + } + } + + public static @NonNull MetricInstruments makeMetrics(Meter meter) { + return new MetricInstruments(meter, ACTIVITY_NAME); + } + + @Override + public @NonNull MetricInstruments getMetrics() { + return getRootInstrumentationScope().getRequestInstruments(); + } + + @Override + public void onBlockingRequest() { + meterIncrementEvent(getMetrics().blockingRequestCounter); + } + + @Override + public void onCaptureSuppressed() { + meterIncrementEvent(getMetrics().requestsNotOffloadedCounter); + } + + @Override + public void onFullyParsedRequest() { + meterIncrementEvent(getMetrics().fullyParsedRequestCounter); + } + + @Override + public void onBytesRead(int size) { + meterIncrementEvent(getMetrics().bytesReadCounter, size); + } + } + + public static class BlockingContext + extends HttpMessageContext + implements IWireCaptureContexts.IBlockingContext { + public BlockingContext(RootWireLoggingContext rootWireLoggingContext, + IConnectionContext enclosingScope, + long sourceRequestIndex) { + super(rootWireLoggingContext, enclosingScope, sourceRequestIndex); + } + + @Override + public String getActivityName() { + return ACTIVITY_NAME; + } + + public static class MetricInstruments extends CommonScopedMetricInstruments { + private MetricInstruments(Meter meter, String activityName) { + super(meter, activityName); + } + } + + public static @NonNull MetricInstruments makeMetrics(Meter meter) { + return new MetricInstruments(meter, ACTIVITY_NAME); + } + + @Override + public RequestContext.MetricInstruments getMetrics() { + return getRootInstrumentationScope().requestInstruments; + } + } + + public static class WaitingForResponseContext + extends HttpMessageContext + implements IWireCaptureContexts.IWaitingForResponseContext { + public WaitingForResponseContext(RootWireLoggingContext rootWireLoggingContext, + IConnectionContext enclosingScope, + long sourceRequestIndex) { + super(rootWireLoggingContext, enclosingScope, sourceRequestIndex); + } + + @Override + public String getActivityName() { + return ACTIVITY_NAME; + } + + public static class MetricInstruments extends CommonScopedMetricInstruments { + private MetricInstruments(Meter meter, String activityName) { + super(meter, activityName); + } + } + + public static @NonNull MetricInstruments makeMetrics(Meter meter) { + return new MetricInstruments(meter, ACTIVITY_NAME); + } + + @Override + public RequestContext.MetricInstruments getMetrics() { + return getRootInstrumentationScope().requestInstruments; + } + } + + public static class ResponseContext + extends HttpMessageContext + implements IWireCaptureContexts.IResponseContext { + public ResponseContext(RootWireLoggingContext rootWireLoggingContext, + IConnectionContext enclosingScope, + long sourceRequestIndex) { + super(rootWireLoggingContext, enclosingScope, sourceRequestIndex); + } + + @Override + public String getActivityName() { + return ACTIVITY_NAME; + } + + public static class MetricInstruments extends CommonScopedMetricInstruments { + + private final LongCounter bytesWritten; + + private MetricInstruments(Meter meter, String activityName) { + super(meter, activityName); + bytesWritten = meter + .counterBuilder(MetricNames.BYTES_WRITTEN).setUnit(BYTES_UNIT).build(); + } + } + + public static @NonNull MetricInstruments makeMetrics(Meter meter) { + return new MetricInstruments(meter, ACTIVITY_NAME); + } + + @Override + public @NonNull MetricInstruments getMetrics() { + return getRootInstrumentationScope().getResponseInstruments(); + } + + @Override + public void onBytesWritten(int size) { + meterIncrementEvent(getMetrics().bytesWritten, size); + } + } +} 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/ConditionallyReliableLoggingHttpHandlerTest.java similarity index 76% rename from TrafficCapture/nettyWireLogging/src/test/java/org/opensearch/migrations/trafficcapture/netty/ConditionallyReliableLoggingHttpRequestHandlerTest.java rename to TrafficCapture/nettyWireLogging/src/test/java/org/opensearch/migrations/trafficcapture/netty/ConditionallyReliableLoggingHttpHandlerTest.java index 292b3fa1b..f4fe7b893 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/ConditionallyReliableLoggingHttpHandlerTest.java @@ -4,10 +4,11 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import io.netty.channel.embedded.EmbeddedChannel; -import lombok.AllArgsConstructor; +import io.opentelemetry.sdk.testing.junit5.OpenTelemetryExtension; 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,16 +19,18 @@ import org.opensearch.migrations.trafficcapture.CodedOutputStreamHolder; import org.opensearch.migrations.trafficcapture.OrderedStreamLifecyleManager; import org.opensearch.migrations.trafficcapture.StreamChannelConnectionCaptureSerializer; +import org.opensearch.migrations.trafficcapture.netty.tracing.RootWireLoggingContext; +import org.opensearch.migrations.trafficcapture.protos.TrafficObservation; import org.opensearch.migrations.trafficcapture.protos.TrafficStream; import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.SequenceInputStream; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicBoolean; @@ -35,23 +38,29 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import java.util.stream.Collectors; +import java.util.stream.Stream; @Slf4j -public class ConditionallyReliableLoggingHttpRequestHandlerTest { +public class ConditionallyReliableLoggingHttpHandlerTest { + @RegisterExtension + static final OpenTelemetryExtension otelTesting = OpenTelemetryExtension.create(); + + private static class TestRootContext extends RootWireLoggingContext { + public TestRootContext() { + super(otelTesting.getOpenTelemetry()); + } + } 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,20 +77,21 @@ 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()); } } - private static void writeMessageAndVerify(byte[] fullTrafficBytes, Consumer channelWriter) throws IOException { + var rootInstrumenter = new TestRootContext(); var streamManager = new TestStreamManager(); var offloader = new StreamChannelConnectionCaptureSerializer("Test", "c", streamManager); EmbeddedChannel channel = new EmbeddedChannel( - new ConditionallyReliableLoggingHttpRequestHandler(offloader, new RequestCapturePredicate(), x->true)); // true: block every request + new ConditionallyReliableLoggingHttpHandler(rootInstrumenter, + "n", "c", ctx -> offloader, + new RequestCapturePredicate(), x->true)); // true: block every request channelWriter.accept(channel); // we wrote the correct data to the downstream handler/channel @@ -99,11 +109,14 @@ private static void writeMessageAndVerify(byte[] fullTrafficBytes, Consumerto.hasRead()) + .filter(TrafficObservation::hasRead) .map(to->new ByteArrayInputStream(to.getRead().getData().toByteArray())) .collect(Collectors.toList()))); Assertions.assertArrayEquals(fullTrafficBytes, combinedTrafficPacketsSteam.readAllBytes()); Assertions.assertEquals(1, streamManager.flushCount.get()); + + Assertions.assertTrue(!otelTesting.getSpans().isEmpty()); + Assertions.assertTrue(!otelTesting.getMetrics().isEmpty()); } private static byte[] consumeIntoArray(ByteBuf m) { @@ -145,12 +158,14 @@ private static Consumer getSingleByteAtATimeWriter(boolean useP @Test @ValueSource(booleans = {false, true}) public void testThatSuppressedCaptureWorks() throws Exception { + var rootInstrumenter = new TestRootContext(); var streamMgr = new TestStreamManager(); var offloader = new StreamChannelConnectionCaptureSerializer("Test", "connection", streamMgr); var headerCapturePredicate = new HeaderValueFilteringCapturePredicate(Map.of("user-Agent", "uploader")); EmbeddedChannel channel = new EmbeddedChannel( - new ConditionallyReliableLoggingHttpRequestHandler(offloader, headerCapturePredicate, x->true)); + new ConditionallyReliableLoggingHttpHandler(rootInstrumenter,"n", "c", + ctx -> 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); @@ -168,23 +183,29 @@ public void testThatSuppressedCaptureWorks() throws Exception { @ParameterizedTest @ValueSource(booleans = {false, true}) public void testThatHealthCheckCaptureCanBeSuppressed(boolean singleBytes) throws Exception { + var rootInstrumenter = new TestRootContext(); var streamMgr = new TestStreamManager(); var offloader = new StreamChannelConnectionCaptureSerializer("Test", "connection", streamMgr); var headerCapturePredicate = new HeaderValueFilteringCapturePredicate(Map.of("user-Agent", ".*uploader.*")); EmbeddedChannel channel = new EmbeddedChannel( - new ConditionallyReliableLoggingHttpRequestHandler(offloader, headerCapturePredicate, x->true)); + new ConditionallyReliableLoggingHttpHandler(rootInstrumenter,"n", "c", + ctx -> offloader, headerCapturePredicate, x->false)); getWriter(singleBytes, true, SimpleRequests.HEALTH_CHECK.getBytes(StandardCharsets.UTF_8)).accept(channel); + channel.writeOutbound(Unpooled.wrappedBuffer("response1".getBytes(StandardCharsets.UTF_8))); getWriter(singleBytes, true, SimpleRequests.SMALL_POST.getBytes(StandardCharsets.UTF_8)).accept(channel); + var bytesForResponsePreserved = "response2".getBytes(StandardCharsets.UTF_8); + channel.writeOutbound(Unpooled.wrappedBuffer(bytesForResponsePreserved)); + channel.close(); var requestBytes = (SimpleRequests.HEALTH_CHECK + SimpleRequests.SMALL_POST).getBytes(StandardCharsets.UTF_8); // we wrote the correct data to the downstream handler/channel - var outputData = new SequenceInputStream(Collections.enumeration(channel.inboundMessages().stream() + var consumedData = new SequenceInputStream(Collections.enumeration(channel.inboundMessages().stream() .map(m->new ByteArrayInputStream(consumeIntoArray((ByteBuf)m))) .collect(Collectors.toList()))) .readAllBytes(); - log.info("outputdata = " + new String(outputData, StandardCharsets.UTF_8)); - Assertions.assertArrayEquals(requestBytes, outputData); + log.info("captureddata = " + new String(consumedData, StandardCharsets.UTF_8)); + Assertions.assertArrayEquals(requestBytes, consumedData); Assertions.assertNotNull(streamMgr.byteBufferAtomicReference, "This would be null if the handler didn't block until the output was written"); @@ -194,34 +215,41 @@ public void testThatHealthCheckCaptureCanBeSuppressed(boolean singleBytes) throw trafficStream.getSubStream(0).hasRead()); Assertions.assertEquals(1, streamMgr.flushCount.get()); var observations = trafficStream.getSubStreamList(); - if (singleBytes) { - var sawRequestDropped = new AtomicBoolean(false); - var observationsAfterDrop = observations.stream().dropWhile(o->{ - var wasDrop = o.hasRequestDropped(); - sawRequestDropped.compareAndSet(false, wasDrop); - return !sawRequestDropped.get() || wasDrop; - }).collect(Collectors.toList()); + { + var readObservationStreamToUse = singleBytes ? skipReadsBeforeDrop(observations) : observations.stream(); var combinedTrafficPacketsSteam = - new SequenceInputStream(Collections.enumeration(observationsAfterDrop.stream() - .filter(to->to.hasRead()) - .map(to->new ByteArrayInputStream(to.getRead().getData().toByteArray())) + new SequenceInputStream(Collections.enumeration(readObservationStreamToUse + .filter(to -> to.hasRead()) + .map(to -> new ByteArrayInputStream(to.getRead().getData().toByteArray())) .collect(Collectors.toList()))); + var reconstitutedTrafficStreamReads = combinedTrafficPacketsSteam.readAllBytes(); Assertions.assertArrayEquals(SimpleRequests.SMALL_POST.getBytes(StandardCharsets.UTF_8), - combinedTrafficPacketsSteam.readAllBytes()); - } else { + reconstitutedTrafficStreamReads); + } + + // check that we only got one response + { var combinedTrafficPacketsSteam = new SequenceInputStream(Collections.enumeration(observations.stream() - .filter(to->to.hasRead()) - .map(to->new ByteArrayInputStream(to.getRead().getData().toByteArray())) + .filter(to->to.hasWrite()) + .map(to->new ByteArrayInputStream(to.getWrite().getData().toByteArray())) .collect(Collectors.toList()))); - var reconstitutedTrafficStreamReads = combinedTrafficPacketsSteam.readAllBytes(); - log.info("reconstitutedTrafficStreamReads="+ - new String(reconstitutedTrafficStreamReads, StandardCharsets.UTF_8)); - Assertions.assertArrayEquals(SimpleRequests.SMALL_POST.getBytes(StandardCharsets.UTF_8), - reconstitutedTrafficStreamReads); + var reconstitutedTrafficStreamWrites = combinedTrafficPacketsSteam.readAllBytes(); + log.info("reconstitutedTrafficStreamWrites="+ + new String(reconstitutedTrafficStreamWrites, StandardCharsets.UTF_8)); + Assertions.assertArrayEquals(bytesForResponsePreserved, reconstitutedTrafficStreamWrites); } } + private static Stream skipReadsBeforeDrop(List observations) { + var sawRequestDropped = new AtomicBoolean(false); + return observations.stream().dropWhile(o->{ + var wasDrop = o.hasRequestDropped(); + sawRequestDropped.compareAndSet(false, wasDrop); + return !sawRequestDropped.get() || wasDrop; + }); + } + private Consumer getWriter(boolean singleBytes, boolean usePool, byte[] bytes) { if (singleBytes) { return getSingleByteAtATimeWriter(usePool, bytes); diff --git a/TrafficCapture/replayerPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/build.gradle b/TrafficCapture/replayerPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/build.gradle index a974d3aea..814c9e2b8 100644 --- a/TrafficCapture/replayerPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/build.gradle +++ b/TrafficCapture/replayerPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/build.gradle @@ -13,7 +13,9 @@ dependencies { implementation project(':replayerPlugins:jsonMessageTransformers:jsonJoltMessageTransformer') testImplementation project(':replayerPlugins:jsonMessageTransformers:jsonJoltMessageTransformer') + testImplementation project(':coreUtilities') testImplementation project(':trafficReplayer') + testImplementation testFixtures(project(path: ':coreUtilities')) testImplementation testFixtures(project(path: ':testUtilities')) testImplementation testFixtures(project(path: ':trafficReplayer')) 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..4eb408162 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 @@ -6,6 +6,7 @@ import org.opensearch.migrations.replay.datahandlers.http.HttpJsonTransformingConsumer; import org.opensearch.migrations.replay.datatypes.HttpRequestTransformationStatus; import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; +import org.opensearch.migrations.tracing.InstrumentationTest; import org.opensearch.migrations.transform.JsonJoltTransformBuilder; import org.opensearch.migrations.transform.JsonJoltTransformer; @@ -21,7 +22,7 @@ import java.util.zip.GZIPInputStream; @Slf4j -public class AddCompressionEncodingTest { +public class AddCompressionEncodingTest extends InstrumentationTest { public static final byte BYTE_FILL_VALUE = (byte) '7'; @@ -33,30 +34,31 @@ 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, + rootContext.getTestConnectionRequestContext(0)); final var payloadPartSize = 511; final var numParts = 1025; String sourceHeaders = "GET / HTTP/1.1\n" + "host: localhost\n" + - "content-length: " + (numParts*payloadPartSize) + "\n"; + "content-length: " + (numParts * payloadPartSize) + "\n"; - DiagnosticTrackableCompletableFuture tail = + DiagnosticTrackableCompletableFuture tail = compressingTransformer.consumeBytes(sourceHeaders.getBytes(StandardCharsets.UTF_8)) - .thenCompose(v-> compressingTransformer.consumeBytes("\n".getBytes(StandardCharsets.UTF_8)), - ()->"AddCompressionEncodingTest.compressingTransformer"); + .thenCompose(v -> compressingTransformer.consumeBytes("\n".getBytes(StandardCharsets.UTF_8)), + () -> "AddCompressionEncodingTest.compressingTransformer"); final byte[] payloadPart = new byte[payloadPartSize]; Arrays.fill(payloadPart, BYTE_FILL_VALUE); - for (var i = new AtomicInteger(numParts); i.get()>0; i.decrementAndGet()) { - tail = tail.thenCompose(v->compressingTransformer.consumeBytes(payloadPart), - ()->"AddCompressionEncodingTest.consumeBytes:"+i.get()); + for (var i = new AtomicInteger(numParts); i.get() > 0; i.decrementAndGet()) { + tail = tail.thenCompose(v -> compressingTransformer.consumeBytes(payloadPart), + () -> "AddCompressionEncodingTest.consumeBytes:" + i.get()); } var fullyProcessedResponse = - tail.thenCompose(v->compressingTransformer.finalizeRequest(), - ()->"AddCompressionEncodingTest.fullyProcessedResponse"); + tail.thenCompose(v -> compressingTransformer.finalizeRequest(), + () -> "AddCompressionEncodingTest.fullyProcessedResponse"); fullyProcessedResponse.get(); + try (var bais = new ByteArrayInputStream(testPacketCapture.getBytesCaptured()); var unzipStream = new GZIPInputStream(bais); var isr = new InputStreamReader(unzipStream, StandardCharsets.UTF_8); diff --git a/TrafficCapture/replayerPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/java/org/opensearch/migrations/replay/PayloadRepackingTest.java b/TrafficCapture/replayerPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/java/org/opensearch/migrations/replay/PayloadRepackingTest.java index 8ba273ce1..b6be07217 100644 --- a/TrafficCapture/replayerPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/java/org/opensearch/migrations/replay/PayloadRepackingTest.java +++ b/TrafficCapture/replayerPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/java/org/opensearch/migrations/replay/PayloadRepackingTest.java @@ -9,6 +9,8 @@ import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; import org.opensearch.migrations.testutils.WrapWithNettyLeakDetection; +import org.opensearch.migrations.tracing.InstrumentationTest; +import org.opensearch.migrations.tracing.TestContext; import org.opensearch.migrations.transform.JsonJoltTransformBuilder; import org.opensearch.migrations.transform.JsonJoltTransformer; @@ -22,7 +24,7 @@ @Slf4j @WrapWithNettyLeakDetection(repetitions = 1) -public class PayloadRepackingTest { +public class PayloadRepackingTest extends InstrumentationTest { public static Stream> expandList(Stream> stream, List possibilities) { return stream.flatMap(list-> possibilities.stream().map(innerB -> { @@ -45,8 +47,12 @@ public static Arguments[] makeCombinations() { public void testSimplePayloadTransform(boolean doGzip, boolean doChunked) throws Exception { var transformerBuilder = JsonJoltTransformer.newBuilder(); - if (doGzip) { transformerBuilder.addCannedOperation(JsonJoltTransformBuilder.CANNED_OPERATION.ADD_GZIP); } - if (doChunked) { transformerBuilder.addCannedOperation(JsonJoltTransformBuilder.CANNED_OPERATION.MAKE_CHUNKED); } + if (doGzip) { + transformerBuilder.addCannedOperation(JsonJoltTransformBuilder.CANNED_OPERATION.ADD_GZIP); + } + if (doChunked) { + transformerBuilder.addCannedOperation(JsonJoltTransformBuilder.CANNED_OPERATION.MAKE_CHUNKED); + } Random r = new Random(2); var stringParts = IntStream.range(0, 1) @@ -59,8 +65,8 @@ public void testSimplePayloadTransform(boolean doGzip, boolean doChunked) throws expectedRequestHeaders.add("host", "localhost"); expectedRequestHeaders.add("Content-Length", "46"); - TestUtils.runPipelineAndValidate(transformerBuilder.build(), null,null, - stringParts, expectedRequestHeaders, + TestUtils.runPipelineAndValidate(rootContext, transformerBuilder.build(), null, + null, stringParts, expectedRequestHeaders, referenceStringBuilder -> TestUtils.resolveReferenceString(referenceStringBuilder)); } @@ -91,7 +97,8 @@ public void testJsonPayloadTransformation() throws Exception { ObjectMapper mapper = new ObjectMapper(); var simpleTransform = mapper.readValue(simplePayloadTransform, - new TypeReference>(){}); + new TypeReference>() { + }); transformerBuilder.addCannedOperation(JsonJoltTransformBuilder.CANNED_OPERATION.PASS_THRU); transformerBuilder.addOperationObject(simpleTransform); @@ -104,7 +111,7 @@ public void testJsonPayloadTransformation() throws Exception { expectedRequestHeaders.add("content-type", "application/json; charset=UTF-8"); expectedRequestHeaders.add("Content-Length", "55"); - TestUtils.runPipelineAndValidate(transformerBuilder.build(), null, + TestUtils.runPipelineAndValidate(rootContext, transformerBuilder.build(), null, extraHeaders, List.of(jsonPayload), expectedRequestHeaders, x -> "{\"top\":[{\"Name\":\"A\",\"Value\":1},{\"Name\":\"B\",\"Value\":2}]}"); } diff --git a/TrafficCapture/trafficCaptureProxyServer/build.gradle b/TrafficCapture/trafficCaptureProxyServer/build.gradle index f9f14abe4..74fd3dfe8 100644 --- a/TrafficCapture/trafficCaptureProxyServer/build.gradle +++ b/TrafficCapture/trafficCaptureProxyServer/build.gradle @@ -13,8 +13,11 @@ configurations { } dependencies { - implementation 'org.opensearch.plugin:opensearch-security:2.6.0.0' - opensearchSecurityPlugin 'org.opensearch.plugin:opensearch-security:2.6.0.0' + implementation 'org.opensearch.plugin:opensearch-security:2.11.1.0' + implementation 'org.opensearch:opensearch-common:2.11.0' + implementation 'org.opensearch:opensearch-core:2.11.0' + implementation 'org.opensearch:opensearch:2.11.0' + opensearchSecurityPlugin 'org.opensearch.plugin:opensearch-security:2.11.1.0' implementation files(zipTree("$configurations.opensearchSecurityPlugin.singleFile").matching { include "*.jar" exclude "slf*.jar" @@ -26,10 +29,6 @@ dependencies { implementation project(':coreUtilities') implementation group: 'io.netty', name: 'netty-all', version: '4.1.100.Final' - implementation 'org.opensearch:opensearch-common:2.6.0' - implementation 'org.opensearch:opensearch-core:2.6.0' - implementation 'org.opensearch:opensearch:2.8.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' implementation group: 'org.apache.logging.log4j', name: 'log4j-slf4j2-impl', version: '2.20.0' @@ -41,6 +40,7 @@ dependencies { testImplementation project(':captureProtobufs') testImplementation testFixtures(project(path: ':testUtilities')) testImplementation testFixtures(project(path: ':captureOffloader')) + testImplementation testFixtures(project(path: ':coreUtilities')) } 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..a705845ff 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.RootOtelContext; 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,7 +186,7 @@ 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 -> new StreamChannelConnectionCaptureSerializer<>(null, ctx.getConnectionId(), new StreamLifecycleManager<>() { @Override public CodedOutputStreamHolder createStream() { @@ -235,13 +234,15 @@ static Properties buildKafkaProperties(Parameters params) throws IOException { return kafkaProps; } - private static IConnectionCaptureFactory getConnectionCaptureFactory(Parameters params) throws IOException { + private static IConnectionCaptureFactory + getConnectionCaptureFactory(Parameters params, RootCaptureContext rootContext) throws IOException { var nodeId = getNodeId(); // Resist the urge for now though until it comes in as a request/need. if (params.traceDirectory != null) { return new FileConnectionCaptureFactory(nodeId, params.traceDirectory, params.maximumTrafficStreamSize); } else if (params.kafkaConnection != null) { - return new KafkaCaptureFactory(nodeId, new KafkaProducer<>(buildKafkaProperties(params)), params.maximumTrafficStreamSize); + return new KafkaCaptureFactory(rootContext, + nodeId, new KafkaProducer<>(buildKafkaProperties(params)), params.maximumTrafficStreamSize); } else if (params.noCapture) { return getNullConnectionCaptureFactory(); } else { @@ -301,9 +302,8 @@ public static void main(String[] args) throws InterruptedException, IOException var params = parseArgs(args); var backsideUri = convertStringToUri(params.backsideUriString); - if (params.otelCollectorEndpoint != null) { - initializeOpenTelemetry("capture-proxy", params.otelCollectorEndpoint); - } + var rootContext = new RootCaptureContext( + RootOtelContext.initializeOpenTelemetryWithCollectorOrAsNoop(params.otelCollectorEndpoint, "capture")); var sksOp = Optional.ofNullable(params.sslConfigFilePath) .map(sslConfigFile->new DefaultSecurityKeyStore(getSettings(sslConfigFile), @@ -326,8 +326,8 @@ public static void main(String[] args) throws InterruptedException, IOException }).orElse(null); var headerCapturePredicate = new HeaderValueFilteringCapturePredicate(convertPairListToMap(params.suppressCaptureHeaderPairs)); - proxy.start(backsideConnectionPool, params.numThreads, sslEngineSupplier, - getConnectionCaptureFactory(params), headerCapturePredicate); + proxy.start(rootContext, backsideConnectionPool, params.numThreads, sslEngineSupplier, + getConnectionCaptureFactory(params, rootContext), headerCapturePredicate); } catch (Exception e) { log.atError().setCause(e).setMessage("Caught exception while setting up the server and rethrowing").log(); throw e; diff --git a/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/RootCaptureContext.java b/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/RootCaptureContext.java new file mode 100644 index 000000000..af2be2a0c --- /dev/null +++ b/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/RootCaptureContext.java @@ -0,0 +1,25 @@ +package org.opensearch.migrations.trafficcapture.proxyserver; + +import io.opentelemetry.api.OpenTelemetry; +import lombok.Getter; +import org.opensearch.migrations.trafficcapture.kafkaoffloader.tracing.IRootKafkaOffloaderContext; +import org.opensearch.migrations.trafficcapture.kafkaoffloader.tracing.KafkaRecordContext; +import org.opensearch.migrations.trafficcapture.netty.tracing.RootWireLoggingContext; +import org.opensearch.migrations.trafficcapture.netty.tracing.WireCaptureContexts; + +public class RootCaptureContext extends RootWireLoggingContext implements IRootKafkaOffloaderContext { + + public static final String SCOPE_NAME = "captureProxy"; + @Getter + public final KafkaRecordContext.MetricInstruments kafkaOffloadingInstruments; + + public RootCaptureContext(OpenTelemetry openTelemetry) { + this(openTelemetry, SCOPE_NAME); + } + + public RootCaptureContext(OpenTelemetry openTelemetry, String scopeName) { + super(openTelemetry, scopeName); + var meter = this.getMeterProvider().get(scopeName); + kafkaOffloadingInstruments = KafkaRecordContext.makeMetrics(meter); + } +} diff --git a/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/BacksideConnectionPool.java b/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/BacksideConnectionPool.java index 95c1ad115..b1bf526cb 100644 --- a/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/BacksideConnectionPool.java +++ b/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/BacksideConnectionPool.java @@ -1,8 +1,6 @@ package org.opensearch.migrations.trafficcapture.proxyserver.netty; import io.netty.channel.socket.nio.NioSocketChannel; -import io.netty.handler.logging.LogLevel; -import org.opensearch.migrations.coreutils.MetricsLogger; import org.slf4j.event.Level; import io.netty.bootstrap.Bootstrap; import io.netty.channel.ChannelDuplexHandler; diff --git a/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/ExpiringSubstitutableItemPool.java b/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/ExpiringSubstitutableItemPool.java index 34dd2a5f8..46a20b517 100644 --- a/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/ExpiringSubstitutableItemPool.java +++ b/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/ExpiringSubstitutableItemPool.java @@ -303,7 +303,12 @@ private void beginLoadingNewItemIfNecessary() { } @Override + @SneakyThrows public String toString() { + return eventLoop.submit(() -> toStringOnThread()).get(); + } + + private String toStringOnThread() { final StringBuilder sb = new StringBuilder("ExpiringSubstitutableItemPool{"); sb.append("poolSize=").append(poolSize); if (eventLoop.inEventLoop()) { 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/NettyScanningHttpProxy.java b/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/NettyScanningHttpProxy.java index e6ed7e28e..869f84e3d 100644 --- a/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/NettyScanningHttpProxy.java +++ b/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/NettyScanningHttpProxy.java @@ -10,6 +10,7 @@ import lombok.NonNull; import org.opensearch.migrations.trafficcapture.IConnectionCaptureFactory; import org.opensearch.migrations.trafficcapture.netty.RequestCapturePredicate; +import org.opensearch.migrations.trafficcapture.netty.tracing.IRootWireLoggingContext; import javax.net.ssl.SSLEngine; import java.util.function.Supplier; @@ -28,7 +29,8 @@ public int getProxyPort() { return proxyPort; } - public void start(BacksideConnectionPool backsideConnectionPool, + public void start(IRootWireLoggingContext rootContext, + BacksideConnectionPool backsideConnectionPool, int numThreads, Supplier sslEngineSupplier, IConnectionCaptureFactory connectionCaptureFactory, @@ -39,7 +41,7 @@ public void start(BacksideConnectionPool backsideConnectionPool, try { mainChannel = serverBootstrap.group(bossGroup, workerGroup) .channel(NioServerSocketChannel.class) - .childHandler(new ProxyChannelInitializer<>(backsideConnectionPool, sslEngineSupplier, + .childHandler(new ProxyChannelInitializer<>(rootContext, backsideConnectionPool, sslEngineSupplier, connectionCaptureFactory, requestCapturePredicate)) .childOption(ChannelOption.AUTO_READ, false) .bind(proxyPort).sync().channel(); 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..bb9f0ed29 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 @@ -7,24 +7,27 @@ import io.netty.handler.ssl.SslHandler; import lombok.NonNull; import org.opensearch.migrations.trafficcapture.IConnectionCaptureFactory; -import org.opensearch.migrations.trafficcapture.netty.ConditionallyReliableLoggingHttpRequestHandler; +import org.opensearch.migrations.trafficcapture.netty.ConditionallyReliableLoggingHttpHandler; import org.opensearch.migrations.trafficcapture.netty.RequestCapturePredicate; -import org.opensearch.migrations.trafficcapture.netty.LoggingHttpResponseHandler; +import org.opensearch.migrations.trafficcapture.netty.tracing.IRootWireLoggingContext; import javax.net.ssl.SSLEngine; import java.io.IOException; import java.util.function.Supplier; public class ProxyChannelInitializer extends ChannelInitializer { - private final IConnectionCaptureFactory connectionCaptureFactory; private final Supplier sslEngineProvider; + private final IRootWireLoggingContext rootContext; private final BacksideConnectionPool backsideConnectionPool; private final RequestCapturePredicate requestCapturePredicate; - public ProxyChannelInitializer(BacksideConnectionPool backsideConnectionPool, Supplier sslEngineSupplier, + public ProxyChannelInitializer(IRootWireLoggingContext rootContext, + BacksideConnectionPool backsideConnectionPool, + Supplier sslEngineSupplier, IConnectionCaptureFactory connectionCaptureFactory, @NonNull RequestCapturePredicate requestCapturePredicate) { + this.rootContext = rootContext; this.backsideConnectionPool = backsideConnectionPool; this.sslEngineProvider = sslEngineSupplier; this.connectionCaptureFactory = connectionCaptureFactory; @@ -46,10 +49,10 @@ 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 ConditionallyReliableLoggingHttpHandler<>(rootContext, + "", connectionId, 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/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/NettyScanningHttpProxyTest.java b/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/NettyScanningHttpProxyTest.java index c405fafe6..158ed0251 100644 --- a/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/NettyScanningHttpProxyTest.java +++ b/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/NettyScanningHttpProxyTest.java @@ -10,9 +10,12 @@ import org.opensearch.migrations.testutils.SimpleHttpClientForTesting; import org.opensearch.migrations.testutils.SimpleHttpResponse; import org.opensearch.migrations.testutils.SimpleHttpServer; +import org.opensearch.migrations.tracing.InMemoryInstrumentationBundle; +import org.opensearch.migrations.tracing.RootOtelContext; import org.opensearch.migrations.trafficcapture.IConnectionCaptureFactory; import org.opensearch.migrations.trafficcapture.InMemoryConnectionCaptureFactory; import org.opensearch.migrations.trafficcapture.netty.RequestCapturePredicate; +import org.opensearch.migrations.trafficcapture.netty.tracing.RootWireLoggingContext; import org.opensearch.migrations.trafficcapture.protos.TrafficStream; import java.io.ByteArrayOutputStream; @@ -87,7 +90,9 @@ public void testRoundTrip() throws CountDownLatch interactionsCapturedCountdown = new CountDownLatch(NUM_EXPECTED_TRAFFIC_STREAMS); var captureFactory = new InMemoryConnectionCaptureFactory(TEST_NODE_ID_STRING, 1024*1024, () -> interactionsCapturedCountdown.countDown()); - var servers = startServers(captureFactory); + var inMemoryInstrumentationBundle = new InMemoryInstrumentationBundle(true, true); + var rootCtx = new RootWireLoggingContext(inMemoryInstrumentationBundle.openTelemetrySdk); + var servers = startServers(rootCtx, captureFactory); try (var client = new SimpleHttpClientForTesting()) { var nettyEndpoint = URI.create("http://localhost:" + servers.v1().getProxyPort() + "/"); @@ -170,7 +175,7 @@ private static String makeTestRequestViaClient(SimpleHttpClientForTesting client } private static Tuple - startServers(IConnectionCaptureFactory connectionCaptureFactory) throws + startServers(RootWireLoggingContext rootCtx, IConnectionCaptureFactory connectionCaptureFactory) throws PortFinder.ExceededMaxPortAssigmentAttemptException { var nshp = new AtomicReference(); @@ -198,8 +203,9 @@ private static String makeTestRequestViaClient(SimpleHttpClientForTesting client try { var connectionPool = new BacksideConnectionPool(testServerUri, null, 10, Duration.ofSeconds(10)); - nshp.get().start(connectionPool, 1, null, connectionCaptureFactory, - new RequestCapturePredicate()); + + nshp.get().start(rootCtx, connectionPool, 1, null, + connectionCaptureFactory, new RequestCapturePredicate()); System.out.println("proxy port = " + port); } catch (InterruptedException e) { Thread.currentThread().interrupt(); diff --git a/TrafficCapture/trafficReplayer/build.gradle b/TrafficCapture/trafficReplayer/build.gradle index 7ca67b9d0..368eb1901 100644 --- a/TrafficCapture/trafficReplayer/build.gradle +++ b/TrafficCapture/trafficReplayer/build.gradle @@ -63,6 +63,8 @@ dependencies { implementation 'org.apache.commons:commons-compress:1.24.0' testFixturesImplementation project(':replayerPlugins:jsonMessageTransformers:jsonMessageTransformerInterface') + testFixturesImplementation project(':coreUtilities') + testFixturesImplementation testFixtures(project(path: ':coreUtilities')) testFixturesImplementation testFixtures(project(path: ':testUtilities')) testFixturesImplementation group: 'org.slf4j', name: 'slf4j-api', version: '2.0.7' @@ -70,10 +72,10 @@ dependencies { 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' - testImplementation project(':captureOffloader') testImplementation testFixtures(project(path: ':captureOffloader')) testImplementation testFixtures(project(path: ':testUtilities')) + testImplementation testFixtures(project(path: ':coreUtilities')) testImplementation project(':replayerPlugins:jsonMessageTransformers:jsonJMESPathMessageTransformerProvider') testImplementation project(':replayerPlugins:jsonMessageTransformers:jsonJoltMessageTransformerProvider') testImplementation project(':replayerPlugins:jsonMessageTransformers:openSearch23PlusTargetTransformerProvider') 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..2da589b91 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,7 @@ package org.opensearch.migrations.replay; import lombok.NonNull; -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.trafficcapture.protos.TrafficStream; import java.time.Instant; @@ -22,7 +20,7 @@ enum State { ACCUMULATING_WRITES } - public final ISourceTrafficChannelKey trafficChannelKey; + public final ITrafficStreamKey trafficChannelKey; private RequestResponsePacketPair rrPair; AtomicLong newestPacketTimestampInMillis; State state; @@ -48,17 +46,15 @@ public Accumulation(@NonNull ITrafficStreamKey trafficChannelKey, dropObservationsLeftoverFromPrevious ? State.IGNORING_LAST_REQUEST : State.WAITING_FOR_NEXT_READ_CHUNK; } - public RequestResponsePacketPair getOrCreateTransactionPair(ITrafficStreamKey forTrafficStreamKey) { + public RequestResponsePacketPair getOrCreateTransactionPair(ITrafficStreamKey forTrafficStreamKey, + Instant originTimestamp) { if (rrPair != null) { return rrPair; } - rrPair = new RequestResponsePacketPair(forTrafficStreamKey); - return rrPair; - } - - public UniqueReplayerRequestKey getRequestKey() { - return new UniqueReplayerRequestKey(getRrPair().getBeginningTrafficStreamKey(), + this.rrPair = new RequestResponsePacketPair(forTrafficStreamKey, originTimestamp, startingSourceRequestIndex, getIndexOfCurrentRequest()); + //this.rrPair.getRequestContext() + return rrPair; } public boolean hasSignaledRequests() { 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..6db6e9bdc 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 @@ -1,20 +1,28 @@ package org.opensearch.migrations.replay; import lombok.NonNull; -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.IReplayContexts; 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 onRequestReceived(@NonNull IReplayContexts.IReplayerHttpTransactionContext ctx, + @NonNull HttpMessageAndTimestamp request); + + void onFullDataReceived(@NonNull IReplayContexts.IReplayerHttpTransactionContext ctx, + @NonNull RequestResponsePacketPair rrpp); + void onTrafficStreamsExpired(RequestResponsePacketPair.ReconstructionStatus status, + @NonNull IReplayContexts.IChannelKeyContext ctx, @NonNull List trafficStreamKeysBeingHeld); - void onConnectionClose(@NonNull ISourceTrafficChannelKey key, int channelInteractionNumber, - RequestResponsePacketPair.ReconstructionStatus status, @NonNull Instant when, + + void onConnectionClose(int channelInteractionNumber, + @NonNull IReplayContexts.IChannelKeyContext ctx, + RequestResponsePacketPair.ReconstructionStatus status, + @NonNull Instant when, @NonNull List trafficStreamKeysBeingHeld); - void onTrafficStreamIgnored(@NonNull ITrafficStreamKey tsk); + + void onTrafficStreamIgnored(@NonNull IReplayContexts.ITrafficStreamsLifecycleContext 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..c5dc6505b 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 @@ -1,11 +1,14 @@ package org.opensearch.migrations.replay; +import lombok.AllArgsConstructor; import lombok.NonNull; 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.replay.datatypes.ITrafficStreamKey; +import org.opensearch.migrations.replay.datatypes.UniqueReplayerRequestKey; +import org.opensearch.migrations.replay.tracing.IReplayContexts; import org.opensearch.migrations.replay.traffic.expiration.BehavioralPolicy; import org.opensearch.migrations.replay.traffic.expiration.ExpiringTrafficStreamMap; import org.opensearch.migrations.replay.traffic.source.ITrafficStreamWithKey; @@ -52,7 +55,7 @@ public class CapturedTrafficToHttpTransactionAccumulator { public static final Duration EXPIRATION_GRANULARITY = Duration.ofSeconds(1); private final ExpiringTrafficStreamMap liveStreams; - private final AccumulationCallbacks listener; + private final SpanWrappingAccumulationCallbacks listener; private final AtomicInteger requestCounter = new AtomicInteger(); private final AtomicInteger reusedKeepAliveCounter = new AtomicInteger(); @@ -93,9 +96,45 @@ public void onExpireAccumulation(String partitionId, Accumulation accumulation) RequestResponsePacketPair.ReconstructionStatus.EXPIRED_PREMATURELY); } }); - this.listener = accumulationCallbacks; + this.listener = new SpanWrappingAccumulationCallbacks(accumulationCallbacks); } + @AllArgsConstructor + private static class SpanWrappingAccumulationCallbacks { + private final AccumulationCallbacks underlying; + public void onRequestReceived(IReplayContexts.IRequestAccumulationContext requestCtx, + @NonNull HttpMessageAndTimestamp request) { + requestCtx.close(); + underlying.onRequestReceived(requestCtx.getLogicalEnclosingScope(), request); + } + + public void onFullDataReceived(@NonNull UniqueReplayerRequestKey key, + @NonNull RequestResponsePacketPair rrpp) { + rrpp.getResponseContext().close(); + underlying.onFullDataReceived(rrpp.getHttpTransactionContext(), rrpp); + } + + public void onConnectionClose(@NonNull Accumulation accum, + RequestResponsePacketPair.ReconstructionStatus status, + @NonNull Instant when, + @NonNull List trafficStreamKeysBeingHeld) { + var tsCtx = accum.trafficChannelKey.getTrafficStreamsContext(); + underlying.onConnectionClose(accum.numberOfResets.get(), tsCtx.getLogicalEnclosingScope(), + status, when, trafficStreamKeysBeingHeld); + } + + public void onTrafficStreamsExpired(RequestResponsePacketPair.ReconstructionStatus status, + IReplayContexts.ITrafficStreamsLifecycleContext tsCtx, + @NonNull List trafficStreamKeysBeingHeld) { + underlying.onTrafficStreamsExpired(status, tsCtx.getLogicalEnclosingScope(), trafficStreamKeysBeingHeld); + } + + public void onTrafficStreamIgnored(@NonNull ITrafficStreamKey tsk) { + var tsCtx = tsk.getTrafficStreamsContext(); + underlying.onTrafficStreamIgnored(tsk.getTrafficStreamsContext()); + } + }; + public int numberOfConnectionsCreated() { return liveStreams.numberOfConnectionsCreated(); } public int numberOfRequestsOnReusedConnections() { return reusedKeepAliveCounter.get(); } public int numberOfConnectionsClosed() { return closedConnectionCounter.get(); } @@ -123,17 +162,17 @@ private static String summarizeTrafficStream(TrafficStream ts) { public void accept(ITrafficStreamWithKey trafficStreamAndKey) { var yetToBeSequencedTrafficStream = trafficStreamAndKey.getStream(); - log.atTrace().setMessage(()->"Got trafficStream: "+summarizeTrafficStream(yetToBeSequencedTrafficStream)).log(); + log.atTrace().setMessage(() -> "Got trafficStream: " + summarizeTrafficStream(yetToBeSequencedTrafficStream)).log(); var partitionId = yetToBeSequencedTrafficStream.getNodeId(); var connectionId = yetToBeSequencedTrafficStream.getConnectionId(); var tsk = trafficStreamAndKey.getKey(); - var accum = liveStreams.getOrCreateWithoutExpiration(tsk, k->createInitialAccumulation(trafficStreamAndKey)); + var accum = liveStreams.getOrCreateWithoutExpiration(tsk, k -> createInitialAccumulation(trafficStreamAndKey)); var trafficStream = trafficStreamAndKey.getStream(); - for (int i=0; i"Connection terminated: removing " + partitionId + ":" + connectionId + + log.atInfo().setMessage(() -> "Connection terminated: removing " + partitionId + ":" + connectionId + " from liveStreams map").log(); liveStreams.remove(partitionId, connectionId); break; @@ -219,15 +258,20 @@ private static List getTrafficStreamsHeldByAccum(Accumulation TrafficObservation observation, @NonNull ITrafficStreamKey trafficStreamKey, Instant timestamp) { + var originTimestamp = TrafficStreamUtils.instantFromProtoTimestamp(observation.getTs()); if (observation.hasClose()) { - accum.getOrCreateTransactionPair(trafficStreamKey).holdTrafficStream(trafficStreamKey); - rotateAccumulationIfNecessary(trafficStreamKey.getConnectionId(), accum); + accum.getOrCreateTransactionPair(trafficStreamKey, originTimestamp) + .holdTrafficStream(trafficStreamKey); + var heldTrafficStreams = getTrafficStreamsHeldByAccum(accum); + if (rotateAccumulationIfNecessary(trafficStreamKey.getConnectionId(), accum)) { + heldTrafficStreams = List.of(); + } closedConnectionCounter.incrementAndGet(); - listener.onConnectionClose(accum.trafficChannelKey, accum.getIndexOfCurrentRequest(), - RequestResponsePacketPair.ReconstructionStatus.COMPLETE, timestamp, getTrafficStreamsHeldByAccum(accum)); + listener.onConnectionClose(accum, RequestResponsePacketPair.ReconstructionStatus.COMPLETE, + timestamp, heldTrafficStreams); return Optional.of(CONNECTION_STATUS.CLOSED); } else if (observation.hasConnectionException()) { - accum.getOrCreateTransactionPair(trafficStreamKey).holdTrafficStream(trafficStreamKey); + accum.getOrCreateTransactionPair(trafficStreamKey, originTimestamp).holdTrafficStream(trafficStreamKey); rotateAccumulationIfNecessary(trafficStreamKey.getConnectionId(), accum); exceptionConnectionCounter.incrementAndGet(); accum.resetForNextRequest(); @@ -248,11 +292,12 @@ private Optional handleObservationForReadState(@NonNull Accum } var connectionId = trafficStreamKey.getConnectionId(); + var originTimestamp = TrafficStreamUtils.instantFromProtoTimestamp(observation.getTs()); if (observation.hasRead()) { if (!accum.hasRrPair()) { requestCounter.incrementAndGet(); } - var rrPair = accum.getOrCreateTransactionPair(trafficStreamKey); + var rrPair = accum.getOrCreateTransactionPair(trafficStreamKey, originTimestamp); log.atTrace().setMessage(() -> "Adding request data for accum[" + connectionId + "]=" + accum).log(); rrPair.addRequestData(timestamp, observation.getRead().getData().toByteArray()); log.atTrace().setMessage(() -> "Added request data for accum[" + connectionId + "]=" + accum).log(); @@ -261,11 +306,12 @@ private Optional handleObservationForReadState(@NonNull Accum handleEndOfRequest(accum); } else if (observation.hasReadSegment()) { log.atTrace().setMessage(()->"Adding request segment for accum[" + connectionId + "]=" + accum).log(); - var rrPair = accum.getOrCreateTransactionPair(trafficStreamKey); + var rrPair = accum.getOrCreateTransactionPair(trafficStreamKey, originTimestamp); if (rrPair.requestData == null) { rrPair.requestData = new HttpMessageAndTimestamp.Request(timestamp); requestCounter.incrementAndGet(); } + rrPair.addRequestData(timestamp, observation.getRead().getData().toByteArray()); rrPair.requestData.addSegment(observation.getReadSegment().getData().toByteArray()); log.atTrace().setMessage(()->"Added request segment for accum[" + connectionId + "]=" + accum).log(); } else if (observation.hasSegmentEnd()) { @@ -319,7 +365,8 @@ 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(tsk->listener.onTrafficStreamIgnored(tsk)); } log.atTrace().setMessage(()->"resetting to forget "+ accum.trafficChannelKey).log(); accum.resetToIgnoreAndForgetCurrentRequest(); @@ -340,7 +387,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 +401,32 @@ 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 httpMessage = 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); + .setAttribute(MetricsAttributeKey.REQUEST_ID, + rrPair.getRequestContext().getLogicalEnclosingScope().getReplayerRequestKey().toString()) + .setAttribute(MetricsAttributeKey.CONNECTION_ID, + rrPair.getRequestContext().getLogicalEnclosingScope().getLogicalEnclosingScope().getConnectionId()).emit(); + assert (httpMessage != null); + assert (!httpMessage.hasInProgressSegment()); + var requestCtx = rrPair.getRequestContext(); + rrPair.rotateRequestGatheringToResponse(); + listener.onRequestReceived(requestCtx, httpMessage); 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(); + var requestKey = rrPair.getHttpTransactionContext().getReplayerRequestKey(); + metricsLogger.atSuccess(MetricsEvent.ACCUMULATED_FULL_CAPTURED_SOURCE_RESPONSE) + .setAttribute(MetricsAttributeKey.REQUEST_ID, requestKey.toString()) + .setAttribute(MetricsAttributeKey.CONNECTION_ID, + requestKey.getTrafficStreamKey().getConnectionId()).emit(); rrPair.completionStatus = status; - listener.onFullDataReceived(accumulation.getRequestKey(), rrPair); + listener.onFullDataReceived(requestKey, rrPair); log.atTrace().setMessage("resetting for end of response").log(); accumulation.resetForNextRequest(); } @@ -381,7 +434,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(); } @@ -404,6 +458,7 @@ private void fireAccumulationsCallbacksAndClose(Accumulation accumulation, "reproducing this to the target cluster.").log(); if (accumulation.hasRrPair()) { listener.onTrafficStreamsExpired(status, + accumulation.trafficChannelKey.getTrafficStreamsContext(), Collections.unmodifiableList(accumulation.getRrPair().trafficStreamKeysBeingHeld)); } return; @@ -418,8 +473,8 @@ private void fireAccumulationsCallbacksAndClose(Accumulation accumulation, } } finally { if (accumulation.hasSignaledRequests()) { - listener.onConnectionClose(accumulation.trafficChannelKey, accumulation.getIndexOfCurrentRequest(), - status, accumulation.getLastTimestamp(), getTrafficStreamsHeldByAccum(accumulation)); + listener.onConnectionClose(accumulation, 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..2dddd1f2a 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,30 +11,40 @@ 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.replay.datahandlers.NettyPacketToHttpConsumer; import org.opensearch.migrations.replay.datatypes.ConnectionReplaySession; -import org.opensearch.migrations.replay.datatypes.ISourceTrafficChannelKey; +import org.opensearch.migrations.replay.tracing.IReplayContexts; +import org.opensearch.migrations.replay.tracing.ReplayContexts; import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; import org.opensearch.migrations.replay.util.StringTrackableCompletableFuture; import java.net.URI; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; @Slf4j public class ClientConnectionPool { - public static final String TARGET_CONNECTION_POOL_NAME = "targetConnectionPool"; private final URI serverUri; private final SslContext sslContext; public final NioEventLoopGroup eventLoopGroup; private final LoadingCache connectionId2ChannelCache; - private final AtomicInteger numConnectionsCreated = new AtomicInteger(0); - private final AtomicInteger numConnectionsClosed = new AtomicInteger(0); + public ConnectionReplaySession buildConnectionReplaySession(final IReplayContexts.IChannelKeyContext channelKeyCtx) { + if (eventLoopGroup.isShuttingDown()) { + throw new IllegalStateException("Event loop group is shutting down. Not creating a new session."); + } + log.trace("creating connection session"); + // arguably the most only thing that matters here is associating this item with an + // EventLoop (thread). As the channel needs to be recycled, we'll come back to the + // event loop that was tied to the original channel to bind all future channels to + // the same event loop. That means that we don't have to worry about concurrent + // accesses/changes to the OTHER value that we're storing within the cache. + return new ConnectionReplaySession(eventLoopGroup.next(), channelKeyCtx); + } public ClientConnectionPool(URI serverUri, SslContext sslContext, int numThreads) { this.serverUri = serverUri; @@ -42,36 +52,23 @@ public ClientConnectionPool(URI serverUri, SslContext sslContext, int numThreads this.eventLoopGroup = new NioEventLoopGroup(numThreads, new DefaultThreadFactory(TARGET_CONNECTION_POOL_NAME)); - connectionId2ChannelCache = CacheBuilder.newBuilder().build(new CacheLoader<>() { - @Override - public ConnectionReplaySession load(final String s) { - if (eventLoopGroup.isShuttingDown()) { - throw new IllegalStateException("Event loop group is shutting down. Not creating a new session."); - } - numConnectionsCreated.incrementAndGet(); - log.trace("creating connection session"); - // arguably the most only thing that matters here is associating this item with an - // EventLoop (thread). As the channel needs to be recycled, we'll come back to the - // event loop that was tied to the original channel to bind all future channels to - // the same event loop. That means that we don't have to worry about concurrent - // accesses/changes to the OTHER value that we're storing within the cache. - return new ConnectionReplaySession(eventLoopGroup.next()); - } - }); + connectionId2ChannelCache = CacheBuilder.newBuilder().build(CacheLoader.from(key -> { + throw new UnsupportedOperationException("Use Cache.get(key, callable) instead"); + })); } private DiagnosticTrackableCompletableFuture - getResilientClientChannelProducer(EventLoop eventLoop, String diagnosticLabel) { + getResilientClientChannelProducer(EventLoop eventLoop, IReplayContexts.IChannelKeyContext 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 { @@ -82,14 +79,6 @@ public ConnectionReplaySession load(final String s) { }); } - public int getNumConnectionsCreated() { - return numConnectionsCreated.get(); - } - - public int getNumConnectionsClosed() { - return numConnectionsClosed.get(); - } - public Future shutdownNow() { connectionId2ChannelCache.invalidateAll(); return eventLoopGroup.shutdownGracefully(); @@ -125,7 +114,8 @@ public DiagnosticTrackableCompletableFuture closeConnectionsAndShu () -> "Final shutdown for " + this.getClass().getSimpleName()); } - public void closeConnection(String connId) { + public void closeConnection(IReplayContexts.IChannelKeyContext ctx) { + var connId = ctx.getConnectionId(); log.atInfo().setMessage(() -> "closing connection for " + connId).log(); var channelsFuture = connectionId2ChannelCache.getIfPresent(connId); if (channelsFuture != null) { @@ -135,9 +125,8 @@ public void closeConnection(String connId) { } public Future - submitEventualSessionGet(ISourceTrafficChannelKey channelKey, boolean ignoreIfNotPresent) { - ConnectionReplaySession channelFutureAndSchedule = - getCachedSession(channelKey, ignoreIfNotPresent); + submitEventualSessionGet(IReplayContexts.IChannelKeyContext ctx, boolean ignoreIfNotPresent) { + ConnectionReplaySession channelFutureAndSchedule = getCachedSession(ctx, ignoreIfNotPresent); if (channelFutureAndSchedule == null) { var rval = new DefaultPromise(eventLoopGroup.next()); rval.setSuccess(null); @@ -146,20 +135,21 @@ 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) { - var crs = dontCreate ? connectionId2ChannelCache.getIfPresent(channelKey.getConnectionId()) : - connectionId2ChannelCache.get(channelKey.getConnectionId()); - if (crs != null) { - crs.setChannelId(channelKey); - } + public ConnectionReplaySession getCachedSession(IReplayContexts.IChannelKeyContext channelKeyCtx, + boolean dontCreate) { + + var crs = dontCreate ? connectionId2ChannelCache.getIfPresent(channelKeyCtx.getConnectionId()) : + connectionId2ChannelCache.get(channelKeyCtx.getConnectionId(), + () -> buildConnectionReplaySession(channelKeyCtx)); + log.atTrace().setMessage(()->"returning ReplaySession=" + crs + " for " + channelKeyCtx.getConnectionId() + + " from " + channelKeyCtx).log(); return crs; } @@ -169,11 +159,11 @@ public ConnectionReplaySession getCachedSession(ISourceTrafficChannelKey channel new StringTrackableCompletableFuture(new CompletableFuture<>(), ()->"Waiting for closeFuture() on channel"); - numConnectionsClosed.incrementAndGet(); channelAndFutureWork.getChannelFutureFuture().map(cff->cff .thenAccept(cf-> { cf.channel().close() .addListener(closeFuture -> { + channelAndFutureWork.getSocketContext().close(); if (closeFuture.isSuccess()) { channelClosedFuture.future.complete(channelAndFutureWork.getInnerChannelFuture().channel()); } else { @@ -183,7 +173,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.getSocketContext() + "). " + 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..424780143 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 @@ -1,8 +1,8 @@ package org.opensearch.migrations.replay; import org.opensearch.migrations.replay.datahandlers.IPacketFinalizingConsumer; -import org.opensearch.migrations.replay.datatypes.UniqueReplayerRequestKey; +import org.opensearch.migrations.replay.tracing.IReplayContexts; public interface PacketConsumerFactory { - IPacketFinalizingConsumer create(UniqueReplayerRequestKey requestKey); + IPacketFinalizingConsumer create(IReplayContexts.IReplayerHttpTransactionContext 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..83c399791 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 @@ -6,13 +6,14 @@ import org.opensearch.migrations.replay.datahandlers.http.HttpJsonTransformingConsumer; 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.IReplayContexts; 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 +26,9 @@ public PacketToTransformingHttpHandlerFactory(IJsonTransformer jsonTransformer, @Override public IPacketFinalizingConsumer> - create(UniqueReplayerRequestKey requestKey) { + create(IReplayContexts.IReplayerHttpTransactionContext httpTransactionContext) { log.trace("creating HttpJsonTransformingConsumer"); return new HttpJsonTransformingConsumer<>(jsonTransformer, authTransformerFactory, - new TransformedPacketReceiver(), requestKey.toString(), requestKey); + new TransformedPacketReceiver(), httpTransactionContext); } } 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..983f98611 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 @@ -5,12 +5,8 @@ import io.netty.handler.codec.http.HttpHeaders; import lombok.NonNull; import lombok.extern.slf4j.Slf4j; -import org.opensearch.migrations.coreutils.MetricsAttributeKey; -import org.opensearch.migrations.coreutils.MetricsEvent; -import org.opensearch.migrations.coreutils.MetricsLogBuilder; -import org.opensearch.migrations.coreutils.MetricsLogger; -import org.opensearch.migrations.replay.datatypes.UniqueReplayerRequestKey; -import org.opensearch.migrations.replay.datatypes.UniqueSourceRequestKey; +import org.opensearch.migrations.replay.datatypes.TransformedPackets; +import org.opensearch.migrations.replay.tracing.IReplayContexts; import java.time.Duration; import java.util.Base64; @@ -40,14 +36,16 @@ public class ParsedHttpMessagesAsDicts { public final Optional> sourceResponseOp; public final Optional> targetRequestOp; public final Optional> targetResponseOp; + public final IReplayContexts.ITupleHandlingContext context; - public ParsedHttpMessagesAsDicts(SourceTargetCaptureTuple tuple) { + public ParsedHttpMessagesAsDicts(@NonNull SourceTargetCaptureTuple tuple) { this(tuple, Optional.ofNullable(tuple.sourcePair)); } - protected ParsedHttpMessagesAsDicts(SourceTargetCaptureTuple tuple, + protected ParsedHttpMessagesAsDicts(@NonNull SourceTargetCaptureTuple tuple, Optional sourcePairOp) { - this(getSourceRequestOp(tuple.uniqueRequestKey, sourcePairOp), + this(tuple.context, + getSourceRequestOp(tuple.context, sourcePairOp), getSourceResponseOp(tuple, sourcePairOp), getTargetRequestOp(tuple), getTargetResponseOp(tuple)); @@ -56,74 +54,51 @@ protected ParsedHttpMessagesAsDicts(SourceTargetCaptureTuple tuple, private static Optional> getTargetResponseOp(SourceTargetCaptureTuple tuple) { return Optional.ofNullable(tuple.targetResponseData) .filter(r -> !r.isEmpty()) - .map(d -> convertResponse(tuple.uniqueRequestKey, d, tuple.targetResponseDuration)); + .map(d -> convertResponse(tuple.context, d, tuple.targetResponseDuration)); } private static Optional> getTargetRequestOp(SourceTargetCaptureTuple tuple) { return Optional.ofNullable(tuple.targetRequestData) - .map(d -> d.asByteArrayStream()) - .map(d -> convertRequest(tuple.uniqueRequestKey, d.collect(Collectors.toList()))); + .map(TransformedPackets::asByteArrayStream) + .map(d -> convertRequest(tuple.context, d.collect(Collectors.toList()))); } private static Optional> getSourceResponseOp(SourceTargetCaptureTuple tuple, Optional sourcePairOp) { return sourcePairOp.flatMap(p -> Optional.ofNullable(p.responseData).flatMap(d -> Optional.ofNullable(d.packetBytes)) - .map(d -> convertResponse(tuple.uniqueRequestKey, d, + .map(d -> convertResponse(tuple.context, d, // TODO: These durations are not measuring the same values! Duration.between(tuple.sourcePair.requestData.getLastPacketTimestamp(), tuple.sourcePair.responseData.getLastPacketTimestamp())))); } - private static Optional> getSourceRequestOp(@NonNull UniqueSourceRequestKey diagnosticKey, - Optional sourcePairOp) { + private static Optional> + getSourceRequestOp(@NonNull IReplayContexts.ITupleHandlingContext context, + Optional sourcePairOp) { return sourcePairOp.flatMap(p -> Optional.ofNullable(p.requestData).flatMap(d -> Optional.ofNullable(d.packetBytes)) - .map(d -> convertRequest(diagnosticKey, d))); + .map(d -> convertRequest(context, d))); } - public ParsedHttpMessagesAsDicts(Optional> sourceRequestOp1, + public ParsedHttpMessagesAsDicts(IReplayContexts.ITupleHandlingContext context, + Optional> sourceRequestOp1, Optional> sourceResponseOp2, Optional> targetRequestOp3, Optional> targetResponseOp4) { + this.context = context; this.sourceRequestOp = sourceRequestOp1; this.sourceResponseOp = sourceResponseOp2; this.targetRequestOp = targetRequestOp3; this.targetResponseOp = targetResponseOp4; + fillStatusCodeMetrics(context, sourceResponseOp, targetResponseOp); } - 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); - } - - public MetricsLogBuilder buildStatusCodeMetrics(MetricsLogBuilder logBuilder, UniqueSourceRequestKey requestKey) { - return buildStatusCodeMetrics(logBuilder, requestKey, sourceResponseOp, targetResponseOp); - } - - public static MetricsLogBuilder buildStatusCodeMetrics(MetricsLogBuilder builder, - UniqueSourceRequestKey requestKey, - Optional> sourceResponseOp, - Optional> targetResponseOp) { - var sourceStatus = sourceResponseOp.map(r -> r.get(STATUS_CODE_KEY)); - 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, - 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, - sourceResponseOp.map(r -> r.get("Method"))); - builder = addMetricIfPresent(builder, MetricsAttributeKey.HTTP_ENDPOINT, - sourceResponseOp.map(r -> r.get("Request-URI"))); - return builder; + public static void fillStatusCodeMetrics(@NonNull IReplayContexts.ITupleHandlingContext context, + Optional> sourceResponseOp, + Optional> targetResponseOp) { + sourceResponseOp.ifPresent(r -> context.setSourceStatus((Integer) r.get(STATUS_CODE_KEY))); + targetResponseOp.ifPresent(r -> context.setTargetStatus((Integer) r.get(STATUS_CODE_KEY))); } @@ -150,34 +125,37 @@ private static Map fillMap(LinkedHashMap map, return map; } - private static Map makeSafeMap(@NonNull UniqueSourceRequestKey diagnosticKey, + private static Map makeSafeMap(@NonNull IReplayContexts.ITupleHandlingContext context, Callable> c) { try { return c.call(); } catch (Exception e) { // TODO - this isn't a good design choice. // We should follow through with the spirit of this class and leave this as empty optional values - log.atWarn().setMessage(()->"Putting what may be a bogus value in the output because transforming it " + - "into json threw an exception for "+diagnosticKey.toString()).setCause(e).log(); + log.atWarn().setMessage(() -> "Putting what may be a bogus value in the output because transforming it " + + "into json threw an exception for " + context).setCause(e).log(); return Map.of("Exception", (Object) e.toString()); } } - private static Map convertRequest(@NonNull UniqueSourceRequestKey diagnosticKey, + private static Map convertRequest(@NonNull IReplayContexts.ITupleHandlingContext context, @NonNull List data) { - return makeSafeMap(diagnosticKey, () -> { + return makeSafeMap(context, () -> { var map = new LinkedHashMap(); var message = HttpByteBufFormatter.parseHttpRequestFromBufs(byteToByteBufStream(data), true); map.put("Request-URI", message.uri()); map.put("Method", message.method().toString()); map.put("HTTP-Version", message.protocolVersion().toString()); + context.setMethod(message.method().toString()); + context.setEndpoint(message.uri()); + context.setHttpVersion(message.protocolVersion().toString()); return fillMap(map, message.headers(), message.content()); }); } - private static Map convertResponse(@NonNull UniqueSourceRequestKey diagnosticKey, + private static Map convertResponse(@NonNull IReplayContexts.ITupleHandlingContext context, @NonNull List data, Duration latency) { - return makeSafeMap(diagnosticKey, () -> { + return makeSafeMap(context, () -> { var map = new LinkedHashMap(); var message = HttpByteBufFormatter.parseHttpResponseFromBufs(byteToByteBufStream(data), true); map.put("HTTP-Version", message.protocolVersion()); 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..9e69ad7c8 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,15 +1,13 @@ 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; import org.opensearch.migrations.coreutils.MetricsEvent; import org.opensearch.migrations.coreutils.MetricsLogger; -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.IReplayContexts; import org.opensearch.migrations.replay.traffic.source.BufferedFlowController; import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; @@ -123,56 +121,53 @@ private static void logStartOfWork(Object stringableKey, long newCount, Instant } public DiagnosticTrackableCompletableFuture - scheduleTransformationWork(UniqueReplayerRequestKey requestKey, Instant originalStart, + scheduleTransformationWork(IReplayContexts.IReplayerHttpTransactionContext 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, 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(IReplayContexts.IReplayerHttpTransactionContext ctx, + Instant originalStart, Instant originalEnd, int numPackets, Stream packets) { var newCount = totalCountOfScheduledTasksOutstanding.incrementAndGet(); final String label = "request"; var start = timeShifter.transformSourceTimeToRealTime(originalStart); var end = timeShifter.transformSourceTimeToRealTime(originalEnd); var interval = numPackets > 1 ? Duration.between(start, end).dividedBy(numPackets-1L) : Duration.ZERO; + var requestKey = ctx.getReplayerRequestKey(); logStartOfWork(requestKey, newCount, start, label); metricsLogger.atSuccess(MetricsEvent.SCHEDULED_REQUEST_TO_BE_SENT) .setAttribute(MetricsAttributeKey.REQUEST_ID, requestKey.toString()) .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 DiagnosticTrackableCompletableFuture + closeConnection(int channelInteractionNum, + IReplayContexts.IChannelKeyContext ctx, Instant timestamp) { var newCount = totalCountOfScheduledTasksOutstanding.incrementAndGet(); final String label = "close"; var atTime = timeShifter.transformSourceTimeToRealTime(timestamp); + var channelKey = ctx.getChannelKey(); logStartOfWork(new IndexedChannelInteraction(channelKey, channelInteractionNum), newCount, atTime, label); - var future = networkSendOrchestrator.scheduleClose(channelKey, channelInteractionNum, atTime); - hookWorkFinishingUpdates(future, timestamp, channelKey, label); + var future = networkSendOrchestrator.scheduleClose(ctx, channelInteractionNum, atTime); + return hookWorkFinishingUpdates(future, timestamp, channelKey, label); } public DiagnosticTrackableCompletableFuture closeConnectionsAndShutdown() { return networkSendOrchestrator.clientConnectionPool.closeConnectionsAndShutdown(); } - public int getNumConnectionsCreated() { - return networkSendOrchestrator.clientConnectionPool.getNumConnectionsCreated(); - } - - public int getNumConnectionsClosed() { - return networkSendOrchestrator.clientConnectionPool.getNumConnectionsClosed(); - } - public void setFirstTimestamp(Instant firstPacketTimestamp) { timeShifter.setFirstTimestamp(firstPacketTimestamp); } 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..6c18b0dfd 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 @@ -3,7 +3,12 @@ import com.google.common.base.Objects; import lombok.NonNull; import lombok.extern.slf4j.Slf4j; +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.IReplayContexts; +import org.opensearch.migrations.tracing.IScopedInstrumentationAttributes; +import org.opensearch.migrations.tracing.IWithTypedEnclosingScope; import java.nio.charset.StandardCharsets; import java.time.Instant; @@ -22,18 +27,56 @@ public enum ReconstructionStatus { HttpMessageAndTimestamp requestData; HttpMessageAndTimestamp responseData; - @NonNull final ITrafficStreamKey firstTrafficStreamKeyForRequest; + @NonNull + final ISourceTrafficChannelKey firstTrafficStreamKeyForRequest; List trafficStreamKeysBeingHeld; ReconstructionStatus completionStatus; + // switch between RequestAccumulation/ResponseAccumulation objects when we're parsing, + // or just leave this null, in which case, the context from the trafficStreamKey should be used + private IScopedInstrumentationAttributes requestOrResponseAccumulationContext; - public RequestResponsePacketPair(@NonNull ITrafficStreamKey startingAtTrafficStreamKey) { - firstTrafficStreamKeyForRequest = startingAtTrafficStreamKey; + public RequestResponsePacketPair(@NonNull ITrafficStreamKey startingAtTrafficStreamKey, Instant sourceTimestamp, + int startingSourceRequestIndex, int indexOfCurrentRequest) { + this.firstTrafficStreamKeyForRequest = startingAtTrafficStreamKey; + var requestKey = new UniqueReplayerRequestKey(startingAtTrafficStreamKey, + startingSourceRequestIndex, indexOfCurrentRequest); + var httpTransactionContext = startingAtTrafficStreamKey.getTrafficStreamsContext() + .createHttpTransactionContext(requestKey, sourceTimestamp); + requestOrResponseAccumulationContext = httpTransactionContext.createRequestAccumulationContext(); } - @NonNull ITrafficStreamKey getBeginningTrafficStreamKey() { + @NonNull ISourceTrafficChannelKey getBeginningTrafficStreamKey() { return firstTrafficStreamKeyForRequest; } + public IReplayContexts.IReplayerHttpTransactionContext getHttpTransactionContext() { + var looseCtx = requestOrResponseAccumulationContext; + // the req/response ctx types in the assert below will always implement this with the + // IReplayerHttpTransactionContext parameter, but this seems clearer + // than trying to engineer a compile time static check + assert looseCtx instanceof IWithTypedEnclosingScope; + assert looseCtx instanceof IReplayContexts.IRequestAccumulationContext + || looseCtx instanceof IReplayContexts.IResponseAccumulationContext; + return ((IWithTypedEnclosingScope) looseCtx) + .getLogicalEnclosingScope(); + + } + + public @NonNull IReplayContexts.IRequestAccumulationContext getRequestContext() { + return (IReplayContexts.IRequestAccumulationContext) requestOrResponseAccumulationContext; + } + + public @NonNull IReplayContexts.IResponseAccumulationContext getResponseContext() { + return (IReplayContexts.IResponseAccumulationContext) requestOrResponseAccumulationContext; + } + + public void rotateRequestGatheringToResponse() { + var looseCtx = requestOrResponseAccumulationContext; + assert looseCtx instanceof IReplayContexts.IRequestAccumulationContext; + requestOrResponseAccumulationContext = + getRequestContext().getLogicalEnclosingScope().createResponseAccumulationContext(); + } + public void addRequestData(Instant packetTimeStamp, byte[] data) { if (log.isTraceEnabled()) { log.trace(this + " Adding request data: " + new String(data, StandardCharsets.UTF_8)); 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..9639b1897 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 @@ -5,12 +5,12 @@ import io.netty.channel.EventLoop; import lombok.extern.slf4j.Slf4j; import org.opensearch.migrations.replay.datahandlers.NettyPacketToHttpConsumer; +import org.opensearch.migrations.replay.datatypes.ChannelTask; import org.opensearch.migrations.replay.datatypes.ChannelTaskType; import org.opensearch.migrations.replay.datatypes.ConnectionReplaySession; -import org.opensearch.migrations.replay.datatypes.ISourceTrafficChannelKey; 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.IReplayContexts; import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; import org.opensearch.migrations.replay.util.StringTrackableCompletableFuture; @@ -36,80 +36,84 @@ public RequestSenderOrchestrator(ClientConnectionPool clientConnectionPool) { } public DiagnosticTrackableCompletableFuture - scheduleWork(ISourceTrafficChannelKey channelKey, Instant timestamp, + scheduleWork(IReplayContexts.IReplayerHttpTransactionContext ctx, Instant timestamp, Supplier> task) { - var connectionSession = clientConnectionPool.getCachedSession(channelKey, false); + var connectionSession = clientConnectionPool.getCachedSession(ctx.getChannelKeyContext(), 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(); + var scheduledContext = ctx.createScheduledContext(timestamp); // 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 // first needing to wait for a connection to succeed. In fact, making them more independent // means that the work item being enqueued is less likely to cause a connection timeout. - connectionSession.eventLoop.schedule(()-> - task.get().map(f->f.whenComplete((v,t) -> { - if (t!=null) { - finalTunneledResponse.future.completeExceptionally(t); - } else { - finalTunneledResponse.future.complete(v); - } - }), - ()->""), + connectionSession.eventLoop.schedule(()-> { + scheduledContext.close(); + return task.get().map(f -> f.whenComplete((v, t) -> { + if (t != null) { + finalTunneledResponse.future.completeExceptionally(t); + } else { + finalTunneledResponse.future.complete(v); + } + }), + () -> ""); + }, getDelayFromNowMs(timestamp), TimeUnit.MILLISECONDS); return finalTunneledResponse; } public DiagnosticTrackableCompletableFuture - scheduleRequest(UniqueReplayerRequestKey requestKey, Instant start, Duration interval, Stream packets) { + scheduleRequest(UniqueReplayerRequestKey requestKey, IReplayContexts.IReplayerHttpTransactionContext 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.getLogicalEnclosingScope(), + requestKey.getReplayerRequestIndex(), false, finalTunneledResponse, + channelFutureAndRequestSchedule-> scheduleSendOnConnectionReplaySession(ctx, channelFutureAndRequestSchedule, finalTunneledResponse, start, interval, packets)); } - public StringTrackableCompletableFuture scheduleClose(ISourceTrafficChannelKey channelKey, - int channelInteractionNum, Instant timestamp) { + public StringTrackableCompletableFuture scheduleClose(IReplayContexts.IChannelKeyContext 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()); + clientConnectionPool.closeConnection(ctx); finalTunneledResponse.future.complete(null); }))); return finalTunneledResponse; } private DiagnosticTrackableCompletableFuture - asynchronouslyInvokeRunnableToSetupFuture(ISourceTrafficChannelKey channelKey, int channelInteractionNumber, + asynchronouslyInvokeRunnableToSetupFuture(IReplayContexts.IChannelKeyContext ctx, int channelInteractionNumber, boolean ignoreIfChannelNotPresent, DiagnosticTrackableCompletableFuture finalTunneledResponse, Consumer successFn) { var channelFutureAndScheduleFuture = - clientConnectionPool.submitEventualSessionGet(channelKey, ignoreIfChannelNotPresent); + clientConnectionPool.submitEventualSessionGet(ctx, ignoreIfChannelNotPresent); 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,25 +122,28 @@ 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(); runAfterChannelSetup(channelFutureAndRequestSchedule, finalTunneledResponse, replaySession -> { + log.atTrace().setMessage(()->"adding work item at slot " + + channelInteractionNumber + " for " + ctx + " with " + + replaySession.scheduleSequencer).log(); replaySession.scheduleSequencer.add(channelInteractionNumber, () -> successFn.accept(channelFutureAndRequestSchedule), - x -> x.run()); + Runnable::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 +154,12 @@ public StringTrackableCompletableFuture scheduleClose(ISourceTrafficChanne return finalTunneledResponse; } - private void scheduleOnConnectionReplaySession(ISourceTrafficChannelKey channelKey, int channelInteractionIdx, + private void scheduleOnConnectionReplaySession(IReplayContexts.IChannelKeyContext 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,9 +170,11 @@ 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 running the scheduled task: " + ctx + + " interaction: " + channelInteraction).log(); } else { - log.atInfo().setMessage(()->"scheduled future has finished for "+channelInteraction).log(); + log.atInfo().setMessage(()->"scheduled task has finished for " + ctx + " interaction: " + + channelInteraction).log(); } }); } else { @@ -187,7 +197,7 @@ private void scheduleOnConnectionReplaySession(ISourceTrafficChannelKey chan var sf = eventLoop.schedule(runnable, getDelayFromNowMs(kvp.getKey()), TimeUnit.MILLISECONDS); sf.addListener(sfp->{ if (!sfp.isSuccess()) { - log.atWarn().setCause(sfp.cause()).setMessage(()->"Scheduled future was not successful for " + + log.atWarn().setCause(sfp.cause()).setMessage(()->"Scheduled future did not successfully run " + channelInteraction).log(); } }); @@ -195,19 +205,26 @@ private void scheduleOnConnectionReplaySession(ISourceTrafficChannelKey chan }), ()->""); } - private void scheduleSendOnConnectionReplaySession(UniqueReplayerRequestKey requestKey, + private void scheduleSendOnConnectionReplaySession(IReplayContexts.IReplayerHttpTransactionContext 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(), - packetReceiverRef), - eventLoop, packets.iterator(), start, interval, new AtomicInteger(), responseFuture); - scheduleOnConnectionReplaySession(requestKey.trafficStreamKey, requestKey.getSourceRequestIndex(), + Runnable packetSender = () -> { + sendNextPartAndContinue(() -> + memoizePacketConsumer(ctx, channelFutureAndRequestSchedule.getInnerChannelFuture(), + packetReceiverRef), + eventLoop, packets.iterator(), start, interval, new AtomicInteger(), responseFuture); + }; + var scheduledContext = ctx.createScheduledContext(start); + scheduleOnConnectionReplaySession(ctx.getLogicalEnclosingScope(), + ctx.getReplayerRequestKey().getSourceRequestIndex(), channelFutureAndRequestSchedule, responseFuture, start, - new ChannelTask(ChannelTaskType.TRANSMIT, packetSender)); + new ChannelTask(ChannelTaskType.TRANSMIT, ()->{ + scheduledContext.close(); + packetSender.run(); + })); } private void runAfterChannelSetup(ConnectionReplaySession channelFutureAndItsFutureRequests, @@ -234,14 +251,15 @@ private long getDelayFromNowMs(Instant to) { } private static NettyPacketToHttpConsumer - getPacketReceiver(UniqueReplayerRequestKey requestKey, ChannelFuture channelFuture, - AtomicReference packetReceiver) { + memoizePacketConsumer(IReplayContexts.IReplayerHttpTransactionContext httpTransactionContext, ChannelFuture channelFuture, + AtomicReference packetReceiver) { if (packetReceiver.get() == null) { - packetReceiver.set(new NettyPacketToHttpConsumer(channelFuture, requestKey.toString(), requestKey)); + packetReceiver.set(new NettyPacketToHttpConsumer(channelFuture, httpTransactionContext)); } return packetReceiver.get(); } + // TODO - rewrite this - the recursion (at least as it is) is terribly confusing private void sendNextPartAndContinue(Supplier packetHandlerSupplier, EventLoop eventLoop, Iterator iterator, Instant start, Duration interval, AtomicInteger counter, diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/ResultsToLogsConsumer.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/ResultsToLogsConsumer.java index 250bf21dd..a38bf273f 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/ResultsToLogsConsumer.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/ResultsToLogsConsumer.java @@ -42,7 +42,7 @@ private Map toJSONObject(SourceTargetCaptureTuple tuple, ParsedH parsed.targetRequestOp.ifPresent(r -> tupleMap.put("targetRequest", r)); parsed.targetResponseOp.ifPresent(r -> tupleMap.put("targetResponse", r)); - tupleMap.put("connectionId", formatUniqueRequestKey(tuple.uniqueRequestKey)); + tupleMap.put("connectionId", formatUniqueRequestKey(tuple.getRequestKey())); Optional.ofNullable(tuple.errorCause).ifPresent(e -> tupleMap.put("error", e.toString())); return tupleMap; @@ -127,7 +127,7 @@ public static String toTransactionSummaryString(SourceTargetCaptureTuple tuple, // TARGET_LATENCY .add(t.map(r->""+r.get(ParsedHttpMessagesAsDicts.RESPONSE_TIME_MS_KEY)).orElse(MISSING_STR)) // REQUEST_ID - .add(formatUniqueRequestKey(tuple.uniqueRequestKey)) + .add(formatUniqueRequestKey(tuple.getRequestKey())) .toString(); } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/SourceTargetCaptureTuple.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/SourceTargetCaptureTuple.java index 98fd89fda..5bf8b1d5c 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/SourceTargetCaptureTuple.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/SourceTargetCaptureTuple.java @@ -4,7 +4,8 @@ import lombok.extern.slf4j.Slf4j; import org.opensearch.migrations.replay.datatypes.HttpRequestTransformationStatus; import org.opensearch.migrations.replay.datatypes.TransformedPackets; -import org.opensearch.migrations.replay.datatypes.UniqueSourceRequestKey; +import org.opensearch.migrations.replay.datatypes.UniqueReplayerRequestKey; +import org.opensearch.migrations.replay.tracing.IReplayContexts; import java.time.Duration; import java.util.List; @@ -13,22 +14,22 @@ @Slf4j public class SourceTargetCaptureTuple implements AutoCloseable { - final UniqueSourceRequestKey uniqueRequestKey; final RequestResponsePacketPair sourcePair; final TransformedPackets targetRequestData; final List targetResponseData; final HttpRequestTransformationStatus transformationStatus; final Throwable errorCause; Duration targetResponseDuration; + final IReplayContexts.ITupleHandlingContext context; - public SourceTargetCaptureTuple(@NonNull UniqueSourceRequestKey uniqueRequestKey, + public SourceTargetCaptureTuple(@NonNull IReplayContexts.ITupleHandlingContext tupleHandlingContext, RequestResponsePacketPair sourcePair, TransformedPackets targetRequestData, List targetResponseData, HttpRequestTransformationStatus transformationStatus, Throwable errorCause, Duration targetResponseDuration) { - this.uniqueRequestKey = uniqueRequestKey; + this.context = tupleHandlingContext; this.sourcePair = sourcePair; this.targetRequestData = targetRequestData; this.targetResponseData = targetResponseData; @@ -39,14 +40,14 @@ public SourceTargetCaptureTuple(@NonNull UniqueSourceRequestKey uniqueRequestKey @Override public void close() { - Optional.ofNullable(targetRequestData).ifPresent(d->d.close()); + Optional.ofNullable(targetRequestData).ifPresent(TransformedPackets::close); } @Override public String toString() { return HttpByteBufFormatter.setPrintStyleFor(HttpByteBufFormatter.PacketPrintFormat.TRUNCATED, () -> { final StringJoiner sj = new StringJoiner("\n ", "SourceTargetCaptureTuple{","}"); - sj.add("diagnosticLabel=").add(uniqueRequestKey.toString()); + sj.add("diagnosticLabel=").add(context.toString()); if (sourcePair != null) { sj.add("sourcePair=").add(sourcePair.toString()); } if (targetResponseDuration != null) { sj.add("targetResponseDuration=").add(targetResponseDuration+""); } Optional.ofNullable(targetRequestData).ifPresent(d-> sj.add("targetRequestData=") @@ -59,4 +60,8 @@ public String toString() { return sj.toString(); }); } + + public UniqueReplayerRequestKey getRequestKey() { + return context.getLogicalEnclosingScope().getReplayerRequestKey(); + } } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/TrafficCaptureSourceFactory.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/TrafficCaptureSourceFactory.java index 93d1bb664..6ad84db0e 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/TrafficCaptureSourceFactory.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/TrafficCaptureSourceFactory.java @@ -3,6 +3,7 @@ import lombok.extern.slf4j.Slf4j; import org.opensearch.migrations.replay.kafka.KafkaBehavioralPolicy; import org.opensearch.migrations.replay.kafka.KafkaTrafficCaptureSource; +import org.opensearch.migrations.replay.tracing.RootReplayerContext; import org.opensearch.migrations.replay.traffic.source.BlockingTrafficSource; import org.opensearch.migrations.replay.traffic.source.ISimpleTrafficCaptureSource; import org.opensearch.migrations.replay.traffic.source.InputStreamOfTraffic; @@ -18,12 +19,14 @@ public class TrafficCaptureSourceFactory { private TrafficCaptureSourceFactory() {} public static BlockingTrafficSource - createTrafficCaptureSource(TrafficReplayer.Parameters appParams, Duration bufferTimeWindow) throws IOException { - return new BlockingTrafficSource(createUnbufferedTrafficCaptureSource(appParams), bufferTimeWindow); + createTrafficCaptureSource(RootReplayerContext ctx, + TrafficReplayer.Parameters appParams, Duration bufferTimeWindow) throws IOException { + return new BlockingTrafficSource(createUnbufferedTrafficCaptureSource(ctx, appParams), bufferTimeWindow); } public static ISimpleTrafficCaptureSource - createUnbufferedTrafficCaptureSource(TrafficReplayer.Parameters appParams) throws IOException { + createUnbufferedTrafficCaptureSource(RootReplayerContext ctx, + TrafficReplayer.Parameters appParams) throws IOException { boolean isKafkaActive = TrafficReplayer.validateRequiredKafkaParams(appParams.kafkaTrafficBrokers, appParams.kafkaTrafficTopic, appParams.kafkaTrafficGroupId); boolean isInputFileActive = appParams.inputFilename != null; @@ -32,13 +35,14 @@ private TrafficCaptureSourceFactory() {} } if (isKafkaActive) { - return KafkaTrafficCaptureSource.buildKafkaConsumer(appParams.kafkaTrafficBrokers, appParams.kafkaTrafficTopic, - appParams.kafkaTrafficGroupId, appParams.kafkaTrafficEnableMSKAuth, appParams.kafkaTrafficPropertyFile, + return KafkaTrafficCaptureSource.buildKafkaSource(ctx, + appParams.kafkaTrafficBrokers, appParams.kafkaTrafficTopic, + appParams.kafkaTrafficGroupId, appParams.kafkaTrafficEnableMSKAuth, + appParams.kafkaTrafficPropertyFile, Clock.systemUTC(), new KafkaBehavioralPolicy()); - } else if (isInputFileActive) { - return new InputStreamOfTraffic(new FileInputStream(appParams.inputFilename)); } else { - return new InputStreamOfTraffic(System.in); + return new InputStreamOfTraffic(ctx, + isInputFileActive ? new FileInputStream(appParams.inputFilename) : System.in); } } } 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..564c649de 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 @@ -13,40 +13,42 @@ import lombok.NonNull; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; -import org.opensearch.migrations.coreutils.MetricsLogger; import org.opensearch.migrations.replay.datahandlers.IPacketFinalizingConsumer; -import org.opensearch.migrations.transform.IHttpMessage; import org.opensearch.migrations.replay.datatypes.HttpRequestTransformationStatus; -import org.opensearch.migrations.replay.datatypes.ISourceTrafficChannelKey; import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; import org.opensearch.migrations.replay.datatypes.TransformedPackets; import org.opensearch.migrations.replay.datatypes.UniqueReplayerRequestKey; +import org.opensearch.migrations.replay.tracing.IReplayContexts; +import org.opensearch.migrations.replay.tracing.IRootReplayerContext; +import org.opensearch.migrations.replay.tracing.RootReplayerContext; import org.opensearch.migrations.replay.traffic.source.BlockingTrafficSource; import org.opensearch.migrations.replay.traffic.source.ITrafficCaptureSource; import org.opensearch.migrations.replay.traffic.source.ITrafficStreamWithKey; import org.opensearch.migrations.replay.traffic.source.TrafficStreamLimiter; import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; import org.opensearch.migrations.replay.util.StringTrackableCompletableFuture; +import org.opensearch.migrations.tracing.RootOtelContext; import org.opensearch.migrations.trafficcapture.protos.TrafficStreamUtils; import org.opensearch.migrations.transform.IAuthTransformer; import org.opensearch.migrations.transform.IAuthTransformerFactory; +import org.opensearch.migrations.transform.IHttpMessage; import org.opensearch.migrations.transform.IJsonTransformer; import org.opensearch.migrations.transform.RemovingAuthTransformerFactory; import org.opensearch.migrations.transform.StaticAuthTransformerFactory; import org.slf4j.event.Level; import org.slf4j.spi.LoggingEventBuilder; -import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; import software.amazon.awssdk.arns.Arn; +import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; import software.amazon.awssdk.regions.Region; import javax.net.ssl.SSLException; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.nio.charset.StandardCharsets; import java.io.EOFException; import java.io.IOException; import java.lang.ref.WeakReference; import java.net.URI; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Paths; import java.time.Duration; import java.time.Instant; import java.util.ArrayList; @@ -67,13 +69,9 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.opensearch.migrations.coreutils.MetricsLogger.initializeOpenTelemetry; - @Slf4j public class TrafficReplayer { - private static final MetricsLogger TUPLE_METRICS_LOGGER = new MetricsLogger("SourceTargetCaptureTuple"); - public static final String SIGV_4_AUTH_HEADER_SERVICE_REGION_ARG = "--sigv4-auth-header-service-region"; public static final String AUTH_HEADER_VALUE_ARG = "--auth-header-value"; public static final String REMOVE_AUTH_HEADER_VALUE_ARG = "--remove-auth-header"; @@ -86,20 +84,21 @@ public class TrafficReplayer { private final TrafficStreamLimiter liveTrafficStreamLimiter; private final AtomicInteger successfulRequestCount; private final AtomicInteger exceptionRequestCount; - private ConcurrentHashMap> requestFutureMap; - private ConcurrentHashMap> requestToFinalWorkFuturesMap; - private AtomicBoolean stopReadingRef; - private AtomicReference> allRemainingWorkFutureOrShutdownSignalRef; - private AtomicReference shutdownReasonRef; - private AtomicReference> shutdownFutureRef; - private AtomicReference>> nextChunkFutureRef; - private ConcurrentHashMap liveRequests = new ConcurrentHashMap<>(); + private final AtomicBoolean stopReadingRef; + private final AtomicReference> allRemainingWorkFutureOrShutdownSignalRef; + private final AtomicReference shutdownReasonRef; + private final AtomicReference> shutdownFutureRef; + private final AtomicReference>> nextChunkFutureRef; + private final ConcurrentHashMap liveRequests = new ConcurrentHashMap<>(); private Future nettyShutdownFuture; - public class DualException extends Exception { + public static class DualException extends Exception { public final Throwable originalCause; public final Throwable immediateCause; public DualException(Throwable originalCause, Throwable immediateCause) { @@ -114,43 +113,47 @@ public DualException(String message, Throwable originalCause, Throwable immediat } } - public class TerminationException extends DualException { + public static class TerminationException extends DualException { public TerminationException(Throwable originalCause, Throwable immediateCause) { super(originalCause, immediateCause); } } - public TrafficReplayer(URI serverUri, + public TrafficReplayer(IRootReplayerContext context, + URI serverUri, String fullTransformerConfig, IAuthTransformerFactory authTransformerFactory, boolean allowInsecureConnections) throws SSLException { - this(serverUri, fullTransformerConfig, authTransformerFactory, null, allowInsecureConnections, + this(context, serverUri, fullTransformerConfig, authTransformerFactory, null, allowInsecureConnections, 0, 1024); } - public TrafficReplayer(URI serverUri, + public TrafficReplayer(IRootReplayerContext context, + URI serverUri, String fullTransformerConfig, IAuthTransformerFactory authTransformerFactory, String userAgent, boolean allowInsecureConnections, int numSendingThreads, int maxConcurrentOutstandingRequests) throws SSLException { - this(serverUri, authTransformerFactory, allowInsecureConnections, + this(context, serverUri, authTransformerFactory, allowInsecureConnections, numSendingThreads, maxConcurrentOutstandingRequests, new TransformationLoader() .getTransformerFactoryLoader(serverUri.getHost(), userAgent, fullTransformerConfig) ); } - public TrafficReplayer(URI serverUri, + public TrafficReplayer(IRootReplayerContext context, + URI serverUri, IAuthTransformerFactory authTransformer, boolean allowInsecureConnections, int numSendingThreads, int maxConcurrentOutstandingRequests, IJsonTransformer jsonTransformer) throws SSLException { + this.topLevelContext = context; if (serverUri.getPort() < 0) { throw new IllegalArgumentException("Port not present for URI: "+serverUri); } @@ -198,7 +201,7 @@ public static boolean validateRequiredKafkaParams(String brokers, String topic, return true; } - static class Parameters { + public static class Parameters { @Parameter(required = true, arity = 1, description = "URI of the target cluster/domain") @@ -383,24 +386,21 @@ public static void main(String[] args) System.exit(3); return; } - if (params.otelCollectorEndpoint != null) { - initializeOpenTelemetry("traffic-replayer", params.otelCollectorEndpoint); - } - - try (var blockingTrafficSource = TrafficCaptureSourceFactory.createTrafficCaptureSource(params, + var topContext = new RootReplayerContext(RootOtelContext.initializeOpenTelemetryWithCollectorOrAsNoop(params.otelCollectorEndpoint, + "replay")); + try (var blockingTrafficSource = TrafficCaptureSourceFactory.createTrafficCaptureSource(topContext, params, Duration.ofSeconds(params.lookaheadTimeSeconds)); var authTransformer = buildAuthTransformerFactory(params)) { String transformerConfig = getTransformerConfig(params); - if (transformerConfig != null) - { - log.info("Transformations config string: ", transformerConfig); + if (transformerConfig != null) { + log.atInfo().setMessage(()->"Transformations config string: " + transformerConfig).log(); } - var tr = new TrafficReplayer(uri, transformerConfig, authTransformer, params.userAgent, + var tr = new TrafficReplayer(topContext, uri, transformerConfig, authTransformer, params.userAgent, params.allowInsecureConnections, params.numClientThreads, params.maxConcurrentRequests); setupShutdownHookForReplayer(tr); - var tupleWriter = new TupleParserChainConsumer(TUPLE_METRICS_LOGGER, new ResultsToLogsConsumer()); + var tupleWriter = new TupleParserChainConsumer(new ResultsToLogsConsumer()); var timeShifter = new TimeShifter(params.speedupFactor); tr.setupRunAndWaitForReplayWithShutdownChecks(Duration.ofSeconds(params.observedPacketConnectionTimeout), blockingTrafficSource, timeShifter, tupleWriter); @@ -437,11 +437,11 @@ private static void setupShutdownHookForReplayer(TrafficReplayer tr) { * aspect out from the core logic below. */ private static String formatAuthArgFlagsAsString() { - return List.of(REMOVE_AUTH_HEADER_VALUE_ARG, - AUTH_HEADER_VALUE_ARG, - AWS_AUTH_HEADER_USER_AND_SECRET_ARG, - SIGV_4_AUTH_HEADER_SERVICE_REGION_ARG).stream() - .collect(Collectors.joining(", ")); + return String.join(", ", + REMOVE_AUTH_HEADER_VALUE_ARG, + AUTH_HEADER_VALUE_ARG, + AWS_AUTH_HEADER_USER_AND_SECRET_ARG, + SIGV_4_AUTH_HEADER_SERVICE_REGION_ARG); } private static IAuthTransformerFactory buildAuthTransformerFactory(Parameters params) { @@ -484,7 +484,7 @@ private static IAuthTransformerFactory buildAuthTransformerFactory(Parameters pa String region = serviceAndRegion[1]; return new IAuthTransformerFactory() { - DefaultCredentialsProvider defaultCredentialsProvider = DefaultCredentialsProvider.create(); + final DefaultCredentialsProvider defaultCredentialsProvider = DefaultCredentialsProvider.create(); @Override public IAuthTransformer getAuthTransformer(IHttpMessage httpMessage) { return new SigV4Signer(defaultCredentialsProvider, serviceName, region, "https", null); @@ -524,10 +524,8 @@ void setupRunAndWaitForReplay(Duration observedPacketConnectionTimeout, } finally { trafficToHttpTransactionAccumulator.close(); wrapUpWorkAndEmitSummary(replayEngine, trafficToHttpTransactionAccumulator); - if (shutdownFutureRef.get() == null) { - assert requestToFinalWorkFuturesMap.isEmpty() : - "expected to wait for all the in flight requests to fully flush and self destruct themselves"; - } + assert shutdownFutureRef.get() != null || requestToFinalWorkFuturesMap.isEmpty() : + "expected to wait for all the in flight requests to fully flush and self destruct themselves"; } } @@ -549,8 +547,7 @@ private void wrapUpWorkAndEmitSummary(ReplayEngine replayEngine, CapturedTraffic logLevel = secondaryLogLevel; } } - if (requestToFinalWorkFuturesMap.size() > 0 || - exceptionRequestCount.get() > 0) { + if (!requestToFinalWorkFuturesMap.isEmpty() || exceptionRequestCount.get() > 0) { log.atWarn().setMessage("{} in-flight requests being dropped due to pending shutdown; " + "{} requests to the target threw an exception; " + "{} requests were successfully processed.") @@ -601,11 +598,13 @@ class TrafficReplayerAccumulationCallbacks implements AccumulationCallbacks { private ITrafficCaptureSource trafficCaptureSource; @Override - public void onRequestReceived(UniqueReplayerRequestKey requestKey, HttpMessageAndTimestamp request) { + public void onRequestReceived(IReplayContexts.@NonNull IReplayerHttpTransactionContext ctx, + @NonNull HttpMessageAndTimestamp request) { replayEngine.setFirstTimestamp(request.getFirstPacketTimestamp()); liveTrafficStreamLimiter.addWork(1); - var requestPushFuture = transformAndSendRequest(replayEngine, request, requestKey); + var requestPushFuture = transformAndSendRequest(replayEngine, request, ctx); + var requestKey = ctx.getReplayerRequestKey(); requestFutureMap.put(requestKey, requestPushFuture); liveRequests.put(requestKey, true); requestPushFuture.map(f->f.whenComplete((v,t)->{ @@ -618,12 +617,13 @@ public void onRequestReceived(UniqueReplayerRequestKey requestKey, HttpMessageAn } @Override - public void onFullDataReceived(@NonNull UniqueReplayerRequestKey requestKey, + public void onFullDataReceived(@NonNull IReplayContexts.IReplayerHttpTransactionContext ctx, @NonNull RequestResponsePacketPair rrPair) { - log.atInfo().setMessage(()->"Done receiving captured stream for " + requestKey + + log.atInfo().setMessage(()->"Done receiving captured stream for " + ctx + ":" + rrPair.requestData).log(); + var requestKey = ctx.getReplayerRequestKey(); var resultantCf = requestFutureMap.remove(requestKey) - .map(f -> f.handle((summary,t)->handleCompletedTransaction(requestKey, rrPair, summary, t)), + .map(f -> f.handle((summary,t)->handleCompletedTransaction(ctx, rrPair, summary, t)), () -> "TrafficReplayer.runReplayWithIOStreams.progressTracker"); if (!resultantCf.future.isDone()) { log.trace("Adding " + requestKey + " to targetTransactionInProgressMap"); @@ -634,18 +634,22 @@ public void onFullDataReceived(@NonNull UniqueReplayerRequestKey requestKey, } } - Void handleCompletedTransaction(@NonNull UniqueReplayerRequestKey requestKey, RequestResponsePacketPair rrPair, + Void handleCompletedTransaction(@NonNull IReplayContexts.IReplayerHttpTransactionContext context, + RequestResponsePacketPair rrPair, TransformedTargetRequestAndResponse summary, Throwable t) { - try { + try (var httpContext = rrPair.getHttpTransactionContext()) { // if this comes in with a serious Throwable (not an Exception), don't bother // packaging it up and calling the callback. // Escalate it up out handling stack and shutdown. if (t == null || t instanceof Exception) { - packageAndWriteResponse(resultTupleConsumer, requestKey, rrPair, summary, (Exception) t); - commitTrafficStreams(rrPair.trafficStreamKeysBeingHeld, rrPair.completionStatus); + try (var tupleHandlingContext = httpContext.createTupleContext()) { + packageAndWriteResponse(tupleHandlingContext, resultTupleConsumer, + rrPair, summary, (Exception) t); + } + commitTrafficStreams(rrPair.completionStatus, rrPair.trafficStreamKeysBeingHeld); return null; } else { - log.atError().setCause(t).setMessage(()->"Throwable passed to handle() for " + requestKey + + log.atError().setCause(t).setMessage(() -> "Throwable passed to handle() for " + context + ". Rethrowing.").log(); throw Lombok.sneakyThrow(t); } @@ -661,11 +665,12 @@ Void handleCompletedTransaction(@NonNull UniqueReplayerRequestKey requestKey, Re .setMessage("Unexpected exception while sending the " + "aggregated response and context for {} to the callback. " + "Proceeding, but the tuple receiver context may be compromised.") - .addArgument(requestKey) + .addArgument(context) .setCause(e) .log(); throw e; } finally { + var requestKey = context.getReplayerRequestKey(); requestToFinalWorkFuturesMap.remove(requestKey); log.trace("removed rrPair.requestData to " + "targetTransactionInProgressMap for " + @@ -675,61 +680,67 @@ Void handleCompletedTransaction(@NonNull UniqueReplayerRequestKey requestKey, Re @Override public void onTrafficStreamsExpired(RequestResponsePacketPair.ReconstructionStatus status, - List trafficStreamKeysBeingHeld) { - commitTrafficStreams(trafficStreamKeysBeingHeld, status); + IReplayContexts.@NonNull IChannelKeyContext ctx, + @NonNull List trafficStreamKeysBeingHeld) { + commitTrafficStreams(status, trafficStreamKeysBeingHeld); } @SneakyThrows - private void commitTrafficStreams(List trafficStreamKeysBeingHeld, - RequestResponsePacketPair.ReconstructionStatus status) { - commitTrafficStreams(trafficStreamKeysBeingHeld, - status != RequestResponsePacketPair.ReconstructionStatus.CLOSED_PREMATURELY); + private void commitTrafficStreams(RequestResponsePacketPair.ReconstructionStatus status, + List trafficStreamKeysBeingHeld) { + commitTrafficStreams(status != RequestResponsePacketPair.ReconstructionStatus.CLOSED_PREMATURELY, + trafficStreamKeysBeingHeld); } @SneakyThrows - private void commitTrafficStreams(List trafficStreamKeysBeingHeld, boolean shouldCommit) { + private void commitTrafficStreams(boolean shouldCommit, + List trafficStreamKeysBeingHeld) { if (shouldCommit && trafficStreamKeysBeingHeld != null) { for (var tsk : trafficStreamKeysBeingHeld) { + tsk.getTrafficStreamsContext().close(); trafficCaptureSource.commitTrafficStream(tsk); } } } @Override - public void onConnectionClose(ISourceTrafficChannelKey channelKey, int channelInteractionNum, - RequestResponsePacketPair.ReconstructionStatus status, Instant timestamp, - List trafficStreamKeysBeingHeld) { + public void onConnectionClose(int channelInteractionNum, + IReplayContexts.@NonNull IChannelKeyContext ctx, + RequestResponsePacketPair.ReconstructionStatus status, + @NonNull Instant timestamp, @NonNull List trafficStreamKeysBeingHeld) { replayEngine.setFirstTimestamp(timestamp); - replayEngine.closeConnection(channelKey, channelInteractionNum, timestamp); - commitTrafficStreams(trafficStreamKeysBeingHeld, status); + var cf = replayEngine.closeConnection(channelInteractionNum, ctx, timestamp); + cf.map(f->f.whenComplete((v,t)->{ + commitTrafficStreams(status, trafficStreamKeysBeingHeld); + }), ()->"closing the channel in the ReplayEngine"); } @Override - public void onTrafficStreamIgnored(@NonNull ITrafficStreamKey tsk) { - commitTrafficStreams(List.of(tsk), true); + public void onTrafficStreamIgnored(@NonNull IReplayContexts.ITrafficStreamsLifecycleContext ctx) { + commitTrafficStreams(true, List.of(ctx.getTrafficStreamKey())); } private TransformedTargetRequestAndResponse - packageAndWriteResponse(Consumer tupleWriter, - @NonNull UniqueReplayerRequestKey requestKey, + packageAndWriteResponse(IReplayContexts.ITupleHandlingContext tupleHandlingContext, + Consumer tupleWriter, RequestResponsePacketPair rrPair, TransformedTargetRequestAndResponse summary, Exception t) { log.trace("done sending and finalizing data to the packet handler"); - try (var requestResponseTuple = getSourceTargetCaptureTuple(requestKey, rrPair, summary, t)) { + try (var requestResponseTuple = getSourceTargetCaptureTuple(tupleHandlingContext, rrPair, summary, t)) { log.atInfo().setMessage(()->"Source/Target Request/Response tuple: " + requestResponseTuple).log(); tupleWriter.accept(requestResponseTuple); } if (t != null) { throw new CompletionException(t); } if (summary.getError() != null) { - log.atInfo().setCause(summary.getError()).setMessage("Exception for request {}: ") - .addArgument(requestKey).log(); + log.atInfo().setCause(summary.getError()).setMessage("Exception for {}: ") + .addArgument(tupleHandlingContext).log(); exceptionRequestCount.incrementAndGet(); } else if (summary.getTransformationStatus() == HttpRequestTransformationStatus.ERROR) { - log.atInfo().setCause(summary.getError()).setMessage("Unknown error transforming the request {}: ") - .addArgument(requestKey).log(); + log.atInfo().setCause(summary.getError()).setMessage("Unknown error transforming {}: ") + .addArgument(tupleHandlingContext).log(); exceptionRequestCount.incrementAndGet(); } else { successfulRequestCount.incrementAndGet(); @@ -833,19 +844,19 @@ private static String formatWorkItem(DiagnosticTrackableCompletableFuture(), HttpRequestTransformationStatus.ERROR, t, Duration.ZERO); } else { - requestResponseTriple = new SourceTargetCaptureTuple(uniqueRequestKey, rrPair, + requestResponseTuple = new SourceTargetCaptureTuple(tupleHandlingContext, rrPair, summary.requestPackets, summary.getReceiptTimeAndResponsePackets() .map(Map.Entry::getValue).collect(Collectors.toList()), @@ -854,41 +865,40 @@ private static String formatWorkItem(DiagnosticTrackableCompletableFuture transformAndSendRequest(ReplayEngine replayEngine, HttpMessageAndTimestamp request, - UniqueReplayerRequestKey requestKey) { - return transformAndSendRequest(inputRequestTransformerFactory, replayEngine, - request.getFirstPacketTimestamp(), request.getLastPacketTimestamp(), requestKey, + IReplayContexts.IReplayerHttpTransactionContext ctx) { + return transformAndSendRequest(inputRequestTransformerFactory, replayEngine, ctx, + request.getFirstPacketTimestamp(), request.getLastPacketTimestamp(), request.packetBytes::stream); } public static DiagnosticTrackableCompletableFuture transformAndSendRequest(PacketToTransformingHttpHandlerFactory inputRequestTransformerFactory, - ReplayEngine replayEngine, + ReplayEngine replayEngine, IReplayContexts.IReplayerHttpTransactionContext ctx, @NonNull Instant start, @NonNull Instant end, - UniqueReplayerRequestKey requestKey, Supplier> packetsSupplier) { try { - var transformationCompleteFuture = replayEngine.scheduleTransformationWork(requestKey, start, ()-> - transformAllData(inputRequestTransformerFactory.create(requestKey), packetsSupplier)); - log.atDebug().setMessage(()->"finalizeRequest future for transformation of " + requestKey + + var transformationCompleteFuture = replayEngine.scheduleTransformationWork(ctx, start, ()-> + transformAllData(inputRequestTransformerFactory.create(ctx), packetsSupplier)); + log.atDebug().setMessage(()->"finalizeRequest future for transformation of " + ctx + " = " + 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(ctx, start, end, transformedResult.transformedOutput.size(), transformedResult.transformedOutput.streamRetained()) - .map(future->future.thenApply(t-> + .map(future->future.thenApply(t -> new TransformedTargetRequestAndResponse(transformedResult.transformedOutput, t, transformedResult.transformationStatus, t.error)), ()->"(if applicable) packaging transformed result into a completed TransformedTargetRequestAndResponse object") - .map(future->future.exceptionally(t-> + .map(future->future.exceptionally(t -> new TransformedTargetRequestAndResponse(transformedResult.transformedOutput, transformedResult.transformationStatus, t)), ()->"(if applicable) packaging transformed result into a failed TransformedTargetRequestAndResponse object"), @@ -968,7 +978,8 @@ public void pullCaptureFromSourceToAccumulator( if (stopReadingRef.get()) { break; } - this.nextChunkFutureRef.set(trafficChunkStream.readNextTrafficStreamChunk()); + this.nextChunkFutureRef.set(trafficChunkStream + .readNextTrafficStreamChunk(topLevelContext::createReadChunkContext)); List trafficStreams = null; try { trafficStreams = this.nextChunkFutureRef.get().get(); diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/TupleParserChainConsumer.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/TupleParserChainConsumer.java index 365e5cf80..dc91ce27f 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/TupleParserChainConsumer.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/TupleParserChainConsumer.java @@ -1,28 +1,21 @@ package org.opensearch.migrations.replay; import lombok.NonNull; -import org.opensearch.migrations.coreutils.MetricsLogger; -import java.util.Optional; import java.util.function.BiConsumer; import java.util.function.Consumer; public class TupleParserChainConsumer implements Consumer { - private final MetricsLogger optionalMetricsLoggerToEmitStats; private final BiConsumer innerConsumer; - public TupleParserChainConsumer(MetricsLogger optionalMetricsLoggerToEmitStats, - @NonNull BiConsumer innerConsumer) { - this.optionalMetricsLoggerToEmitStats = optionalMetricsLoggerToEmitStats; + public TupleParserChainConsumer(@NonNull BiConsumer innerConsumer) { this.innerConsumer = innerConsumer; } @Override public void accept(SourceTargetCaptureTuple tuple) { var parsedMsgs = new ParsedHttpMessagesAsDicts(tuple); - Optional.ofNullable(optionalMetricsLoggerToEmitStats) - .ifPresent(ml->parsedMsgs.buildStatusCodeMetrics(ml, tuple.uniqueRequestKey).emit()); innerConsumer.accept(tuple, parsedMsgs); } } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/Utils.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/Utils.java index ba383d458..9a4a987f2 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/Utils.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/Utils.java @@ -14,10 +14,6 @@ import java.util.Base64; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.BiFunction; -import java.util.function.Function; -import java.util.stream.Collector; -import java.util.stream.Collectors; import java.util.stream.Stream; import java.util.zip.GZIPInputStream; import java.util.zip.GZIPOutputStream; @@ -36,20 +32,6 @@ public static long setIfLater(AtomicLong referenceValue, long pointInTimeMillis) return referenceValue.updateAndGet(existing -> Math.max(existing, pointInTimeMillis)); } - /** - * See https://en.wikipedia.org/wiki/Fold_(higher-order_function) - */ - public static Collector - foldLeft(final B seedValue, final BiFunction f) { - return Collectors.collectingAndThen( - Collectors.reducing( - Function.identity(), - a -> b -> f.apply(b, a), - Function::andThen), - finisherArg -> finisherArg.apply(seedValue) - ); - } - @SneakyThrows(value = {IOException.class}) public static String packetsToCompressedTrafficStream(Stream byteArrStream) { var tsb = TrafficStream.newBuilder() @@ -57,7 +39,7 @@ public static String packetsToCompressedTrafficStream(Stream byteArrStre var trafficStreamOfReads = byteArrStream.map(bArr->ReadObservation.newBuilder().setData(ByteString.copyFrom(bArr)).build()) .map(r->TrafficObservation.newBuilder().setRead(r)) - .collect(foldLeft(tsb, (existing,newObs)->tsb.addSubStream(newObs))) + .collect(org.opensearch.migrations.Utils.foldLeft(tsb, (existing, newObs)->tsb.addSubStream(newObs))) .build(); try (var baos = new ByteArrayOutputStream()) { try (var gzStream = new GZIPOutputStream(baos)) { 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..922a5a94d 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 @@ -23,19 +23,25 @@ import org.opensearch.migrations.coreutils.MetricsEvent; import org.opensearch.migrations.coreutils.MetricsLogger; import org.opensearch.migrations.replay.AggregatedRawResponse; -import org.opensearch.migrations.replay.datatypes.UniqueReplayerRequestKey; +import org.opensearch.migrations.replay.datahandlers.http.helpers.ReadMeteringHandler; +import org.opensearch.migrations.replay.datahandlers.http.helpers.WriteMeteringHandler; import org.opensearch.migrations.replay.netty.BacksideHttpWatcherHandler; import org.opensearch.migrations.replay.netty.BacksideSnifferHandler; +import org.opensearch.migrations.replay.tracing.IReplayContexts; import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; import org.opensearch.migrations.replay.util.StringTrackableCompletableFuture; +import org.opensearch.migrations.tracing.IScopedInstrumentationAttributes; +import org.opensearch.migrations.tracing.IWithTypedEnclosingScope; import java.net.URI; import java.time.Instant; +import java.util.NoSuchElementException; import java.util.Optional; import java.util.concurrent.CompletableFuture; @Slf4j public class NettyPacketToHttpConsumer implements IPacketFinalizingConsumer { + /** * 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. @@ -44,6 +50,9 @@ public class NettyPacketToHttpConsumer implements IPacketFinalizingConsumer activeChannelFuture; private final Channel channel; AggregatedRawResponse.Builder responseBuilder; - final String diagnosticLabel; - private UniqueReplayerRequestKey uniqueRequestKeyForMetricsLogging; + IWithTypedEnclosingScope currentRequestContextUnion; public NettyPacketToHttpConsumer(NioEventLoopGroup eventLoopGroup, URI serverUri, SslContext sslContext, - String diagnosticLabel, UniqueReplayerRequestKey uniqueRequestKeyForMetricsLogging) { - this(createClientConnection(eventLoopGroup, sslContext, serverUri, diagnosticLabel), - diagnosticLabel, uniqueRequestKeyForMetricsLogging); + IReplayContexts.IReplayerHttpTransactionContext httpTransactionContext) { + this(createClientConnection(eventLoopGroup, sslContext, serverUri, + httpTransactionContext.getLogicalEnclosingScope()), httpTransactionContext); } - public NettyPacketToHttpConsumer(ChannelFuture clientConnection, String diagnosticLabel, UniqueReplayerRequestKey uniqueRequestKeyForMetricsLogging) { - this.diagnosticLabel = "[" + diagnosticLabel + "] "; - this.uniqueRequestKeyForMetricsLogging = uniqueRequestKeyForMetricsLogging; + public NettyPacketToHttpConsumer(ChannelFuture clientConnection, + IReplayContexts.IReplayerHttpTransactionContext ctx) { + var parentContext = ctx.createTargetRequestContext(); + this.setCurrentMessageContext(parentContext.createHttpSendingContext()); responseBuilder = AggregatedRawResponse.builder(Instant.now()); DiagnosticTrackableCompletableFuture initialFuture = new StringTrackableCompletableFuture<>(new CompletableFuture<>(), @@ -86,8 +95,24 @@ public NettyPacketToHttpConsumer(ChannelFuture clientConnection, String diagnost }); } - public static ChannelFuture createClientConnection(EventLoopGroup eventLoopGroup, SslContext sslContext, - URI serverUri, String diagnosticLabel) { + private & + IScopedInstrumentationAttributes> + void setCurrentMessageContext(T requestSendingContext) { + currentRequestContextUnion = requestSendingContext; + } + + private IScopedInstrumentationAttributes getCurrentRequestSpan() { + return (IScopedInstrumentationAttributes) currentRequestContextUnion; + } + + public IReplayContexts.ITargetRequestContext getParentContext() { + return currentRequestContextUnion.getLogicalEnclosingScope(); + } + + public static ChannelFuture createClientConnection(EventLoopGroup eventLoopGroup, + SslContext sslContext, + URI serverUri, + IReplayContexts.IChannelKeyContext channelKeyContext) { String host = serverUri.getHost(); int port = serverUri.getPort(); log.atTrace().setMessage(()->"Active - setting up backend connection to " + host + ":" + port).log(); @@ -105,14 +130,14 @@ 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); var sslHandler = new SslHandler(sslEngine); addLoggingHandler(pipeline, "A"); - pipeline.addLast("ssl", sslHandler); + pipeline.addLast(SSL_HANDLER_NAME, sslHandler); sslHandler.handshakeFuture().addListener(handshakeFuture -> { if (handshakeFuture.isSuccess()) { rval.setSuccess(); @@ -126,7 +151,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()); } @@ -151,6 +176,25 @@ private void activateChannelForThisConsumer() { throw new IllegalStateException("Channel " + channel + "is being used elsewhere already!"); } var pipeline = channel.pipeline(); + // add these size counters BEFORE TLS? Notice that when removing from the pipeline, we need to be more careful + pipeline.addFirst(WRITE_COUNT_WATCHER_HANDLER_NAME, new WriteMeteringHandler(size->{ + // client side, so this is the request + if (size == 0) { return; } + if (!(this.currentRequestContextUnion instanceof IReplayContexts.IRequestSendingContext)) { + this.getCurrentRequestSpan().close(); + this.setCurrentMessageContext(getParentContext().createHttpSendingContext()); + } + getParentContext().onBytesSent(size); + })); + pipeline.addFirst(READ_COUNT_WATCHER_HANDLER_NAME, new ReadMeteringHandler(size->{ + // client side, so this is the response + if (size == 0) { return; } + if (!(this.currentRequestContextUnion instanceof IReplayContexts.IReceivingHttpResponseContext)) { + this.getCurrentRequestSpan().close(); + this.setCurrentMessageContext(getParentContext().createHttpReceivingContext()); + } + getParentContext().onBytesReceived(size); + })); addLoggingHandler(pipeline, "B"); pipeline.addLast(new BacksideSnifferHandler(responseBuilder)); addLoggingHandler(pipeline, "C"); @@ -173,13 +217,25 @@ private static void addLoggingHandler(ChannelPipeline pipeline, String name) { } private void deactivateChannel() { - var pipeline = channel.pipeline(); - log.atDebug().setMessage(()->"Resetting the pipeline currently at: " + pipeline).log(); - while (!(pipeline.last() instanceof SslHandler) && (pipeline.last() != null)) { - pipeline.removeLast(); + try { + var pipeline = channel.pipeline(); + log.atDebug().setMessage(() -> "Resetting the pipeline currently at: " + pipeline).log(); + for (var handlerName : new String[]{WRITE_COUNT_WATCHER_HANDLER_NAME, READ_COUNT_WATCHER_HANDLER_NAME}) { + try { + pipeline.remove(handlerName); + } catch (NoSuchElementException e) { + log.atDebug().setMessage(()->"Ignoring an exception that the "+handlerName+" wasn't present").log(); + } + } + while (!(pipeline.last() instanceof SslHandler) && (pipeline.last() != null)) { + pipeline.removeLast(); + } + channel.config().setAutoRead(false); + log.atDebug().setMessage(() -> "Reset the pipeline back to: " + pipeline).log(); + } finally { + getCurrentRequestSpan().close(); + getParentContext().close(); } - channel.config().setAutoRead(false); - log.atDebug().setMessage(()->"Reset the pipeline back to: " + pipeline).log(); } @Override @@ -190,8 +246,9 @@ 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(()-> httpContext().getReplayerRequestKey() + + "outbound channel was not set up successfully, NOT writing bytes hash=" + + System.identityHashCode(packetData)).log(); channel.close(); return DiagnosticTrackableCompletableFuture.Factory.failedFuture(channelInitException, ()->""); } @@ -201,6 +258,10 @@ public DiagnosticTrackableCompletableFuture consumeBytes(ByteBuf pa return activeChannelFuture; } + private IReplayContexts.IReplayerHttpTransactionContext httpContext() { + return getParentContext().getLogicalEnclosingScope(); + } + private DiagnosticTrackableCompletableFuture writePacketAndUpdateFuture(ByteBuf packetData) { final var completableFuture = new DiagnosticTrackableCompletableFuture(new CompletableFuture<>(), @@ -211,8 +272,8 @@ public DiagnosticTrackableCompletableFuture consumeBytes(ByteBuf pa 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(()-> httpContext().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 +283,19 @@ 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, + httpContext().getReplayerRequestKey().toString()) .setAttribute(MetricsAttributeKey.CONNECTION_ID, - uniqueRequestKeyForMetricsLogging.getTrafficStreamKey().getConnectionId()).emit(); + httpContext().getReplayerRequestKey().getTrafficStreamKey().getConnectionId()) + .emit(); completableFuture.future.completeExceptionally(cause); channel.close(); } @@ -241,8 +304,8 @@ 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, httpContext().getReplayerRequestKey()) + .setAttribute(MetricsAttributeKey.CONNECTION_ID, httpContext().getConnectionId()) .setAttribute(MetricsAttributeKey.SIZE_IN_BYTES, readableBytes).emit(); return completableFuture; } @@ -251,6 +314,11 @@ public DiagnosticTrackableCompletableFuture consumeBytes(ByteBuf pa public DiagnosticTrackableCompletableFuture finalizeRequest() { var ff = activeChannelFuture.getDeferredFutureThroughHandle((v,t)-> { + if (!(this.currentRequestContextUnion instanceof IReplayContexts.IReceivingHttpResponseContext)) { + this.getCurrentRequestSpan().close(); + this.setCurrentMessageContext(getParentContext().createWaitingForResponseContext()); + } + var future = new CompletableFuture(); var rval = new DiagnosticTrackableCompletableFuture(future, ()->"NettyPacketToHttpConsumer.finalizeRequest()"); @@ -262,7 +330,7 @@ public DiagnosticTrackableCompletableFuture consumeBytes(ByteBuf pa future.complete(responseBuilder.addErrorCause(t).build()); } return rval; - }, ()->"Waiting for previous consumes to set the callback") + }, ()->"Waiting for previous consumes to set the future") .map(f->f.whenComplete((v,t)-> deactivateChannel()), ()->"clearing pipeline"); log.atTrace().setMessage(()->"Chaining finalization work off of " + activeChannelFuture + ". Returning finalization future="+ff).log(); 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..16592f790 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 @@ -4,14 +4,14 @@ import io.netty.channel.embedded.EmbeddedChannel; import io.netty.handler.codec.http.HttpRequestDecoder; import lombok.extern.slf4j.Slf4j; +import org.opensearch.migrations.Utils; import org.opensearch.migrations.coreutils.MetricsAttributeKey; import org.opensearch.migrations.coreutils.MetricsEvent; import org.opensearch.migrations.coreutils.MetricsLogger; +import org.opensearch.migrations.replay.tracing.IReplayContexts; 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.util.DiagnosticTrackableCompletableFuture; import org.opensearch.migrations.replay.util.StringTrackableCompletableFuture; import org.opensearch.migrations.transform.IAuthTransformerFactory; @@ -50,7 +50,7 @@ public class HttpJsonTransformingConsumer implements IPacketFinalizingConsume private final RequestPipelineOrchestrator pipelineOrchestrator; private final EmbeddedChannel channel; private static final MetricsLogger metricsLogger = new MetricsLogger("HttpJsonTransformingConsumer"); - private UniqueReplayerRequestKey requestKeyForMetricsLogging; + private IReplayContexts.IRequestTransformationContext transformationContext; /** * Roughly try to keep track of how big each data chunk was that came into the transformer. These values @@ -67,16 +67,15 @@ public class HttpJsonTransformingConsumer implements IPacketFinalizingConsume public HttpJsonTransformingConsumer(IJsonTransformer transformer, IAuthTransformerFactory authTransformerFactory, IPacketFinalizingConsumer transformedPacketReceiver, - String diagnosticLabel, - UniqueReplayerRequestKey requestKeyForMetricsLogging) { + IReplayContexts.IReplayerHttpTransactionContext httpTransactionContext) { + transformationContext = httpTransactionContext.createTransformationContext(); 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, transformationContext); pipelineOrchestrator.addInitialHandlers(channel.pipeline(), transformer); - this.requestKeyForMetricsLogging = requestKeyForMetricsLogging; } private NettySendByteBufsToPacketHandlerHandler getOffloadingHandler() { @@ -116,6 +115,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) -> { + if (t != null) { + transformationContext.onTransformFailure(); + t = unwindPossibleCompletionException(t); + if (t instanceof NoContentException) { + return redriveWithoutTransformation(offloadingHandler.packetReceiver, t); + } else { + transformationContext.close(); + metricsLogger.atError(MetricsEvent.TRANSFORMING_REQUEST_FAILED, t) + .setAttribute(MetricsAttributeKey.REQUEST_ID, transformationContext.toString()) + .setAttribute(MetricsAttributeKey.CONNECTION_ID, transformationContext.getLogicalEnclosingScope().getConnectionId()) + .setAttribute(MetricsAttributeKey.CHANNEL_ID, channel.id().asLongText()).emit(); + throw new CompletionException(t); + } + } else { + transformationContext.close(); + transformationContext.onTransformSuccess(); + metricsLogger.atSuccess(MetricsEvent.REQUEST_WAS_TRANSFORMED) + .setAttribute(MetricsAttributeKey.REQUEST_ID, transformationContext) + .setAttribute(MetricsAttributeKey.CONNECTION_ID, transformationContext.getLogicalEnclosingScope().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) { while (t instanceof CompletionException) { - t = ((CompletionException) t).getCause(); + t = t.getCause(); } return t; } @@ -176,12 +180,17 @@ 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.getLogicalEnclosingScope().getConnectionId()) .setAttribute(MetricsAttributeKey.CHANNEL_ID, channel.id().asLongText()).emit(); - return finalizedFuture.map(f->f.thenApply(r->reason == null ? - new TransformedOutputAndResult(r, HttpRequestTransformationStatus.SKIPPED, null) : - new TransformedOutputAndResult(r, HttpRequestTransformationStatus.ERROR, reason)), + return finalizedFuture.map(f->f.thenApply(r -> reason == null ? + new TransformedOutputAndResult<>(r, HttpRequestTransformationStatus.SKIPPED, null) : + new TransformedOutputAndResult<>(r, HttpRequestTransformationStatus.ERROR, reason) + ) + .whenComplete((v,t)->{ + transformationContext.onTransformSkip(); + transformationContext.close(); + }), ()->"HttpJsonTransformingConsumer.redriveWithoutTransformation().map()"); } } \ No newline at end of file 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..56ffea6c7 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 @@ -4,18 +4,19 @@ import io.netty.channel.ChannelInboundHandlerAdapter; import io.netty.handler.codec.http.HttpContent; import io.netty.handler.codec.http.HttpRequest; +import lombok.NonNull; 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.replay.datahandlers.PayloadAccessFaultingMap; import org.opensearch.migrations.replay.datahandlers.PayloadNotLoadedException; -import org.opensearch.migrations.replay.datatypes.UniqueReplayerRequestKey; +import org.opensearch.migrations.replay.tracing.IReplayContexts; import org.opensearch.migrations.transform.IAuthTransformer; import org.opensearch.migrations.transform.IJsonTransformer; -import java.util.List; import java.util.ArrayList; +import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -27,36 +28,31 @@ public class NettyDecodedHttpRequestPreliminaryConvertHandler extends Channel final IJsonTransformer transformer; final List> chunkSizes; final String diagnosticLabel; - private UniqueReplayerRequestKey requestKeyForMetricsLogging; + private final IReplayContexts.IRequestTransformationContext httpTransactionContext; static final MetricsLogger metricsLogger = new MetricsLogger("NettyDecodedHttpRequestPreliminaryConvertHandler"); public NettyDecodedHttpRequestPreliminaryConvertHandler(IJsonTransformer transformer, List> chunkSizes, RequestPipelineOrchestrator requestPipelineOrchestrator, - String diagnosticLabel, - UniqueReplayerRequestKey requestKeyForMetricsLogging) { + IReplayContexts.IRequestTransformationContext httpTransactionContext) { this.transformer = transformer; this.chunkSizes = chunkSizes; this.requestPipelineOrchestrator = requestPipelineOrchestrator; - this.diagnosticLabel = "[" + diagnosticLabel + "] "; - this.requestKeyForMetricsLogging = requestKeyForMetricsLogging; + this.diagnosticLabel = "[" + httpTransactionContext + "] "; + this.httpTransactionContext = httpTransactionContext; } @Override - public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + public void channelRead(@NonNull ChannelHandlerContext ctx, @NonNull Object msg) throws Exception { if (msg instanceof HttpRequest) { + httpTransactionContext.onHeaderParse(); var request = (HttpRequest) msg; - log.info(new StringBuilder(diagnosticLabel) - .append(" parsed request: ") - .append(request.method()) - .append(" ") - .append(request.uri()) - .append(" ") - .append(request.protocolVersion().text()) - .toString()); + log.atInfo().setMessage(()-> diagnosticLabel + " parsed request: " + + request.method() + " " + request.uri() + " " + request.protocolVersion().text()).log(); metricsLogger.atSuccess(MetricsEvent.CAPTURED_REQUEST_PARSED_TO_HTTP) - .setAttribute(MetricsAttributeKey.REQUEST_ID, requestKeyForMetricsLogging) - .setAttribute(MetricsAttributeKey.CONNECTION_ID, requestKeyForMetricsLogging.getTrafficStreamKey().getConnectionId()) + .setAttribute(MetricsAttributeKey.REQUEST_ID, httpTransactionContext) + .setAttribute(MetricsAttributeKey.CONNECTION_ID, + httpTransactionContext.getLogicalEnclosingScope().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/NettyJsonBodyAccumulateHandler.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettyJsonBodyAccumulateHandler.java index be256a0d6..7ee13a88f 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettyJsonBodyAccumulateHandler.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettyJsonBodyAccumulateHandler.java @@ -6,6 +6,7 @@ import io.netty.handler.codec.http.LastHttpContent; import lombok.SneakyThrows; import org.opensearch.migrations.replay.datahandlers.JsonAccumulator; +import org.opensearch.migrations.replay.tracing.IReplayContexts; import org.opensearch.migrations.transform.JsonKeysForHttpMessage; /** @@ -18,13 +19,16 @@ */ public class NettyJsonBodyAccumulateHandler extends ChannelInboundHandlerAdapter { + private final IReplayContexts.IRequestTransformationContext context; + public static class IncompleteJsonBodyException extends NoContentException {} JsonAccumulator jsonAccumulator; HttpJsonMessageWithFaultingPayload capturedHttpJsonMessage; @SneakyThrows - public NettyJsonBodyAccumulateHandler() { + public NettyJsonBodyAccumulateHandler(IReplayContexts.IRequestTransformationContext context) { + this.context = context; this.jsonAccumulator = new JsonAccumulator(); } @@ -36,6 +40,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception var jsonObject = jsonAccumulator.consumeByteBuffer(((HttpContent)msg).content().nioBuffer()); if (jsonObject != null) { capturedHttpJsonMessage.payload().put(JsonKeysForHttpMessage.INLINED_JSON_BODY_DOCUMENT_KEY, jsonObject); + context.onJsonPayloadParseSucceeded(); ctx.fireChannelRead(capturedHttpJsonMessage); } else if (msg instanceof LastHttpContent) { throw new IncompleteJsonBodyException(); 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..0d7fd432e 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.IReplayContexts; 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; + IReplayContexts.IReplayerHttpTransactionContext httpTransactionContext; public NettySendByteBufsToPacketHandlerHandler(IPacketFinalizingConsumer packetReceiver, - String diagnosticLabel) { + IReplayContexts.IReplayerHttpTransactionContext httpTransactionContext) { this.packetReceiver = packetReceiver; this.packetReceiverCompletionFutureRef = new AtomicReference<>(); - this.diagnosticLabel = diagnosticLabel; + this.httpTransactionContext = httpTransactionContext; currentFuture = DiagnosticTrackableCompletableFuture.Factory.completedFuture(null, - ()->"currentFuture for NettySendByteBufsToPacketHandlerHandler initialized to the base case for " + diagnosticLabel); + ()->"currentFuture for NettySendByteBufsToPacketHandlerHandler initialized to the base case for " + httpTransactionContext); } @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..3655b57f4 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,9 @@ 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.datahandlers.http.helpers.LastHttpContentListener; +import org.opensearch.migrations.replay.datahandlers.http.helpers.ReadMeteringHandler; +import org.opensearch.migrations.replay.tracing.IReplayContexts; import org.opensearch.migrations.transform.IAuthTransformer; import org.opensearch.migrations.transform.IAuthTransformerFactory; import org.opensearch.migrations.transform.IJsonTransformer; @@ -42,22 +44,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 final IReplayContexts.IRequestTransformationContext httpTransactionContext; @Getter final IAuthTransformerFactory authTransfomerFactory; public RequestPipelineOrchestrator(List> chunkSizes, IPacketFinalizingConsumer packetReceiver, IAuthTransformerFactory incomingAuthTransformerFactory, - String diagnosticLabel, - UniqueReplayerRequestKey requestKeyForMetricsLogging) { + IReplayContexts.IRequestTransformationContext httpTransactionContext) { this.chunkSizes = chunkSizes; this.packetReceiver = packetReceiver; this.authTransfomerFactory = incomingAuthTransformerFactory != null ? incomingAuthTransformerFactory : IAuthTransformerFactory.NullAuthTransformerFactory.instance; - this.diagnosticLabel = diagnosticLabel; - this.requestKeyForMetricsLogging = requestKeyForMetricsLogging; + this.httpTransactionContext = httpTransactionContext; } static void removeThisAndPreviousHandlers(ChannelPipeline pipeline, ChannelHandler targetHandler) { @@ -88,6 +87,7 @@ void addJsonParsingHandlers(ChannelHandlerContext ctx, void addInitialHandlers(ChannelPipeline pipeline, IJsonTransformer transformer) { pipeline.addFirst(HTTP_REQUEST_DECODER_NAME, new HttpRequestDecoder()); addLoggingHandler(pipeline, "A"); + pipeline.addLast(new ReadMeteringHandler(httpTransactionContext::aggregateInputChunk)); // IN: Netty HttpRequest(1) + HttpContent(1) blocks (which may be compressed) + EndOfInput + ByteBuf // OUT: ByteBufs(1) OR Netty HttpRequest(1) + HttpJsonMessage(1) with only headers PLUS + HttpContent(1) blocks // Note1: original Netty headers are preserved so that HttpContentDecompressor can work appropriately. @@ -99,25 +99,29 @@ 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, httpTransactionContext)); addLoggingHandler(pipeline, "B"); } void addContentParsingHandlers(ChannelHandlerContext ctx, IJsonTransformer transformer, IAuthTransformer.StreamingFullMessageTransformer authTransfomer) { + httpTransactionContext.onPayloadParse(); log.debug("Adding content parsing handlers to pipeline"); var pipeline = ctx.pipeline(); + pipeline.addLast(new ReadMeteringHandler(httpTransactionContext::onPayloadBytesIn)); // IN: Netty HttpRequest(1) + HttpJsonMessage(1) with headers + HttpContent(1) blocks (which may be compressed) // OUT: Netty HttpRequest(2) + HttpJsonMessage(1) with headers + HttpContent(2) uncompressed blocks pipeline.addLast(new HttpContentDecompressor()); + pipeline.addLast(new ReadMeteringHandler(httpTransactionContext::onUncompressedBytesIn)); if (transformer != null) { + httpTransactionContext.onJsonPayloadParseRequired(); log.debug("Adding JSON handlers to pipeline"); // IN: Netty HttpRequest(2) + HttpJsonMessage(1) with headers + HttpContent(2) blocks // OUT: Netty HttpRequest(2) + HttpJsonMessage(2) with headers AND payload addLoggingHandler(pipeline, "C"); - pipeline.addLast(new NettyJsonBodyAccumulateHandler()); + pipeline.addLast(new NettyJsonBodyAccumulateHandler(httpTransactionContext)); // IN: Netty HttpRequest(2) + HttpJsonMessage(2) with headers AND payload // OUT: Netty HttpRequest(2) + HttpJsonMessage(3) with headers AND payload (transformed) pipeline.addLast(new NettyJsonBodyConvertHandler(transformer)); @@ -130,9 +134,12 @@ void addContentParsingHandlers(ChannelHandlerContext ctx, pipeline.addLast(new NettyJsonContentAuthSigner(authTransfomer)); addLoggingHandler(pipeline, "G"); } + pipeline.addLast(new LastHttpContentListener(httpTransactionContext::onPayloadParseSuccess)); + pipeline.addLast(new ReadMeteringHandler(httpTransactionContext::onUncompressedBytesOut)); // IN: Netty HttpRequest(2) + HttpJsonMessage(3) with headers only + HttpContent(3) blocks // OUT: Netty HttpRequest(3) + HttpJsonMessage(4) with headers only + HttpContent(4) blocks pipeline.addLast(new NettyJsonContentCompressor()); + pipeline.addLast(new ReadMeteringHandler(httpTransactionContext::onFinalBytesOut)); addLoggingHandler(pipeline, "H"); // IN: Netty HttpRequest(3) + HttpJsonMessage(4) with headers only + HttpContent(4) blocks + EndOfInput // OUT: Netty HttpRequest(3) + HttpJsonMessage(4) with headers only + ByteBufs(2) @@ -146,11 +153,13 @@ void addBaselineHandlers(ChannelPipeline pipeline) { // IN: ByteBufs(2) + HttpJsonMessage(4) with headers only + HttpContent(1) (if the repackaging handlers were skipped) // OUT: ByteBufs(3) which are sized similarly to how they were received pipeline.addLast(new NettyJsonToByteBufHandler(Collections.unmodifiableList(chunkSizes))); + pipeline.addLast(new ReadMeteringHandler(httpTransactionContext::aggregateOutputChunk)); // IN: ByteBufs(3) // 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, + httpTransactionContext.getLogicalEnclosingScope())); } private void addLoggingHandler(ChannelPipeline pipeline, String name) { diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/helpers/LastHttpContentListener.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/helpers/LastHttpContentListener.java new file mode 100644 index 000000000..d80dda43a --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/helpers/LastHttpContentListener.java @@ -0,0 +1,22 @@ +package org.opensearch.migrations.replay.datahandlers.http.helpers; + +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; +import io.netty.handler.codec.http.LastHttpContent; + +public class LastHttpContentListener extends ChannelInboundHandlerAdapter { + + private final Runnable onLastContentReceived; + + public LastHttpContentListener(Runnable onLastContentReceived) { + this.onLastContentReceived = onLastContentReceived; + } + + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + if (msg instanceof LastHttpContent) { + onLastContentReceived.run(); + } + super.channelRead(ctx, msg); + } +} diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/helpers/ReadMeteringHandler.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/helpers/ReadMeteringHandler.java new file mode 100644 index 000000000..ea81036fe --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/helpers/ReadMeteringHandler.java @@ -0,0 +1,24 @@ +package org.opensearch.migrations.replay.datahandlers.http.helpers; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; +import io.netty.handler.codec.http.HttpContent; +import lombok.AllArgsConstructor; + +import java.util.function.IntConsumer; + +@AllArgsConstructor +public class ReadMeteringHandler extends ChannelInboundHandlerAdapter { + private final IntConsumer sizeConsumer; + + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + if (msg instanceof ByteBuf) { + sizeConsumer.accept(((ByteBuf) msg).readableBytes()); + } else if (msg instanceof HttpContent) { + sizeConsumer.accept(((HttpContent) msg).content().readableBytes()); + } + super.channelRead(ctx, msg); + } +} diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/helpers/WriteMeteringHandler.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/helpers/WriteMeteringHandler.java new file mode 100644 index 000000000..7fdfc9ded --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/helpers/WriteMeteringHandler.java @@ -0,0 +1,27 @@ +package org.opensearch.migrations.replay.datahandlers.http.helpers; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelOutboundHandlerAdapter; +import io.netty.channel.ChannelPromise; +import io.netty.handler.codec.http.HttpContent; + +import java.util.function.IntConsumer; + +public class WriteMeteringHandler extends ChannelOutboundHandlerAdapter { + final IntConsumer sizeConsumer; + + public WriteMeteringHandler(IntConsumer sizeConsumer) { + this.sizeConsumer = sizeConsumer; + } + + @Override + public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) throws Exception { + if (msg instanceof ByteBuf) { + sizeConsumer.accept(((ByteBuf) msg).readableBytes()); + } else if (msg instanceof HttpContent) { + sizeConsumer.accept(((HttpContent) msg).content().readableBytes()); + } + super.write(ctx, msg, promise); + } +} 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..557452fba 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,6 +6,7 @@ import lombok.Setter; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; +import org.opensearch.migrations.replay.tracing.IReplayContexts; import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; import org.opensearch.migrations.replay.util.OnlineRadixSorter; @@ -18,6 +19,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 @@ -25,19 +27,21 @@ public class ConnectionReplaySession { * EventLoop so that we can route all calls for this object into that loop/thread. */ public final EventLoop eventLoop; - @Getter @Setter - private DiagnosticTrackableCompletableFuture channelFutureFuture; + @Getter + @Setter + private DiagnosticTrackableCompletableFuture channelFutureFuture; public final OnlineRadixSorter scheduleSequencer; public final TimeToResponseFulfillmentFutureMap schedule; @Getter @Setter - private ISourceTrafficChannelKey channelId; + private final IReplayContexts.ISocketContext socketContext; - public ConnectionReplaySession(EventLoop eventLoop) { + public ConnectionReplaySession(EventLoop eventLoop, IReplayContexts.IChannelKeyContext channelKeyContext) { this.eventLoop = eventLoop; this.scheduleSequencer = new OnlineRadixSorter<>(0); this.schedule = new TimeToResponseFulfillmentFutureMap(); + this.socketContext = channelKeyContext.createSocketContext(); } @SneakyThrows diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/ISourceTrafficChannelKey.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/ISourceTrafficChannelKey.java index 3a0eb4644..3ca8482c4 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/ISourceTrafficChannelKey.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/ISourceTrafficChannelKey.java @@ -1,7 +1,17 @@ package org.opensearch.migrations.replay.datatypes; +import lombok.AllArgsConstructor; +import lombok.Getter; + public interface ISourceTrafficChannelKey { String getNodeId(); String getConnectionId(); + + @Getter + @AllArgsConstructor + class PojoImpl implements ISourceTrafficChannelKey { + String nodeId; + String connectionId; + } } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/ITrafficStreamKey.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/ITrafficStreamKey.java index ab9d6ced4..0b3ea72db 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/ITrafficStreamKey.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/ITrafficStreamKey.java @@ -1,5 +1,10 @@ package org.opensearch.migrations.replay.datatypes; +import lombok.NonNull; +import org.opensearch.migrations.replay.tracing.IReplayContexts; + public interface ITrafficStreamKey extends ISourceTrafficChannelKey { int getTrafficStreamIndex(); + + @NonNull IReplayContexts.ITrafficStreamsLifecycleContext getTrafficStreamsContext(); } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/PojoTrafficStreamWithKey.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/PojoTrafficStreamAndKey.java similarity index 83% rename from TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/PojoTrafficStreamWithKey.java rename to TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/PojoTrafficStreamAndKey.java index 62837c9eb..4387f46a6 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/PojoTrafficStreamWithKey.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/PojoTrafficStreamAndKey.java @@ -8,7 +8,7 @@ @AllArgsConstructor @Getter -public class PojoTrafficStreamWithKey implements ITrafficStreamWithKey { +public class PojoTrafficStreamAndKey implements ITrafficStreamWithKey { public final TrafficStream stream; public final ITrafficStreamKey key; } \ No newline at end of file diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/PojoTrafficStreamKey.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/PojoTrafficStreamKey.java index 564b6dbf5..a9d193fc7 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/PojoTrafficStreamKey.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/PojoTrafficStreamKey.java @@ -1,25 +1,20 @@ package org.opensearch.migrations.replay.datatypes; -import java.util.StringJoiner; - import lombok.EqualsAndHashCode; -import org.opensearch.migrations.trafficcapture.protos.TrafficStream; -import org.opensearch.migrations.trafficcapture.protos.TrafficStreamUtils; +import org.opensearch.migrations.replay.util.TrafficChannelKeyFormatter; @EqualsAndHashCode() -public class PojoTrafficStreamKey implements ITrafficStreamKey { - private final String nodeId; - private final String connectionId; - private final int trafficStreamIndex; +public abstract class PojoTrafficStreamKey extends ISourceTrafficChannelKey.PojoImpl + implements ITrafficStreamKey { + protected final int trafficStreamIndex; - public PojoTrafficStreamKey(TrafficStream stream) { - this(stream.getNodeId(), stream.getConnectionId(), TrafficStreamUtils.getTrafficStreamIndex(stream)); + protected PojoTrafficStreamKey(String nodeId, String connectionId, int index) { + super(nodeId, connectionId); + this.trafficStreamIndex = index; } - public PojoTrafficStreamKey(String nodeId, String connectionId, int index) { - this.nodeId = nodeId; - this.connectionId = connectionId; - this.trafficStreamIndex = index; + protected PojoTrafficStreamKey(PojoImpl tsk, int index) { + this(tsk.nodeId, tsk.connectionId, index); } @Override @@ -39,10 +34,6 @@ public int getTrafficStreamIndex() { @Override public String toString() { - return new StringJoiner(".") - .add(nodeId) - .add(connectionId) - .add(""+trafficStreamIndex) - .toString(); + return TrafficChannelKeyFormatter.format(nodeId, connectionId, trafficStreamIndex); } } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/PojoTrafficStreamKeyAndContext.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/PojoTrafficStreamKeyAndContext.java new file mode 100644 index 000000000..511ab2a0f --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/PojoTrafficStreamKeyAndContext.java @@ -0,0 +1,53 @@ +package org.opensearch.migrations.replay.datatypes; + +import lombok.EqualsAndHashCode; +import lombok.NonNull; +import lombok.Setter; +import org.opensearch.migrations.replay.tracing.IReplayContexts; +import org.opensearch.migrations.trafficcapture.protos.TrafficStream; +import org.opensearch.migrations.trafficcapture.protos.TrafficStreamUtils; + +import java.util.function.Function; + + +@EqualsAndHashCode(callSuper = true) +public class PojoTrafficStreamKeyAndContext extends PojoTrafficStreamKey { + @Setter + IReplayContexts.ITrafficStreamsLifecycleContext trafficStreamsContext; + + public static PojoTrafficStreamKeyAndContext + build(TrafficStream stream, + Function contextSupplier) { + var rval = new PojoTrafficStreamKeyAndContext(stream.getNodeId(), stream.getConnectionId(), + TrafficStreamUtils.getTrafficStreamIndex(stream)); + rval.setTrafficStreamsContext(contextSupplier.apply(rval)); + return rval; + } + + public static PojoTrafficStreamKeyAndContext + build(ISourceTrafficChannelKey sourceKey, int index, + Function contextSupplier) { + return build(sourceKey.getNodeId(), sourceKey.getConnectionId(), index, contextSupplier); + } + + public static PojoTrafficStreamKeyAndContext + build(String nodeId, String connectionId, int index, + Function contextSupplier) { + var rval = new PojoTrafficStreamKeyAndContext(nodeId, connectionId, index); + rval.setTrafficStreamsContext(contextSupplier.apply(rval)); + return rval; + } + + protected PojoTrafficStreamKeyAndContext(TrafficStream stream) { + this(stream.getNodeId(), stream.getConnectionId(), TrafficStreamUtils.getTrafficStreamIndex(stream)); + } + + private PojoTrafficStreamKeyAndContext(String nodeId, String connectionId, int index) { + super(nodeId, connectionId, index); + } + + @NonNull + public IReplayContexts.ITrafficStreamsLifecycleContext getTrafficStreamsContext() { + return trafficStreamsContext; + } +} diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/UniqueReplayerRequestKey.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/UniqueReplayerRequestKey.java index d4cc37b5f..1dd711cad 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/UniqueReplayerRequestKey.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/UniqueReplayerRequestKey.java @@ -1,14 +1,16 @@ package org.opensearch.migrations.replay.datatypes; import lombok.EqualsAndHashCode; +import lombok.Getter; @EqualsAndHashCode(callSuper = true) public class UniqueReplayerRequestKey extends UniqueSourceRequestKey { - public final ISourceTrafficChannelKey trafficStreamKey; + public final ITrafficStreamKey trafficStreamKey; public final int sourceRequestIndexOffsetAtStartOfAccumulation; + @Getter public final int replayerRequestIndex; - public UniqueReplayerRequestKey(ISourceTrafficChannelKey streamKey, int sourceOffsetAtStartOfAccumulation, + public UniqueReplayerRequestKey(ITrafficStreamKey streamKey, int sourceOffsetAtStartOfAccumulation, int replayerIndex) { this.trafficStreamKey = streamKey; this.sourceRequestIndexOffsetAtStartOfAccumulation = sourceOffsetAtStartOfAccumulation; @@ -25,10 +27,6 @@ public int getSourceRequestIndex() { return replayerRequestIndex + sourceRequestIndexOffsetAtStartOfAccumulation; } - public int getReplayerRequestIndex() { - return replayerRequestIndex; - } - @Override public String toString() { // The offset that is shown is a mouthful to describe. diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSource.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSource.java index 267b29fb4..3689e33b2 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSource.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSource.java @@ -1,6 +1,7 @@ package org.opensearch.migrations.replay.kafka; import com.google.protobuf.InvalidProtocolBufferException; +import io.netty.util.concurrent.DefaultThreadFactory; import lombok.NonNull; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; @@ -11,7 +12,11 @@ import org.opensearch.migrations.coreutils.MetricsEvent; import org.opensearch.migrations.coreutils.MetricsLogger; import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; -import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamWithKey; +import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamAndKey; +import org.opensearch.migrations.replay.tracing.ChannelContextManager; +import org.opensearch.migrations.replay.tracing.ITrafficSourceContexts; +import org.opensearch.migrations.replay.tracing.ReplayContexts; +import org.opensearch.migrations.replay.tracing.RootReplayerContext; import org.opensearch.migrations.replay.traffic.source.ISimpleTrafficCaptureSource; import org.opensearch.migrations.replay.traffic.source.ITrafficStreamWithKey; import org.opensearch.migrations.trafficcapture.protos.TrafficStream; @@ -31,9 +36,8 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Supplier; import java.util.stream.Collectors; -import java.util.stream.Stream; -import java.util.stream.StreamSupport; /** * Adapt a Kafka stream into a TrafficCaptureSource. @@ -63,7 +67,6 @@ */ @Slf4j public class KafkaTrafficCaptureSource implements ISimpleTrafficCaptureSource { - public static final String MAX_POLL_INTERVAL_KEY = "max.poll.interval.ms"; // see https://stackoverflow.com/questions/39730126/difference-between-session-timeout-ms-and-max-poll-interval-ms-for-kafka-0-10 public static final String DEFAULT_POLL_INTERVAL_MS = "60000"; @@ -74,38 +77,55 @@ public class KafkaTrafficCaptureSource implements ISimpleTrafficCaptureSource { private final ExecutorService kafkaExecutor; private final AtomicLong trafficStreamsRead; private final KafkaBehavioralPolicy behavioralPolicy; + private final ChannelContextManager channelContextManager; - public KafkaTrafficCaptureSource(Consumer kafkaConsumer, String topic, Duration keepAliveInterval) { - this(kafkaConsumer, topic, keepAliveInterval, Clock.systemUTC(), new KafkaBehavioralPolicy()); + public KafkaTrafficCaptureSource(@NonNull RootReplayerContext globalContext, + Consumer kafkaConsumer, String topic, Duration keepAliveInterval) { + this(globalContext, kafkaConsumer, topic, keepAliveInterval, Clock.systemUTC(), new KafkaBehavioralPolicy()); } - public KafkaTrafficCaptureSource(Consumer kafkaConsumer, + public KafkaTrafficCaptureSource(@NonNull RootReplayerContext globalContext, + Consumer kafkaConsumer, @NonNull String topic, Duration keepAliveInterval, Clock clock, @NonNull KafkaBehavioralPolicy behavioralPolicy) { - trackingKafkaConsumer = new TrackingKafkaConsumer(kafkaConsumer, topic, keepAliveInterval, clock); + this.channelContextManager = new ChannelContextManager(globalContext); + trackingKafkaConsumer = new TrackingKafkaConsumer(globalContext, kafkaConsumer, topic, keepAliveInterval, clock, + this::onKeyFinishedCommitting); trafficStreamsRead = new AtomicLong(); this.behavioralPolicy = behavioralPolicy; kafkaConsumer.subscribe(Collections.singleton(topic), trackingKafkaConsumer); - kafkaExecutor = Executors.newSingleThreadExecutor(); + kafkaExecutor = Executors.newSingleThreadExecutor(new DefaultThreadFactory("kafkaConsumerThread")); + } + + private void onKeyFinishedCommitting(ITrafficStreamKey trafficStreamKey) { + var looseParentScope = trafficStreamKey.getTrafficStreamsContext().getEnclosingScope(); + if (!(looseParentScope instanceof ReplayContexts.KafkaRecordContext)) { + throw new IllegalArgumentException("Expected parent context of type " + ReplayContexts.KafkaRecordContext.class + + " instead of " + looseParentScope + " (of type=" + looseParentScope.getClass() + ")"); + } + var kafkaCtx = (ReplayContexts.KafkaRecordContext) looseParentScope; + kafkaCtx.close(); + channelContextManager.releaseContextFor(kafkaCtx.getImmediateEnclosingScope()); } - public static KafkaTrafficCaptureSource buildKafkaConsumer(@NonNull String brokers, - @NonNull String topic, - @NonNull String groupId, - boolean enableMSKAuth, - String propertyFilePath, - @NonNull Clock clock, - @NonNull KafkaBehavioralPolicy behavioralPolicy) + public static KafkaTrafficCaptureSource buildKafkaSource(@NonNull RootReplayerContext globalContext, + @NonNull String brokers, + @NonNull String topic, + @NonNull String groupId, + boolean enableMSKAuth, + String propertyFilePath, + @NonNull Clock clock, + @NonNull KafkaBehavioralPolicy behavioralPolicy) throws IOException { var kafkaProps = buildKafkaProperties(brokers, groupId, enableMSKAuth, propertyFilePath); kafkaProps.putIfAbsent(MAX_POLL_INTERVAL_KEY, DEFAULT_POLL_INTERVAL_MS); var pollPeriod = Duration.ofMillis(Long.valueOf((String)kafkaProps.get(MAX_POLL_INTERVAL_KEY))); var keepAlivePeriod = getKeepAlivePeriodFromPollPeriod(pollPeriod); - return new KafkaTrafficCaptureSource(new KafkaConsumer<>(kafkaProps), + return new KafkaTrafficCaptureSource(globalContext, new KafkaConsumer<>(kafkaProps), topic, keepAlivePeriod, clock, behavioralPolicy); } @@ -151,8 +171,8 @@ public static Properties buildKafkaProperties(@NonNull String brokers, @Override @SneakyThrows - public void touch() { - CompletableFuture.runAsync(()->trackingKafkaConsumer.touch(), kafkaExecutor).get(); + public void touch(ITrafficSourceContexts.IBackPressureBlockContext context) { + CompletableFuture.runAsync(()->trackingKafkaConsumer.touch(context), kafkaExecutor).get(); } /** @@ -167,18 +187,20 @@ public Optional getNextRequiredTouch() { @Override @SuppressWarnings("unchecked") - public CompletableFuture> readNextTrafficStreamChunk() { + public CompletableFuture> + readNextTrafficStreamChunk(Supplier contextSupplier) { log.atTrace().setMessage("readNextTrafficStreamChunk()").log(); return CompletableFuture.supplyAsync(() -> { log.atTrace().setMessage("async...readNextTrafficStreamChunk()").log(); - return readNextTrafficStreamSynchronously(); + return readNextTrafficStreamSynchronously(contextSupplier.get()); }, kafkaExecutor); } - public List readNextTrafficStreamSynchronously() { + public List + readNextTrafficStreamSynchronously(ITrafficSourceContexts.IReadChunkContext context) { log.atTrace().setMessage("readNextTrafficStreamSynchronously()").log(); try { - return trackingKafkaConsumer.getNextBatchOfRecords((offsetData,kafkaRecord) -> { + return trackingKafkaConsumer.getNextBatchOfRecords(context, (offsetData,kafkaRecord) -> { try { TrafficStream ts = TrafficStream.parseFrom(kafkaRecord.value()); // Ensure we increment trafficStreamsRead even at a higher log level @@ -190,8 +212,15 @@ public List readNextTrafficStreamSynchronously() { var trafficStreamsSoFar = trafficStreamsRead.incrementAndGet(); log.atTrace().setMessage(()->"Parsed traffic stream #" + trafficStreamsSoFar + ": " + offsetData + " " + ts).log(); - var key = new TrafficStreamKeyWithKafkaRecordId(ts, offsetData); - return (ITrafficStreamWithKey) new PojoTrafficStreamWithKey(ts, key); + var key = new TrafficStreamKeyWithKafkaRecordId( + tsk -> { + var channelKeyCtx = channelContextManager.retainOrCreateContext(tsk); + return channelContextManager.getGlobalContext() + .createTrafficStreamContextForKafkaSource(channelKeyCtx, kafkaRecord.key(), + kafkaRecord.serializedKeySize() + kafkaRecord.serializedValueSize()); + }, + ts, offsetData); + return (ITrafficStreamWithKey) new PojoTrafficStreamAndKey(ts, key); } catch (InvalidProtocolBufferException e) { RuntimeException recordError = behavioralPolicy.onInvalidKafkaRecord(kafkaRecord, e); metricsLogger.atError(MetricsEvent.PARSING_TRAFFIC_STREAM_FROM_KAFKA_FAILED, recordError) @@ -209,12 +238,12 @@ public List readNextTrafficStreamSynchronously() { } @Override - public void commitTrafficStream(ITrafficStreamKey trafficStreamKey) { + public CommitResult commitTrafficStream(ITrafficStreamKey trafficStreamKey) { if (!(trafficStreamKey instanceof TrafficStreamKeyWithKafkaRecordId)) { throw new IllegalArgumentException("Expected key of type "+TrafficStreamKeyWithKafkaRecordId.class+ " but received "+trafficStreamKey+" (of type="+trafficStreamKey.getClass()+")"); } - trackingKafkaConsumer.commitKafkaKey((TrafficStreamKeyWithKafkaRecordId) trafficStreamKey); + return trackingKafkaConsumer.commitKafkaKey(trafficStreamKey, (TrafficStreamKeyWithKafkaRecordId) trafficStreamKey); } @Override diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/TrackingKafkaConsumer.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/TrackingKafkaConsumer.java index 418a544be..f1fae148a 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/TrackingKafkaConsumer.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/TrackingKafkaConsumer.java @@ -1,6 +1,8 @@ package org.opensearch.migrations.replay.kafka; -import com.google.protobuf.InvalidProtocolBufferException; +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.NonNull; import lombok.extern.slf4j.Slf4j; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; @@ -8,12 +10,12 @@ import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.requests.TxnOffsetCommitRequest; -import org.opensearch.migrations.coreutils.MetricsAttributeKey; -import org.opensearch.migrations.coreutils.MetricsEvent; -import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamWithKey; -import org.opensearch.migrations.replay.traffic.source.ITrafficStreamWithKey; -import org.opensearch.migrations.trafficcapture.protos.TrafficStream; +import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; +import org.opensearch.migrations.replay.tracing.IKafkaConsumerContexts; +import org.opensearch.migrations.replay.tracing.ITrafficSourceContexts; +import org.opensearch.migrations.replay.tracing.KafkaConsumerContexts; +import org.opensearch.migrations.replay.tracing.RootReplayerContext; +import org.opensearch.migrations.replay.traffic.source.ITrafficCaptureSource; import org.slf4j.event.Level; import java.time.Clock; @@ -23,12 +25,13 @@ import java.util.Collections; import java.util.HashMap; import java.util.Map; -import java.util.Objects; import java.util.Optional; +import java.util.PriorityQueue; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiFunction; -import java.util.function.Function; +import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.Stream; import java.util.stream.StreamSupport; @@ -41,6 +44,32 @@ */ @Slf4j public class TrackingKafkaConsumer implements ConsumerRebalanceListener { + @AllArgsConstructor + private static class OrderedKeyHolder implements Comparable { + @Getter final long offset; + @Getter @NonNull final ITrafficStreamKey tsk; + + @Override + public int compareTo(OrderedKeyHolder o) { + return Long.compare(offset, o.offset); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + OrderedKeyHolder that = (OrderedKeyHolder) o; + + if (offset != that.offset) return false; + return tsk.equals(that.tsk); + } + + @Override + public int hashCode() { + return Long.valueOf(offset).hashCode(); + } + } /** * The keep-alive should already be set to a fraction of the max poll timeout for @@ -52,6 +81,8 @@ public class TrackingKafkaConsumer implements ConsumerRebalanceListener { * which happens after we poll() (on the same thread, as per Consumer requirements). */ public static final int POLL_TIMEOUT_KEEP_ALIVE_DIVISOR = 4; + + @NonNull private final RootReplayerContext globalContext; private final Consumer kafkaConsumer; final String topic; @@ -63,45 +94,64 @@ public class TrackingKafkaConsumer implements ConsumerRebalanceListener { * the generations of each OffsetLifecycleTracker value may be different. */ final Map partitionToOffsetLifecycleTrackerMap; + private final Object commitDataLock = new Object(); // loosening visibility so that a unit test can read this final Map nextSetOfCommitsMap; + final Map> nextSetOfKeysContextsBeingCommitted; + final java.util.function.Consumer onCommitKeyCallback; private final Duration keepAliveInterval; private final AtomicReference lastTouchTimeRef; - private AtomicInteger consumerConnectionGeneration; - private AtomicInteger kafkaRecordsLeftToCommit; + private final AtomicInteger consumerConnectionGeneration; + private final AtomicInteger kafkaRecordsLeftToCommitEventually; + private final AtomicBoolean kafkaRecordsReadyToCommit; - public TrackingKafkaConsumer(Consumer kafkaConsumer, String topic, - Duration keepAliveInterval, Clock c) { + public TrackingKafkaConsumer(@NonNull RootReplayerContext globalContext, + Consumer kafkaConsumer, String topic, + Duration keepAliveInterval, Clock c, + java.util.function.Consumer onCommitKeyCallback) { + this.globalContext = globalContext; this.kafkaConsumer = kafkaConsumer; this.topic = topic; this.clock = c; this.partitionToOffsetLifecycleTrackerMap = new HashMap<>(); this.nextSetOfCommitsMap = new HashMap<>(); + this.nextSetOfKeysContextsBeingCommitted = new HashMap<>(); this.lastTouchTimeRef = new AtomicReference<>(Instant.EPOCH); consumerConnectionGeneration = new AtomicInteger(); - kafkaRecordsLeftToCommit = new AtomicInteger(); + kafkaRecordsLeftToCommitEventually = new AtomicInteger(); + kafkaRecordsReadyToCommit = new AtomicBoolean(); this.keepAliveInterval = keepAliveInterval; + this.onCommitKeyCallback = onCommitKeyCallback; } @Override public void onPartitionsRevoked(Collection partitions) { - safeCommit(); - partitions.forEach(p->{ - nextSetOfCommitsMap.remove(new TopicPartition(topic, p.partition())); - partitionToOffsetLifecycleTrackerMap.remove(p.partition()); - }); - kafkaRecordsLeftToCommit.set(partitionToOffsetLifecycleTrackerMap.values().stream() - .mapToInt(OffsetLifecycleTracker::size).sum()); - log.atWarn().setMessage(()->this+"partitions revoked for "+partitions.stream() - .map(p->p+"").collect(Collectors.joining(","))).log(); + new KafkaConsumerContexts.AsyncListeningContext(globalContext).onPartitionsRevoked(partitions); + synchronized (commitDataLock) { + safeCommit(globalContext::createCommitContext); + partitions.forEach(p -> { + var tp = new TopicPartition(topic, p.partition()); + nextSetOfCommitsMap.remove(tp); + nextSetOfKeysContextsBeingCommitted.remove(tp); + partitionToOffsetLifecycleTrackerMap.remove(p.partition()); + }); + kafkaRecordsLeftToCommitEventually.set(partitionToOffsetLifecycleTrackerMap.values().stream() + .mapToInt(OffsetLifecycleTracker::size).sum()); + kafkaRecordsReadyToCommit.set(!nextSetOfCommitsMap.values().isEmpty()); + log.atWarn().setMessage(() -> this + "partitions revoked for " + partitions.stream() + .map(p -> p + "").collect(Collectors.joining(","))).log(); + } } @Override public void onPartitionsAssigned(Collection newPartitions) { - consumerConnectionGeneration.incrementAndGet(); - newPartitions.forEach(p->partitionToOffsetLifecycleTrackerMap.computeIfAbsent(p.partition(), - x->new OffsetLifecycleTracker(consumerConnectionGeneration.get()))); - log.atWarn().setMessage(()->this+"partitions added for "+newPartitions.stream() - .map(p->p+"").collect(Collectors.joining(","))).log(); + new KafkaConsumerContexts.AsyncListeningContext(globalContext).onPartitionsAssigned(newPartitions); + synchronized (commitDataLock) { + consumerConnectionGeneration.incrementAndGet(); + newPartitions.forEach(p -> partitionToOffsetLifecycleTrackerMap.computeIfAbsent(p.partition(), + x -> new OffsetLifecycleTracker(consumerConnectionGeneration.get()))); + log.atWarn().setMessage(() -> this + "partitions added for " + newPartitions.stream() + .map(p -> p + "").collect(Collectors.joining(","))).log(); + } } public void close() { @@ -112,32 +162,34 @@ public void close() { public Optional getNextRequiredTouch() { var lastTouchTime = lastTouchTimeRef.get(); - var r = kafkaRecordsLeftToCommit.get() == 0 ? Optional.empty() - : Optional.of(lastTouchTime.plus(keepAliveInterval)); + var r = kafkaRecordsLeftToCommitEventually.get() == 0 ? Optional.empty() + : Optional.of(kafkaRecordsReadyToCommit.get() ? Instant.now() : lastTouchTime.plus(keepAliveInterval)); log.atTrace().setMessage(()->"returning next required touch at " + r.map(t->""+t).orElse("N/A") + " from a lastTouchTime of "+lastTouchTime).log(); return r; } - public void touch() { - log.trace("touch() called."); - pause(); - try { - var records = kafkaConsumer.poll(Duration.ZERO); - if (!records.isEmpty()) { - throw new IllegalStateException("Expected no entries once the consumer was paused. " + - "This may have happened because a new assignment slipped into the consumer AFTER pause calls."); + public void touch(ITrafficSourceContexts.IBackPressureBlockContext context) { + try (var touchCtx = context.createNewTouchContext()) { + log.trace("touch() called."); + pause(); + try (var pollCtx = touchCtx.createNewPollContext()) { + var records = kafkaConsumer.poll(Duration.ZERO); + if (!records.isEmpty()) { + throw new IllegalStateException("Expected no entries once the consumer was paused. " + + "This may have happened because a new assignment slipped into the consumer AFTER pause calls."); + } + } catch (IllegalStateException e) { + throw e; + } catch (RuntimeException e) { + log.atWarn().setCause(e).setMessage("Unable to poll the topic: " + topic + " with our Kafka consumer. " + + "Swallowing and awaiting next metadata refresh to try again.").log(); + } finally { + resume(); } - } catch (IllegalStateException e) { - throw e; - } catch (RuntimeException e) { - log.atWarn().setCause(e).setMessage("Unable to poll the topic: " + topic + " with our Kafka consumer. " + - "Swallowing and awaiting next metadata refresh to try again.").log(); - } finally { - resume(); + safeCommit(()->context.createCommitContext()); + lastTouchTimeRef.set(clock.instant()); } - safeCommit(); - lastTouchTimeRef.set(clock.instant()); } private void pause() { @@ -147,11 +199,11 @@ private void pause() { } catch (IllegalStateException e) { log.atError().setCause(e).setMessage(()->"Unable to pause the topic partitions: " + topic + ". " + "The active partitions passed here : " + activePartitions.stream() - .map(x->x.toString()).collect(Collectors.joining(",")) + ". " + + .map(x->""+x).collect(Collectors.joining(",")) + ". " + "The active partitions as tracked here are: " + getActivePartitions().stream() - .map(x->x.toString()).collect(Collectors.joining(",")) + ". " + + .map(x->""+x).collect(Collectors.joining(",")) + ". " + "The active partitions according to the consumer: " + kafkaConsumer.assignment().stream() - .map(x->x.toString()).collect(Collectors.joining(",")) + .map(x->""+x).collect(Collectors.joining(",")) ).log(); } } @@ -165,11 +217,11 @@ private void resume() { "This may not be a fatal error for the entire process as the consumer should eventually" + " rejoin and rebalance. " + "The active partitions passed here : " + activePartitions.stream() - .map(x->x.toString()).collect(Collectors.joining(",")) + ". " + + .map(x->""+x).collect(Collectors.joining(",")) + ". " + "The active partitions as tracked here are: " + getActivePartitions().stream() - .map(x->x.toString()).collect(Collectors.joining(",")) + ". " + + .map(x->""+x).collect(Collectors.joining(",")) + ". " + "The active partitions according to the consumer: " + kafkaConsumer.assignment().stream() - .map(x->x.toString()).collect(Collectors.joining(",")) + .map(x->""+x).collect(Collectors.joining(",")) ).log(); } } @@ -180,9 +232,11 @@ private Collection getActivePartitions() { } public Stream - getNextBatchOfRecords(BiFunction,T> builder) { - var records = safePollWithSwallowedRuntimeExceptions(); - safeCommit(); + getNextBatchOfRecords(ITrafficSourceContexts.IReadChunkContext context, + BiFunction, T> builder) { + safeCommit(()->context.createCommitContext()); + var records = safePollWithSwallowedRuntimeExceptions(context); + safeCommit(()->context.createCommitContext()); return applyBuilder(builder, records); } @@ -194,19 +248,23 @@ private Stream applyBuilder(BiFunction"records in flight="+kafkaRecordsLeftToCommit.get()).log(); + kafkaRecordsLeftToCommitEventually.incrementAndGet(); + log.atTrace().setMessage(()->"records in flight="+ kafkaRecordsLeftToCommitEventually.get()).log(); return builder.apply(offsetDetails, kafkaRecord); }); } - private ConsumerRecords safePollWithSwallowedRuntimeExceptions() { + private ConsumerRecords + safePollWithSwallowedRuntimeExceptions(ITrafficSourceContexts.IReadChunkContext context) { try { lastTouchTimeRef.set(clock.instant()); - var records = kafkaConsumer.poll(keepAliveInterval.dividedBy(POLL_TIMEOUT_KEEP_ALIVE_DIVISOR)); + ConsumerRecords records; + try (var pollContext = context.createPollContext()) { + records = kafkaConsumer.poll(keepAliveInterval.dividedBy(POLL_TIMEOUT_KEEP_ALIVE_DIVISOR)); + } log.atLevel(records.isEmpty()? Level.TRACE:Level.INFO) .setMessage(()->"Kafka consumer poll has fetched "+records.count() + " records. " + - "Records in flight=" + kafkaRecordsLeftToCommit.get()).log(); + "Records in flight=" + kafkaRecordsLeftToCommitEventually.get()).log(); log.atTrace().setMessage(()->"All positions: {"+kafkaConsumer.assignment().stream() .map(tp->tp+": "+kafkaConsumer.position(tp)).collect(Collectors.joining(",")) + "}").log(); log.atTrace().setMessage(()->"All previously COMMITTED positions: {"+kafkaConsumer.assignment().stream() @@ -219,8 +277,11 @@ private ConsumerRecords safePollWithSwallowedRuntimeExceptions() } } - void commitKafkaKey(KafkaCommitOffsetData kafkaTsk) { - var tracker = partitionToOffsetLifecycleTrackerMap.get(kafkaTsk.getPartition()); + ITrafficCaptureSource.CommitResult commitKafkaKey(ITrafficStreamKey streamKey, KafkaCommitOffsetData kafkaTsk) { + OffsetLifecycleTracker tracker; + synchronized (commitDataLock) { + tracker = partitionToOffsetLifecycleTrackerMap.get(kafkaTsk.getPartition()); + } if (tracker == null || tracker.consumerConnectionGeneration != kafkaTsk.getGeneration()) { log.atWarn().setMessage(()->"trafficKey's generation (" + kafkaTsk.getGeneration() + ") is not current (" + (Optional.ofNullable(tracker).map(t->"new generation=" + t.consumerConnectionGeneration) @@ -228,32 +289,63 @@ void commitKafkaKey(KafkaCommitOffsetData kafkaTsk) { + "). Dropping this commit request since the record would " + "have been handled again by a current consumer within this process or another. Full key=" + kafkaTsk).log(); - return; + return ITrafficCaptureSource.CommitResult.Ignored; } + var p = kafkaTsk.getPartition(); Optional newHeadValue; + var k = new TopicPartition(topic, p); + newHeadValue = tracker.removeAndReturnNewHead(kafkaTsk.getOffset()); - newHeadValue.ifPresent(o -> { - var k = new TopicPartition(topic, p); + return newHeadValue.map(o -> { var v = new OffsetAndMetadata(o); log.atDebug().setMessage(()->"Adding new commit " + k + "->" + v + " to map").log(); - nextSetOfCommitsMap.put(k, v); + synchronized (commitDataLock) { + addKeyContextForEventualCommit(streamKey, kafkaTsk, k); + nextSetOfCommitsMap.put(k, v); + } + return ITrafficCaptureSource.CommitResult.AfterNextRead; + }).orElseGet(() -> { + synchronized (commitDataLock) { + addKeyContextForEventualCommit(streamKey, kafkaTsk, k); + } + return ITrafficCaptureSource.CommitResult.BlockedByOtherCommits; }); } - private void safeCommit() { + private void addKeyContextForEventualCommit(ITrafficStreamKey streamKey, KafkaCommitOffsetData kafkaTsk, TopicPartition k) { + nextSetOfKeysContextsBeingCommitted.computeIfAbsent(k, k2 -> new PriorityQueue<>()) + .add(new OrderedKeyHolder(kafkaTsk.getOffset(), streamKey)); + } + + private void safeCommit(Supplier commitContextSupplier) { + HashMap nextCommitsMapCopy; + IKafkaConsumerContexts.ICommitScopeContext context = null; + synchronized (commitDataLock) { + if (nextSetOfCommitsMap.isEmpty()) { + return; + } + context = commitContextSupplier.get(); + nextCommitsMapCopy = new HashMap<>(nextSetOfCommitsMap); + } try { - if (!nextSetOfCommitsMap.isEmpty()) { - log.atDebug().setMessage(() -> "Committing " + nextSetOfCommitsMap).log(); - kafkaConsumer.commitSync(nextSetOfCommitsMap); - nextSetOfCommitsMap.clear(); - log.trace("partitionToOffsetLifecycleTrackerMap="+partitionToOffsetLifecycleTrackerMap); - kafkaRecordsLeftToCommit.set(partitionToOffsetLifecycleTrackerMap.values().stream() - .mapToInt(OffsetLifecycleTracker::size).sum()); - log.atDebug().setMessage(() -> "Done committing now records in flight=" + - kafkaRecordsLeftToCommit.get()).log(); + safeCommitStatic(context, kafkaConsumer, nextCommitsMapCopy); + synchronized (commitDataLock) { + nextCommitsMapCopy.entrySet().stream() + .forEach(kvp->callbackUpTo(onCommitKeyCallback, + nextSetOfKeysContextsBeingCommitted.get(kvp.getKey()), + kvp.getValue().offset())); + nextCommitsMapCopy.forEach((k,v)->nextSetOfCommitsMap.remove(k)); } + // This function will only ever be called in a threadsafe way, mutually exclusive from any + // other call other than commitKafkaKey(). Since commitKafkaKey() doesn't alter + // partitionToOffsetLifecycleTrackerMap, these lines can be outside of the commitDataLock mutex + log.trace("partitionToOffsetLifecycleTrackerMap="+partitionToOffsetLifecycleTrackerMap); + kafkaRecordsLeftToCommitEventually.set(partitionToOffsetLifecycleTrackerMap.values().stream() + .mapToInt(OffsetLifecycleTracker::size).sum()); + log.atDebug().setMessage(() -> "Done committing now records in flight=" + + kafkaRecordsLeftToCommitEventually.get()).log(); } catch (RuntimeException e) { log.atWarn().setCause(e) .setMessage(() -> "Error while committing. " + @@ -266,6 +358,31 @@ private void safeCommit() { nextSetOfCommitsMap.entrySet().stream() .map(kvp -> kvp.getKey() + "->" + kvp.getValue()).collect(Collectors.joining(","))) .log(); + } finally { + if (context != null) { + context.close(); + } + } + } + + private static void safeCommitStatic(IKafkaConsumerContexts.ICommitScopeContext context, + Consumer kafkaConsumer, + HashMap nextCommitsMap) { + assert !nextCommitsMap.isEmpty(); + log.atDebug().setMessage(() -> "Committing " + nextCommitsMap).log(); + try (var kafkaContext = context.createNewKafkaCommitContext()) { + kafkaConsumer.commitSync(nextCommitsMap); + } + } + + private static void callbackUpTo(java.util.function.Consumer onCommitKeyCallback, + PriorityQueue orderedKeyHolders, long upToOffset) { + for (var nextKeyHolder = orderedKeyHolders.peek(); + nextKeyHolder != null && nextKeyHolder.offset <= upToOffset; + nextKeyHolder = orderedKeyHolders.peek()) + { + onCommitKeyCallback.accept(nextKeyHolder.tsk); + orderedKeyHolders.poll(); } } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/TrafficStreamKeyWithKafkaRecordId.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/TrafficStreamKeyWithKafkaRecordId.java index 4a19942f5..f6c79bee3 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/TrafficStreamKeyWithKafkaRecordId.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/TrafficStreamKeyWithKafkaRecordId.java @@ -2,28 +2,35 @@ import lombok.EqualsAndHashCode; import lombok.Getter; -import lombok.ToString; -import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamKey; +import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; +import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamKeyAndContext; +import org.opensearch.migrations.replay.tracing.IReplayContexts; import org.opensearch.migrations.trafficcapture.protos.TrafficStream; import java.util.StringJoiner; +import java.util.function.Function; @EqualsAndHashCode(callSuper = true) @Getter -class TrafficStreamKeyWithKafkaRecordId extends PojoTrafficStreamKey implements KafkaCommitOffsetData { +class TrafficStreamKeyWithKafkaRecordId extends PojoTrafficStreamKeyAndContext implements KafkaCommitOffsetData { private final int generation; private final int partition; private final long offset; - TrafficStreamKeyWithKafkaRecordId(TrafficStream trafficStream, KafkaCommitOffsetData ok) { - this(trafficStream, ok.getGeneration(), ok.getPartition(), ok.getOffset()); + TrafficStreamKeyWithKafkaRecordId(Function contextFactory, + TrafficStream trafficStream, KafkaCommitOffsetData ok) { + this(contextFactory, trafficStream, ok.getGeneration(), ok.getPartition(), ok.getOffset()); } - TrafficStreamKeyWithKafkaRecordId(TrafficStream trafficStream, int generation, int partition, long offset) { + TrafficStreamKeyWithKafkaRecordId(Function contextFactory, + TrafficStream trafficStream, + int generation, int partition, long offset) { super(trafficStream); this.generation = generation; this.partition = partition; this.offset = offset; + var kafkaContext = contextFactory.apply(this); + this.setTrafficStreamsContext(kafkaContext.createTrafficLifecyleContext(this)); } @Override 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/netty/BacksideSnifferHandler.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/netty/BacksideSnifferHandler.java index 730bd58ee..8c919813b 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/netty/BacksideSnifferHandler.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/netty/BacksideSnifferHandler.java @@ -13,6 +13,7 @@ public class BacksideSnifferHandler extends ChannelInboundHandlerAdapter { private final AggregatedRawResponse.Builder aggregatedRawResponseBuilder; + private static final MetricsLogger metricsLogger = new MetricsLogger("BacksideSnifferHandler"); public BacksideSnifferHandler(AggregatedRawResponse.Builder aggregatedRawResponseBuilder) { diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/ChannelContextManager.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/ChannelContextManager.java new file mode 100644 index 000000000..eaa902ac3 --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/ChannelContextManager.java @@ -0,0 +1,65 @@ +package org.opensearch.migrations.replay.tracing; + +import lombok.Getter; +import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; + +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Function; + +public class ChannelContextManager implements Function { + @Getter + private final RootReplayerContext globalContext; + + public ChannelContextManager(RootReplayerContext globalContext) { + this.globalContext = globalContext; + } + + private static class RefCountedContext { + @Getter + final IReplayContexts.IChannelKeyContext context; + private int refCount; + + private RefCountedContext(IReplayContexts.IChannelKeyContext context) { + this.context = context; + } + + IReplayContexts.IChannelKeyContext retain() { + refCount++; + return context; + } + + /** + * Returns true if this was the final release + * + * @return + */ + boolean release() { + refCount--; + assert refCount >= 0; + return refCount == 0; + } + } + + ConcurrentHashMap connectionToChannelContextMap = new ConcurrentHashMap<>(); + + public IReplayContexts.IChannelKeyContext apply(ITrafficStreamKey tsk) { + return retainOrCreateContext(tsk); + } + + public IReplayContexts.IChannelKeyContext retainOrCreateContext(ITrafficStreamKey tsk) { + return connectionToChannelContextMap.computeIfAbsent(tsk.getConnectionId(), + k -> new RefCountedContext(globalContext.createChannelContext(tsk))).retain(); + } + + public IReplayContexts.IChannelKeyContext releaseContextFor(IReplayContexts.IChannelKeyContext ctx) { + var connId = ctx.getConnectionId(); + var refCountedCtx = connectionToChannelContextMap.get(connId); + assert ctx == refCountedCtx.context; + var finalRelease = refCountedCtx.release(); + if (finalRelease) { + ctx.close(); + connectionToChannelContextMap.remove(connId); + } + return ctx; + } +} diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/IKafkaConsumerContexts.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/IKafkaConsumerContexts.java new file mode 100644 index 000000000..e07c45b09 --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/IKafkaConsumerContexts.java @@ -0,0 +1,79 @@ +package org.opensearch.migrations.replay.tracing; + +import org.opensearch.migrations.tracing.IInstrumentationAttributes; +import org.opensearch.migrations.tracing.IScopedInstrumentationAttributes; + +public interface IKafkaConsumerContexts { + + class ScopeNames { + private ScopeNames() {} + public static final String KAFKA_CONSUMER_SCOPE = "TrackingKafkaConsumer"; + } + + class ActivityNames { + private ActivityNames() {} + public static final String TOUCH = "touch"; + public static final String KAFKA_POLL = "kafkaPoll"; + public static final String COMMIT = "commit"; + public static final String KAFKA_COMMIT = "kafkaCommit"; + } + + class MetricNames { + private MetricNames() {} + public static final String PARTITIONS_ASSIGNED_EVENT_COUNT = "partitionsAssigned"; + public static final String PARTITIONS_REVOKED_EVENT_COUNT = "partitionsRevoked"; + public static final String ACTIVE_PARTITIONS_ASSIGNED_COUNT = "numPartitionsAssigned"; + } + + interface IAsyncListeningContext extends IInstrumentationAttributes { + } + + interface IKafkaConsumerScope extends IScopedInstrumentationAttributes { + } + + interface ITouchScopeContext extends IKafkaConsumerScope { + String ACTIVITY_NAME = ActivityNames.TOUCH; + + @Override + default String getActivityName() { + return ACTIVITY_NAME; + } + + IPollScopeContext createNewPollContext(); + } + + interface IPollScopeContext extends IKafkaConsumerScope { + String ACTIVITY_NAME = ActivityNames.KAFKA_POLL; + + @Override + default String getActivityName() { + return ACTIVITY_NAME; + } + } + + /** + * Context for the KafkaConsumer's bookkeeping around and including the commit service call + */ + interface ICommitScopeContext extends IKafkaConsumerScope { + String ACTIVITY_NAME = ActivityNames.COMMIT; + + @Override + default String getActivityName() { + return ACTIVITY_NAME; + } + + IKafkaCommitScopeContext createNewKafkaCommitContext(); + } + + /** + * Context for ONLY the service call to Kafka to perform the commit. + */ + interface IKafkaCommitScopeContext extends IKafkaConsumerScope { + String ACTIVITY_NAME = ActivityNames.KAFKA_COMMIT; + + @Override + default String getActivityName() { + return ACTIVITY_NAME; + } + } +} diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/IReplayContexts.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/IReplayContexts.java new file mode 100644 index 000000000..4008ea34a --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/IReplayContexts.java @@ -0,0 +1,363 @@ +package org.opensearch.migrations.replay.tracing; + +import io.opentelemetry.api.common.AttributeKey; +import io.opentelemetry.api.common.AttributesBuilder; +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.tracing.IScopedInstrumentationAttributes; +import org.opensearch.migrations.tracing.IWithTypedEnclosingScope; + +import java.time.Instant; + +public abstract class IReplayContexts { + + public static class ActivityNames { + private ActivityNames() {} + + public static final String CHANNEL = "channel"; + public static final String TCP_CONNECTION = "tcpConnection"; + public static final String RECORD_LIFETIME = "recordLifetime"; + public static final String TRAFFIC_STREAM_LIFETIME = "trafficStreamLifetime"; + public static final String HTTP_TRANSACTION = "httpTransaction"; + public static final String ACCUMULATING_REQUEST = "accumulatingRequest"; + public static final String ACCUMULATING_RESPONSE = "accumulatingResponse"; + public static final String TRANSFORMATION = "transformation"; + public static final String SCHEDULED = "scheduled"; + public static final String TARGET_TRANSACTION = "targetTransaction"; + public static final String REQUEST_SENDING = "requestSending"; + public static final String WAITING_FOR_RESPONSE = "waitingForResponse"; + public static final String RECEIVING_RESPONSE = "receivingResponse"; + public static final String TUPLE_COMPARISON = "comparingResults"; + } + + public static class MetricNames { + private MetricNames() {} + public static final String KAFKA_RECORD_READ = "kafkaRecordsRead"; + public static final String KAFKA_BYTES_READ = "kafkaBytesRead"; + public static final String TRAFFIC_STREAMS_READ = "trafficStreamsRead"; + public static final String TRANSFORM_HEADER_PARSE = "parsedHeader"; + public static final String TRANSFORM_PAYLOAD_PARSE_REQUIRED = "parsedPayload"; + public static final String TRANSFORM_PAYLOAD_PARSE_SUCCESS = "parsedPayloadSuccess"; + public static final String TRANSFORM_JSON_REQUIRED = "transformedJsonRequired"; + public static final String TRANSFORM_JSON_SUCCEEDED = "transformedJsonSucceeded"; + public static final String TRANSFORM_PAYLOAD_BYTES_IN = "originalPayloadBytesIn"; + public static final String TRANSFORM_UNCOMPRESSED_BYTES_IN = "uncompressedBytesIn"; + public static final String TRANSFORM_UNCOMPRESSED_BYTES_OUT = "uncompressedBytesOut"; + public static final String TRANSFORM_FINAL_PAYLOAD_BYTES_OUT = "finalPayloadBytesOut"; + public static final String TRANSFORM_SUCCESS = "transformSuccess"; + public static final String TRANSFORM_SKIPPED = "transformSkipped"; + public static final String TRANSFORM_ERROR = "transformError"; + public static final String TRANSFORM_BYTES_IN = "transformBytesIn"; + public static final String TRANSFORM_BYTES_OUT = "transformBytesOut"; + public static final String TRANSFORM_CHUNKS_IN = "transformChunksIn"; + public static final String TRANSFORM_CHUNKS_OUT = "transformChunksOut"; + public static final String NETTY_SCHEDULE_LAG = "scheduleLag"; + public static final String SOURCE_TO_TARGET_REQUEST_LAG = "lagBetweenSourceAndTargetRequests"; + public static final String ACTIVE_CHANNELS_YET_TO_BE_FULLY_DISCARDED = "activeReplayerChannels"; + public static final String ACTIVE_TARGET_CONNECTIONS = "activeTargetConnections"; + public static final String CONNECTIONS_OPENED = "connectionsOpened"; + public static final String CONNECTIONS_CLOSED = "connectionsClosedCount"; + public static final String BYTES_WRITTEN_TO_TARGET = "bytesWrittenToTarget"; + public static final String BYTES_READ_FROM_TARGET = "bytesReadFromTarget"; + public static final String TUPLE_COMPARISON = "tupleComparison"; + } + + public interface IAccumulationScope extends IScopedInstrumentationAttributes { + } + + public interface IChannelKeyContext + extends IAccumulationScope, + org.opensearch.migrations.tracing.commoncontexts.IConnectionContext { + String ACTIVITY_NAME = ActivityNames.CHANNEL; + + @Override + default String getActivityName() { + return ACTIVITY_NAME; + } + + // do not add this as a property + // because its components are already being added in the IConnectionContext implementation + ISourceTrafficChannelKey getChannelKey(); + + default String getConnectionId() { + return getChannelKey().getConnectionId(); + } + + default String getNodeId() { + return getChannelKey().getNodeId(); + } + + ISocketContext createSocketContext(); + } + + public interface ISocketContext extends IAccumulationScope, IWithTypedEnclosingScope { + public static final String ACTIVITY_NAME = ActivityNames.TCP_CONNECTION; + + @Override + default String getActivityName() { + return ACTIVITY_NAME; + } + } + + public interface IKafkaRecordContext + extends IAccumulationScope, + IWithTypedEnclosingScope + { + String ACTIVITY_NAME = ActivityNames.RECORD_LIFETIME; + + @Override + default String getActivityName() { + return ACTIVITY_NAME; + } + + static final AttributeKey RECORD_ID_KEY = AttributeKey.stringKey("recordId"); + + String getRecordId(); + + @Override + default AttributesBuilder fillAttributesForSpansBelow(AttributesBuilder builder) { + return IAccumulationScope.super.fillAttributesForSpansBelow( + builder.put(RECORD_ID_KEY, getRecordId())); + } + + ITrafficStreamsLifecycleContext createTrafficLifecyleContext(ITrafficStreamKey tsk); + } + + public interface ITrafficStreamsLifecycleContext + extends IAccumulationScope, + IWithTypedEnclosingScope { + String ACTIVITY_NAME = ActivityNames.TRAFFIC_STREAM_LIFETIME; + + @Override + default String getActivityName() { + return ACTIVITY_NAME; + } + + ITrafficStreamKey getTrafficStreamKey(); + + IChannelKeyContext getChannelKeyContext(); + + default String getConnectionId() { + return getChannelKey().getConnectionId(); + } + + default ISourceTrafficChannelKey getChannelKey() { + return getChannelKeyContext().getChannelKey(); + } + + IReplayerHttpTransactionContext createHttpTransactionContext(UniqueReplayerRequestKey requestKey, + Instant sourceTimestamp); + } + + public interface IReplayerHttpTransactionContext + extends org.opensearch.migrations.tracing.commoncontexts.IHttpTransactionContext, + IAccumulationScope, + IWithTypedEnclosingScope { + AttributeKey REPLAYER_REQUEST_INDEX_KEY = AttributeKey.longKey("replayerRequestIndex"); + + String ACTIVITY_NAME = ActivityNames.HTTP_TRANSACTION; + + @Override + default String getActivityName() { + return ACTIVITY_NAME; + } + + UniqueReplayerRequestKey getReplayerRequestKey(); + + IChannelKeyContext getChannelKeyContext(); + + Instant getTimeOfOriginalRequest(); + + default String getConnectionId() { + return getChannelKey().getConnectionId(); + } + + default ISourceTrafficChannelKey getChannelKey() { + return getChannelKeyContext().getChannelKey(); + } + + default long getSourceRequestIndex() { + return getReplayerRequestKey().getSourceRequestIndex(); + } + + default long replayerRequestIndex() { + return getReplayerRequestKey().getReplayerRequestIndex(); + } + + @Override + default AttributesBuilder fillAttributesForSpansBelow(AttributesBuilder builder) { + return org.opensearch.migrations.tracing.commoncontexts.IHttpTransactionContext.super.fillAttributesForSpansBelow(builder) + .put(REPLAYER_REQUEST_INDEX_KEY, replayerRequestIndex()); + } + + IRequestAccumulationContext createRequestAccumulationContext(); + + IResponseAccumulationContext createResponseAccumulationContext(); + + IRequestTransformationContext createTransformationContext(); + + IScheduledContext createScheduledContext(Instant timestamp); + + ITargetRequestContext createTargetRequestContext(); + + ITupleHandlingContext createTupleContext(); + } + + public interface IRequestAccumulationContext + extends IAccumulationScope, + IWithTypedEnclosingScope { + String ACTIVITY_NAME = ActivityNames.ACCUMULATING_REQUEST; + + @Override + default String getActivityName() { + return ACTIVITY_NAME; + } + } + + public interface IResponseAccumulationContext + extends IAccumulationScope, + IWithTypedEnclosingScope { + String ACTIVITY_NAME = ActivityNames.ACCUMULATING_RESPONSE; + + @Override + default String getActivityName() { + return ACTIVITY_NAME; + } + } + + public interface IRequestTransformationContext + extends IAccumulationScope, + IWithTypedEnclosingScope { + String ACTIVITY_NAME = ActivityNames.TRANSFORMATION; + + @Override + default String getActivityName() { + return ACTIVITY_NAME; + } + + void onHeaderParse(); + + void onPayloadParse(); + + void onPayloadParseSuccess(); + + void onJsonPayloadParseRequired(); + + void onJsonPayloadParseSucceeded(); + + void onPayloadBytesIn(int inputSize); + + void onUncompressedBytesIn(int inputSize); + + void onUncompressedBytesOut(int inputSize); + + void onFinalBytesOut(int outputSize); + + void onTransformSuccess(); + + void onTransformSkip(); + + void onTransformFailure(); + + void aggregateInputChunk(int sizeInBytes); + + void aggregateOutputChunk(int sizeInBytes); + } + + public interface IScheduledContext + extends IAccumulationScope, + IWithTypedEnclosingScope { + String ACTIVITY_NAME = ActivityNames.SCHEDULED; + + @Override + default String getActivityName() { + return ACTIVITY_NAME; + } + } + + public interface ITargetRequestContext + extends IAccumulationScope, + IWithTypedEnclosingScope { + String ACTIVITY_NAME = ActivityNames.TARGET_TRANSACTION; + + @Override + default String getActivityName() { + return ACTIVITY_NAME; + } + + void onBytesSent(int size); + + void onBytesReceived(int size); + + IRequestSendingContext createHttpSendingContext(); + + IWaitingForHttpResponseContext createWaitingForResponseContext(); + + IReceivingHttpResponseContext createHttpReceivingContext(); + } + + public interface IRequestSendingContext + extends IAccumulationScope, + IWithTypedEnclosingScope { + String ACTIVITY_NAME = ActivityNames.REQUEST_SENDING; + + @Override + default String getActivityName() { + return ACTIVITY_NAME; + } + } + + public interface IWaitingForHttpResponseContext + extends IAccumulationScope, + IWithTypedEnclosingScope { + String ACTIVITY_NAME = ActivityNames.WAITING_FOR_RESPONSE; + + @Override + default String getActivityName() { + return ACTIVITY_NAME; + } + } + + public interface IReceivingHttpResponseContext + extends IAccumulationScope, + IWithTypedEnclosingScope { + String ACTIVITY_NAME = ActivityNames.RECEIVING_RESPONSE; + + @Override + default String getActivityName() { + return ACTIVITY_NAME; + } + } + + public interface ITupleHandlingContext + extends IAccumulationScope, + IWithTypedEnclosingScope { + String ACTIVITY_NAME = ActivityNames.TUPLE_COMPARISON; + AttributeKey SOURCE_STATUS_CODE_KEY = AttributeKey.longKey("sourceStatusCode"); + AttributeKey TARGET_STATUS_CODE_KEY = AttributeKey.longKey("targetStatusCode"); + AttributeKey STATUS_CODE_MATCH_KEY = AttributeKey.booleanKey("statusCodesMatch"); + AttributeKey METHOD_KEY = AttributeKey.stringKey("method"); + AttributeKey HTTP_VERSION_KEY = AttributeKey.stringKey("version"); // for the span, not metric + AttributeKey ENDPOINT_KEY = AttributeKey.stringKey("endpoint"); // for the span, not metric + + @Override + default String getActivityName() { + return ACTIVITY_NAME; + } + + void setSourceStatus(Integer sourceStatus); + + void setTargetStatus(Integer targetStatus); + + void setMethod(String method); + + void setEndpoint(String endpointUrl); + + void setHttpVersion(String string); + + default UniqueReplayerRequestKey getReplayerRequestKey() { + return getLogicalEnclosingScope().getReplayerRequestKey(); + } + } +} diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/IRootReplayerContext.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/IRootReplayerContext.java new file mode 100644 index 000000000..843b63879 --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/IRootReplayerContext.java @@ -0,0 +1,12 @@ +package org.opensearch.migrations.replay.tracing; + +import org.opensearch.migrations.replay.datatypes.ISourceTrafficChannelKey; +import org.opensearch.migrations.tracing.IInstrumentConstructor; +import org.opensearch.migrations.tracing.IRootOtelContext; + +public interface IRootReplayerContext extends IRootOtelContext, IInstrumentConstructor { + + ITrafficSourceContexts.IReadChunkContext createReadChunkContext(); + + IReplayContexts.IChannelKeyContext createChannelContext(ISourceTrafficChannelKey tsk); +} diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/ITrafficSourceContexts.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/ITrafficSourceContexts.java new file mode 100644 index 000000000..ee21844be --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/ITrafficSourceContexts.java @@ -0,0 +1,57 @@ +package org.opensearch.migrations.replay.tracing; + +import org.opensearch.migrations.tracing.IScopedInstrumentationAttributes; + +public interface ITrafficSourceContexts { + class ScopeNames { + private ScopeNames() {} + public static final String TRAFFIC_SCOPE = "BlockingTrafficSource"; + } + + class ActivityNames { + private ActivityNames() {} + public static final String READ_NEXT_TRAFFIC_CHUNK = "readNextTrafficStreamChunk"; + public static final String BACK_PRESSURE_BLOCK = "backPressureBlock"; + public static final String WAIT_FOR_NEXT_BACK_PRESSURE_CHECK = "waitForNextBackPressureCheck"; + } + + interface ITrafficSourceContext extends IScopedInstrumentationAttributes { } + + interface IReadChunkContext extends ITrafficSourceContext { + String ACTIVITY_NAME = ActivityNames.READ_NEXT_TRAFFIC_CHUNK; + + @Override + default String getActivityName() { + return ACTIVITY_NAME; + } + + IBackPressureBlockContext createBackPressureContext(); + + IKafkaConsumerContexts.IPollScopeContext createPollContext(); + + IKafkaConsumerContexts.ICommitScopeContext createCommitContext(); + } + + interface IBackPressureBlockContext extends ITrafficSourceContext { + String ACTIVITY_NAME = ActivityNames.BACK_PRESSURE_BLOCK; + + @Override + default String getActivityName() { + return ACTIVITY_NAME; + } + + IWaitForNextSignal createWaitForSignalContext(); + + IKafkaConsumerContexts.ITouchScopeContext createNewTouchContext(); + + IKafkaConsumerContexts.ICommitScopeContext createCommitContext(); + } + + interface IWaitForNextSignal extends ITrafficSourceContext { + String ACTIVITY_NAME = ActivityNames.WAIT_FOR_NEXT_BACK_PRESSURE_CHECK; + + default String getActivityName() { + return ACTIVITY_NAME; + } + } +} diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/KafkaConsumerContexts.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/KafkaConsumerContexts.java new file mode 100644 index 000000000..5c777568d --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/KafkaConsumerContexts.java @@ -0,0 +1,185 @@ +package org.opensearch.migrations.replay.tracing; + +import io.opentelemetry.api.metrics.LongCounter; +import io.opentelemetry.api.metrics.LongUpDownCounter; +import io.opentelemetry.api.metrics.Meter; +import lombok.Getter; +import lombok.NonNull; +import lombok.Setter; +import org.apache.kafka.common.TopicPartition; +import org.opensearch.migrations.tracing.BaseNestedSpanContext; +import org.opensearch.migrations.tracing.CommonMetricInstruments; +import org.opensearch.migrations.tracing.CommonScopedMetricInstruments; +import org.opensearch.migrations.tracing.DirectNestedSpanContext; +import org.opensearch.migrations.tracing.IInstrumentationAttributes; +import org.opensearch.migrations.tracing.IScopedInstrumentationAttributes; + +import java.util.Collection; + +public class KafkaConsumerContexts { + + private KafkaConsumerContexts() {} + + public static class AsyncListeningContext + implements IKafkaConsumerContexts.IAsyncListeningContext { + @Getter + @NonNull + public final RootReplayerContext enclosingScope; + @Getter + @Setter + Exception observedExceptionToIncludeInMetrics; + + public AsyncListeningContext(@NonNull RootReplayerContext enclosingScope) { + this.enclosingScope = enclosingScope; + } + + public static class MetricInstruments extends CommonMetricInstruments { + public final LongCounter kafkaPartitionsRevokedCounter; + public final LongCounter kafkaPartitionsAssignedCounter; + public final LongUpDownCounter kafkaActivePartitionsCounter; + + private MetricInstruments(Meter meter) { + super(meter, "asyncKafkaProcessing"); + kafkaPartitionsRevokedCounter = meter + .counterBuilder(IKafkaConsumerContexts.MetricNames.PARTITIONS_REVOKED_EVENT_COUNT).build(); + kafkaPartitionsAssignedCounter = meter + .counterBuilder(IKafkaConsumerContexts.MetricNames.PARTITIONS_ASSIGNED_EVENT_COUNT).build(); + kafkaActivePartitionsCounter = meter + .upDownCounterBuilder(IKafkaConsumerContexts.MetricNames.ACTIVE_PARTITIONS_ASSIGNED_COUNT).build(); + } + } + + public static @NonNull MetricInstruments makeMetrics(Meter meter) { + return new MetricInstruments(meter); + } + + @NonNull public MetricInstruments getMetrics() { + return enclosingScope.asyncListeningInstruments; + } + + public void onPartitionsRevoked(Collection partitions) { + meterIncrementEvent(getMetrics().kafkaPartitionsRevokedCounter); + onParitionsAssignedChanged(partitions.size()); + } + + public void onPartitionsAssigned(Collection partitions) { + meterIncrementEvent(getMetrics().kafkaPartitionsAssignedCounter); + onParitionsAssignedChanged(partitions.size()); + } + + private void onParitionsAssignedChanged(int delta) { + meterDeltaEvent(getMetrics().kafkaActivePartitionsCounter, delta); + } + } + + public static class TouchScopeContext + extends DirectNestedSpanContext + implements IKafkaConsumerContexts.ITouchScopeContext + { + @Override + public IKafkaConsumerContexts.IPollScopeContext createNewPollContext() { + return new KafkaConsumerContexts.PollScopeContext(getRootInstrumentationScope(), this); + } + + public static class MetricInstruments extends CommonScopedMetricInstruments { + private MetricInstruments(Meter meter, String activityName) { + super(meter, activityName); + } + } + + public TouchScopeContext(@NonNull TrafficSourceContexts.BackPressureBlockContext enclosingScope) { + super(enclosingScope); + initializeSpan(); + } + + public static @NonNull MetricInstruments makeMetrics(Meter meter) { + return new MetricInstruments(meter, ACTIVITY_NAME); + } + + public @NonNull MetricInstruments getMetrics() { + return getRootInstrumentationScope().touchInstruments; + } + } + + public static class PollScopeContext + extends BaseNestedSpanContext + implements IKafkaConsumerContexts.IPollScopeContext { + public static class MetricInstruments extends CommonScopedMetricInstruments { + private MetricInstruments(Meter meter, String activityName) { + super(meter, activityName); + } + } + + public static @NonNull MetricInstruments makeMetrics(Meter meter) { + return new MetricInstruments(meter, ACTIVITY_NAME); + } + + @Override + public @NonNull MetricInstruments getMetrics() { + return getRootInstrumentationScope().pollInstruments; + } + + public PollScopeContext(@NonNull RootReplayerContext rootScope, + @NonNull IScopedInstrumentationAttributes enclosingScope) { + super(rootScope, enclosingScope); + initializeSpan(); + } + } + + public static class CommitScopeContext + extends BaseNestedSpanContext + implements IKafkaConsumerContexts.ICommitScopeContext { + + @Override + public IKafkaConsumerContexts.IKafkaCommitScopeContext createNewKafkaCommitContext() { + return new KafkaConsumerContexts.KafkaCommitScopeContext(this); + } + + public static class MetricInstruments extends CommonScopedMetricInstruments { + private MetricInstruments(Meter meter, String activityName) { + super(meter, activityName); + } + } + + public static @NonNull MetricInstruments makeMetrics(Meter meter) { + return new MetricInstruments(meter, ACTIVITY_NAME); + } + + @Override + public @NonNull MetricInstruments getMetrics() { + return getRootInstrumentationScope().commitInstruments; + } + + public CommitScopeContext(@NonNull RootReplayerContext rootScope, + IScopedInstrumentationAttributes enclosingScope) { + super(rootScope, enclosingScope); + initializeSpan(); + } + } + + public static class KafkaCommitScopeContext + extends DirectNestedSpanContext + implements IKafkaConsumerContexts.IKafkaCommitScopeContext { + public static class MetricInstruments extends CommonScopedMetricInstruments { + private MetricInstruments(Meter meter, String activityName) { + super(meter, activityName); + } + } + + public static @NonNull MetricInstruments makeMetrics(Meter meter) { + return new MetricInstruments(meter, ACTIVITY_NAME); + } + + @Override + public @NonNull MetricInstruments getMetrics() { + return getRootInstrumentationScope().kafkaCommitInstruments; + } + + + public KafkaCommitScopeContext(@NonNull KafkaConsumerContexts.CommitScopeContext enclosingScope) { + super(enclosingScope); + initializeSpan(); + } + + } +} diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/ReplayContexts.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/ReplayContexts.java new file mode 100644 index 000000000..20066be81 --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/ReplayContexts.java @@ -0,0 +1,783 @@ +package org.opensearch.migrations.replay.tracing; + +import io.opentelemetry.api.common.AttributeKey; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.common.AttributesBuilder; +import io.opentelemetry.api.metrics.DoubleHistogram; +import io.opentelemetry.api.metrics.LongCounter; +import io.opentelemetry.api.metrics.LongUpDownCounter; +import io.opentelemetry.api.metrics.Meter; +import lombok.Getter; +import lombok.NonNull; +import lombok.Setter; +import lombok.extern.slf4j.Slf4j; +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.tracing.BaseNestedSpanContext; +import org.opensearch.migrations.tracing.CommonScopedMetricInstruments; +import org.opensearch.migrations.tracing.DirectNestedSpanContext; +import org.opensearch.migrations.tracing.IScopedInstrumentationAttributes; + +import java.time.Duration; +import java.time.Instant; +import java.util.Optional; + +@Slf4j +public abstract class ReplayContexts extends IReplayContexts { + + public static final String COUNT_UNIT_STR = "count"; + public static final String BYTES_UNIT_STR = "bytes"; + + public static class SocketContext + extends DirectNestedSpanContext + implements ISocketContext { + + protected SocketContext(ChannelKeyContext enclosingScope) { + super(enclosingScope); + initializeSpan(); + meterIncrementEvent(getMetrics().channelCreatedCounter); + meterDeltaEvent(getMetrics().activeSocketConnectionsCounter, 1); + } + + public static class MetricInstruments extends CommonScopedMetricInstruments { + final LongUpDownCounter activeSocketConnectionsCounter; + final LongCounter channelCreatedCounter; + final LongCounter channelClosedCounter; + + private MetricInstruments(Meter meter, String activityName) { + super(meter, activityName); + activeSocketConnectionsCounter = meter + .upDownCounterBuilder(MetricNames.ACTIVE_TARGET_CONNECTIONS).build(); + channelCreatedCounter = meter + .counterBuilder(MetricNames.CONNECTIONS_OPENED).build(); + channelClosedCounter = meter + .counterBuilder(MetricNames.CONNECTIONS_CLOSED).build(); + } + } + + @Override + public void sendMeterEventsForEnd() { + super.sendMeterEventsForEnd(); + meterIncrementEvent(getMetrics().channelClosedCounter); + meterDeltaEvent(getMetrics().activeSocketConnectionsCounter, -1); + } + + public static MetricInstruments makeMetrics(Meter meter) { + return new MetricInstruments(meter, ACTIVITY_NAME); + } + + @Override + public MetricInstruments getMetrics() { + return getRootInstrumentationScope().socketInstruments; + } + } + + public static class ChannelKeyContext + extends BaseNestedSpanContext + implements IReplayContexts.IChannelKeyContext { + @Getter + final ISourceTrafficChannelKey channelKey; + + SocketContext socketContext; + + public ChannelKeyContext(RootReplayerContext rootScope, + IScopedInstrumentationAttributes enclosingScope, + ISourceTrafficChannelKey channelKey) { + super(rootScope, enclosingScope); + this.channelKey = channelKey; + initializeSpan(); + meterDeltaEvent(getMetrics().activeChannelCounter, 1); + } + + @Override + public ISocketContext createSocketContext() { + return new SocketContext(this); + } + + public static class MetricInstruments extends CommonScopedMetricInstruments { + final LongUpDownCounter activeChannelCounter; + + private MetricInstruments(Meter meter, String activityName) { + super(meter, activityName); + activeChannelCounter = meter + .upDownCounterBuilder(MetricNames.ACTIVE_CHANNELS_YET_TO_BE_FULLY_DISCARDED).build(); + } + } + + public static @NonNull MetricInstruments makeMetrics(Meter meter) { + return new MetricInstruments(meter, ACTIVITY_NAME); + } + + public @NonNull MetricInstruments getMetrics() { + return getRootInstrumentationScope().channelKeyInstruments; + } + + @Override + public String toString() { + return channelKey.toString(); + } + + @Override + public void sendMeterEventsForEnd() { + super.sendMeterEventsForEnd(); + meterDeltaEvent(getMetrics().activeChannelCounter, -1); + } + } + + public static class KafkaRecordContext + extends BaseNestedSpanContext + implements IReplayContexts.IKafkaRecordContext { + + final String recordId; + + public KafkaRecordContext(RootReplayerContext rootReplayerContext, + IChannelKeyContext enclosingScope, String recordId, int recordSize) { + super(rootReplayerContext, enclosingScope); + this.recordId = recordId; + initializeSpan(); + meterIncrementEvent(getMetrics().recordCounter); + meterIncrementEvent(getMetrics().bytesCounter, recordSize); + } + + @Override + public IChannelKeyContext getLogicalEnclosingScope() { + return (IChannelKeyContext) getEnclosingScope(); + } + + public static class MetricInstruments extends CommonScopedMetricInstruments { + final LongCounter recordCounter; + final LongCounter bytesCounter; + + private MetricInstruments(Meter meter, String activityName) { + super(meter, activityName); + recordCounter = meter.counterBuilder(MetricNames.KAFKA_RECORD_READ) + .setUnit("records").build(); + bytesCounter = meter.counterBuilder(MetricNames.KAFKA_BYTES_READ) + .setUnit(BYTES_UNIT_STR).build(); + } + } + + public static @NonNull MetricInstruments makeMetrics(Meter meter) { + return new MetricInstruments(meter, ACTIVITY_NAME); + } + + public @NonNull MetricInstruments getMetrics() { + return getRootInstrumentationScope().kafkaRecordInstruments; + } + + @Override + public String getRecordId() { + return recordId; + } + + @Override + public IReplayContexts.ITrafficStreamsLifecycleContext + createTrafficLifecyleContext(ITrafficStreamKey tsk) { + return new TrafficStreamLifecycleContext(this.getRootInstrumentationScope(), this, tsk + ); + } + } + + public static class TrafficStreamLifecycleContext + extends BaseNestedSpanContext + implements IReplayContexts.ITrafficStreamsLifecycleContext { + private final ITrafficStreamKey trafficStreamKey; + + protected TrafficStreamLifecycleContext(RootReplayerContext rootScope, + IScopedInstrumentationAttributes enclosingScope, + ITrafficStreamKey trafficStreamKey) { + super(rootScope, enclosingScope); + this.trafficStreamKey = trafficStreamKey; + initializeSpan(); + meterIncrementEvent(getMetrics().streamsRead); + } + + public static class MetricInstruments extends CommonScopedMetricInstruments { + private final LongCounter streamsRead; + + private MetricInstruments(Meter meter, String activityName) { + super(meter, activityName); + streamsRead = meter.counterBuilder(MetricNames.TRAFFIC_STREAMS_READ) + .setUnit("objects").build(); + } + } + + public static @NonNull MetricInstruments makeMetrics(Meter meter) { + return new MetricInstruments(meter, ACTIVITY_NAME); + } + + public @NonNull MetricInstruments getMetrics() { + return getRootInstrumentationScope().trafficStreamLifecycleInstruments; + } + + @Override + public IReplayContexts.IChannelKeyContext getChannelKeyContext() { + return getLogicalEnclosingScope(); + } + + @Override + public HttpTransactionContext createHttpTransactionContext(UniqueReplayerRequestKey requestKey, + Instant sourceTimestamp) { + return new ReplayContexts.HttpTransactionContext(getRootInstrumentationScope(), + this, requestKey, sourceTimestamp); + } + + @Override + public ITrafficStreamKey getTrafficStreamKey() { + return trafficStreamKey; + } + + @Override + public IReplayContexts.IChannelKeyContext getLogicalEnclosingScope() { + var parent = getEnclosingScope(); + while (!(parent instanceof IReplayContexts.IChannelKeyContext)) { + parent = parent.getEnclosingScope(); + } + return (IReplayContexts.IChannelKeyContext) parent; + } + } + + public static class HttpTransactionContext + extends BaseNestedSpanContext + implements IReplayContexts.IReplayerHttpTransactionContext { + final UniqueReplayerRequestKey replayerRequestKey; + @Getter + final Instant timeOfOriginalRequest; + + public HttpTransactionContext(RootReplayerContext rootScope, + IReplayContexts.ITrafficStreamsLifecycleContext enclosingScope, + UniqueReplayerRequestKey replayerRequestKey, + Instant timeOfOriginalRequest) { + super(rootScope, enclosingScope); + this.replayerRequestKey = replayerRequestKey; + this.timeOfOriginalRequest = timeOfOriginalRequest; + initializeSpan(); + } + + @Override + public IReplayContexts.ITupleHandlingContext createTupleContext() { + return new ReplayContexts.TupleHandlingContext(this); + } + + public static class MetricInstruments extends CommonScopedMetricInstruments { + private MetricInstruments(Meter meter, String activityName) { + super(meter, activityName); + } + } + + public static @NonNull MetricInstruments makeMetrics(Meter meter) { + return new MetricInstruments(meter, ACTIVITY_NAME); + } + + public @NonNull MetricInstruments getMetrics() { + return getRootInstrumentationScope().httpTransactionInstruments; + } + + public IReplayContexts.IChannelKeyContext getChannelKeyContext() { + return getLogicalEnclosingScope(); + } + + @Override + public RequestTransformationContext createTransformationContext() { + return new ReplayContexts.RequestTransformationContext(this); + } + + public IReplayContexts.IRequestAccumulationContext createRequestAccumulationContext() { + return new ReplayContexts.RequestAccumulationContext(this); + } + + @Override + public IReplayContexts.IResponseAccumulationContext createResponseAccumulationContext() { + return new ReplayContexts.ResponseAccumulationContext(this); + } + + @Override + public TargetRequestContext createTargetRequestContext() { + return new ReplayContexts.TargetRequestContext(this); + } + + @Override + public IReplayContexts.IScheduledContext createScheduledContext(Instant timestamp) { + return new ReplayContexts.ScheduledContext(this, + Duration.between(Instant.now(), timestamp).toNanos()); + } + + @Override + public UniqueReplayerRequestKey getReplayerRequestKey() { + return replayerRequestKey; + } + + @Override + public String toString() { + return replayerRequestKey.toString(); + } + + @Override + public IReplayContexts.IChannelKeyContext getLogicalEnclosingScope() { + return getImmediateEnclosingScope().getLogicalEnclosingScope(); + } + } + + public static class RequestAccumulationContext + extends DirectNestedSpanContext + implements IReplayContexts.IRequestAccumulationContext { + public RequestAccumulationContext(HttpTransactionContext enclosingScope) { + super(enclosingScope); + initializeSpan(); + } + + public static class MetricInstruments extends CommonScopedMetricInstruments { + private MetricInstruments(Meter meter, String activityName) { + super(meter, activityName); + } + } + + public static @NonNull MetricInstruments makeMetrics(Meter meter) { + return new MetricInstruments(meter, ACTIVITY_NAME); + } + + public @NonNull MetricInstruments getMetrics() { + return getRootInstrumentationScope().requestAccumInstruments; + } + } + + public static class ResponseAccumulationContext + extends DirectNestedSpanContext + implements IReplayContexts.IResponseAccumulationContext { + public ResponseAccumulationContext(HttpTransactionContext enclosingScope) { + super(enclosingScope); + initializeSpan(); + } + + public static class MetricInstruments extends CommonScopedMetricInstruments { + private MetricInstruments(Meter meter, String activityName) { + super(meter, activityName); + } + } + + public static @NonNull MetricInstruments makeMetrics(Meter meter) { + return new MetricInstruments(meter, ACTIVITY_NAME); + } + + public @NonNull MetricInstruments getMetrics() { + return getRootInstrumentationScope().responseAccumInstruments; + } + } + + public static class RequestTransformationContext + extends DirectNestedSpanContext + implements IReplayContexts.IRequestTransformationContext { + public RequestTransformationContext(HttpTransactionContext enclosingScope) { + super(enclosingScope); + initializeSpan(); + } + + public static class MetricInstruments extends CommonScopedMetricInstruments { + private final LongCounter headerParses; + private final LongCounter payloadParses; + private final LongCounter payloadSuccessParses; + private final LongCounter jsonPayloadParses; + private final LongCounter jsonTransformSuccess; + private final LongCounter payloadBytesIn; + private final LongCounter uncompressedBytesIn; + private final LongCounter uncompressedBytesOut; + private final LongCounter finalPayloadBytesOut; + private final LongCounter transformSuccess; + private final LongCounter transformSkipped; + private final LongCounter transformError; + private final LongCounter transformBytesIn; + private final LongCounter transformChunksIn; + private final LongCounter transformBytesOut; + private final LongCounter transformChunksOut; + + private MetricInstruments(Meter meter, String activityName) { + super(meter, activityName); + headerParses = meter.counterBuilder(MetricNames.TRANSFORM_HEADER_PARSE) + .setUnit(COUNT_UNIT_STR).build(); + payloadParses = meter.counterBuilder(MetricNames.TRANSFORM_PAYLOAD_PARSE_REQUIRED) + .setUnit(COUNT_UNIT_STR).build(); + payloadSuccessParses = meter.counterBuilder(MetricNames.TRANSFORM_PAYLOAD_PARSE_SUCCESS) + .setUnit(COUNT_UNIT_STR).build(); + jsonPayloadParses = meter.counterBuilder(MetricNames.TRANSFORM_JSON_REQUIRED) + .setUnit(COUNT_UNIT_STR).build(); + jsonTransformSuccess = meter.counterBuilder(MetricNames.TRANSFORM_JSON_SUCCEEDED) + .setUnit(COUNT_UNIT_STR).build(); + payloadBytesIn = meter.counterBuilder(MetricNames.TRANSFORM_PAYLOAD_BYTES_IN) + .setUnit(BYTES_UNIT_STR).build(); + uncompressedBytesIn = meter.counterBuilder(MetricNames.TRANSFORM_UNCOMPRESSED_BYTES_IN) + .setUnit(BYTES_UNIT_STR).build(); + uncompressedBytesOut = meter.counterBuilder(MetricNames.TRANSFORM_UNCOMPRESSED_BYTES_OUT) + .setUnit(BYTES_UNIT_STR).build(); + finalPayloadBytesOut = meter.counterBuilder(MetricNames.TRANSFORM_FINAL_PAYLOAD_BYTES_OUT) + .setUnit(BYTES_UNIT_STR).build(); + transformSuccess = meter.counterBuilder(MetricNames.TRANSFORM_SUCCESS) + .setUnit(COUNT_UNIT_STR).build(); + transformSkipped = meter.counterBuilder(MetricNames.TRANSFORM_SKIPPED) + .setUnit(COUNT_UNIT_STR).build(); + transformError = meter.counterBuilder(MetricNames.TRANSFORM_ERROR) + .setUnit(COUNT_UNIT_STR).build(); + transformBytesIn = meter.counterBuilder(MetricNames.TRANSFORM_BYTES_IN) + .setUnit(BYTES_UNIT_STR).build(); + transformChunksIn = meter.counterBuilder(MetricNames.TRANSFORM_CHUNKS_IN) + .setUnit(COUNT_UNIT_STR).build(); + transformBytesOut = meter.counterBuilder(MetricNames.TRANSFORM_BYTES_OUT) + .setUnit(BYTES_UNIT_STR).build(); + transformChunksOut = meter.counterBuilder(MetricNames.TRANSFORM_CHUNKS_OUT) + .setUnit(COUNT_UNIT_STR).build(); + + } + } + + public static @NonNull MetricInstruments makeMetrics(Meter meter) { + return new MetricInstruments(meter, ACTIVITY_NAME); + } + + public @NonNull MetricInstruments getMetrics() { + return getRootInstrumentationScope().transformationInstruments; + } + + @Override + public void onHeaderParse() { + meterIncrementEvent(getMetrics().headerParses); + } + + @Override + public void onPayloadParse() { + meterIncrementEvent(getMetrics().payloadParses); + } + + @Override + public void onPayloadParseSuccess() { + meterIncrementEvent(getMetrics().payloadSuccessParses); + } + + @Override + public void onJsonPayloadParseRequired() { + meterIncrementEvent(getMetrics().jsonPayloadParses); + } + + @Override + public void onJsonPayloadParseSucceeded() { + meterIncrementEvent(getMetrics().jsonTransformSuccess); + } + + @Override + public void onPayloadBytesIn(int inputSize) { + meterIncrementEvent(getMetrics().payloadBytesIn, inputSize); + } + + @Override + public void onUncompressedBytesIn(int inputSize) { + meterIncrementEvent(getMetrics().uncompressedBytesIn, inputSize); + } + + @Override + public void onUncompressedBytesOut(int inputSize) { + meterIncrementEvent(getMetrics().uncompressedBytesOut, inputSize); + } + + @Override + public void onFinalBytesOut(int inputSize) { + meterIncrementEvent(getMetrics().finalPayloadBytesOut, inputSize); + } + + @Override + public void onTransformSuccess() { + meterIncrementEvent(getMetrics().transformSuccess); + } + + @Override + public void onTransformSkip() { + meterIncrementEvent(getMetrics().transformSkipped); + } + + @Override + public void onTransformFailure() { + meterIncrementEvent(getMetrics().transformError); + } + + @Override + public void aggregateInputChunk(int sizeInBytes) { + meterIncrementEvent(getMetrics().transformBytesIn, sizeInBytes); + meterIncrementEvent(getMetrics().transformChunksIn); + } + + @Override + public void aggregateOutputChunk(int sizeInBytes) { + meterIncrementEvent(getMetrics().transformBytesOut, sizeInBytes); + meterIncrementEvent(getMetrics().transformChunksOut); + } + } + + public static class ScheduledContext + extends DirectNestedSpanContext + implements IReplayContexts.IScheduledContext { + private final long scheduledForNanoTime; + + public ScheduledContext(HttpTransactionContext enclosingScope, long scheduledForNanoTime) { + super(enclosingScope); + this.scheduledForNanoTime = scheduledForNanoTime; + initializeSpan(); + } + + public static class MetricInstruments extends CommonScopedMetricInstruments { + DoubleHistogram lag; + + private MetricInstruments(Meter meter, String activityName) { + super(meter, activityName); + lag = meter.histogramBuilder(MetricNames.NETTY_SCHEDULE_LAG).setUnit("ms").build(); + } + } + + public static @NonNull MetricInstruments makeMetrics(Meter meter) { + return new MetricInstruments(meter, ACTIVITY_NAME); + } + + public @NonNull MetricInstruments getMetrics() { + return getRootInstrumentationScope().scheduledInstruments; + } + + @Override + public void sendMeterEventsForEnd() { + super.sendMeterEventsForEnd(); + meterHistogramMillis(getMetrics().lag, + Duration.ofNanos(Math.max(0, System.nanoTime() - scheduledForNanoTime))); + } + } + + public static class TargetRequestContext + extends DirectNestedSpanContext + implements IReplayContexts.ITargetRequestContext { + public TargetRequestContext(HttpTransactionContext enclosingScope) { + super(enclosingScope); + initializeSpan(); + meterHistogramMillis(getMetrics().sourceTargetGap, + Duration.between(enclosingScope.getTimeOfOriginalRequest(), Instant.now())); + } + + public static class MetricInstruments extends CommonScopedMetricInstruments { + + private final DoubleHistogram sourceTargetGap; + private final LongCounter bytesWritten; + private final LongCounter bytesRead; + + private MetricInstruments(Meter meter, String activityName) { + super(meter, activityName); + sourceTargetGap = meter.histogramBuilder(MetricNames.SOURCE_TO_TARGET_REQUEST_LAG) + .setUnit("ms").build(); + bytesWritten = meter.counterBuilder(MetricNames.BYTES_WRITTEN_TO_TARGET) + .setUnit(BYTES_UNIT_STR).build(); + bytesRead = meter.counterBuilder(MetricNames.BYTES_READ_FROM_TARGET) + .setUnit(BYTES_UNIT_STR).build(); + } + } + + public static @NonNull MetricInstruments makeMetrics(Meter meter) { + return new MetricInstruments(meter, ACTIVITY_NAME); + } + + public @NonNull MetricInstruments getMetrics() { + return getRootInstrumentationScope().targetRequestInstruments; + } + + @Override + public void onBytesSent(int size) { + meterIncrementEvent(getMetrics().bytesWritten, size); + } + + @Override + public void onBytesReceived(int size) { + meterIncrementEvent(getMetrics().bytesRead, size); + } + + @Override + public IRequestSendingContext createHttpSendingContext() { + return new ReplayContexts.RequestSendingContext(this); + } + + @Override + public IReplayContexts.IReceivingHttpResponseContext createHttpReceivingContext() { + return new ReplayContexts.ReceivingHttpResponseContext(this); + } + + @Override + public IReplayContexts.IWaitingForHttpResponseContext createWaitingForResponseContext() { + return new ReplayContexts.WaitingForHttpResponseContext(this); + } + } + + public static class RequestSendingContext + extends DirectNestedSpanContext + implements IReplayContexts.IRequestSendingContext { + public RequestSendingContext(TargetRequestContext enclosingScope) { + super(enclosingScope); + initializeSpan(); + } + + public static class MetricInstruments extends CommonScopedMetricInstruments { + private MetricInstruments(Meter meter, String activityName) { + super(meter, activityName); + } + } + + public static @NonNull MetricInstruments makeMetrics(Meter meter) { + return new MetricInstruments(meter, ACTIVITY_NAME); + } + + public @NonNull MetricInstruments getMetrics() { + return getRootInstrumentationScope().requestSendingInstruments; + } + } + + public static class WaitingForHttpResponseContext + extends DirectNestedSpanContext + implements IReplayContexts.IWaitingForHttpResponseContext { + public WaitingForHttpResponseContext(TargetRequestContext enclosingScope) { + super(enclosingScope); + initializeSpan(); + } + + public static class MetricInstruments extends CommonScopedMetricInstruments { + private MetricInstruments(Meter meter, String activityName) { + super(meter, activityName); + } + } + + public static @NonNull MetricInstruments makeMetrics(Meter meter) { + return new MetricInstruments(meter, ACTIVITY_NAME); + } + + public @NonNull MetricInstruments getMetrics() { + return getRootInstrumentationScope().waitingForHttpResponseInstruments; + } + + } + + public static class ReceivingHttpResponseContext + extends DirectNestedSpanContext + implements IReplayContexts.IReceivingHttpResponseContext { + public ReceivingHttpResponseContext(TargetRequestContext enclosingScope) { + super(enclosingScope); + initializeSpan(); + } + + public static class MetricInstruments extends CommonScopedMetricInstruments { + private MetricInstruments(Meter meter, String activityName) { + super(meter, activityName); + } + } + + public static @NonNull MetricInstruments makeMetrics(Meter meter) { + return new MetricInstruments(meter, ACTIVITY_NAME); + } + + public @NonNull MetricInstruments getMetrics() { + return getRootInstrumentationScope().receivingHttpInstruments; + } + + } + + @Getter + @Setter + public static class TupleHandlingContext + extends DirectNestedSpanContext + implements IReplayContexts.ITupleHandlingContext { + Integer sourceStatus; + Integer targetStatus; + String method; + + public TupleHandlingContext(HttpTransactionContext enclosingScope) { + super(enclosingScope); + initializeSpan(); + } + + public static class MetricInstruments extends CommonScopedMetricInstruments { + private final LongCounter resultCounter; + + private MetricInstruments(Meter meter, String activityName) { + super(meter, activityName); + resultCounter = meter.counterBuilder(MetricNames.TUPLE_COMPARISON).build(); + } + } + + public static @NonNull MetricInstruments makeMetrics(Meter meter) { + return new MetricInstruments(meter, ACTIVITY_NAME); + } + + public @NonNull MetricInstruments getMetrics() { + return getRootInstrumentationScope().tupleHandlingInstruments; + } + + static final AttributeKey TARGET_STATUS_CODE_ATTR = AttributeKey.longKey("targetStatusCode"); + + public AttributesBuilder getSharedAttributes(AttributesBuilder attributesBuilder) { + final var sourceOp = Optional.ofNullable(sourceStatus); + final var targetOp = Optional.ofNullable(targetStatus); + final boolean didMatch = sourceOp.flatMap(ss -> targetOp.map(ss::equals)).orElse(false); + return addAttributeIfPresent(addAttributeIfPresent(addAttributeIfPresent( + attributesBuilder, + METHOD_KEY, Optional.ofNullable(method)), + SOURCE_STATUS_CODE_KEY, sourceOp.map(TupleHandlingContext::categorizeStatus)), + TARGET_STATUS_CODE_KEY, targetOp.map(TupleHandlingContext::categorizeStatus)) + .put(STATUS_CODE_MATCH_KEY, didMatch); + } + + @Override + public AttributesBuilder fillExtraAttributesForThisSpan(AttributesBuilder builder) { + return getSharedAttributes(super.fillExtraAttributesForThisSpan(builder)); + } + + @Override + public void sendMeterEventsForEnd() { + super.sendMeterEventsForEnd(); + AttributesBuilder attributesBuilderForAggregate = getSharedAttributes(Attributes.builder()); + getCurrentSpan().setAllAttributes(attributesBuilderForAggregate.build()); + meterIncrementEvent(getMetrics().resultCounter, 1, attributesBuilderForAggregate); + } + + /** + * Convert everything in the 2xx range to 200; 300-399 to 300 + * + * @param status + * @return + */ + public static long categorizeStatus(int status) { + return (status / 100L) * 100L; + } + + /** + * Like httpVersion, Endpoint doesn't have a field because it isn't used as an attribute for metrics + * (it would create too much cardinality pressure). So just drop an attribute into a span instead of + * stashing it for both the span and final metric. + */ + @Override + public void setEndpoint(String endpointUrl) { + getCurrentSpan().setAttribute(ENDPOINT_KEY, endpointUrl); + } + + /** + * Like Endpoint, httpVersion doesn't have a field because it isn't used as an attribute for metrics + * (it just isn't expected to be super-useful and could create too much cardinality pressure). + * So just drop an attribute into a span instead of stashing it for both the span and final metric. + */ + @Override + public void setHttpVersion(String httpVersion) { + getCurrentSpan().setAttribute(HTTP_VERSION_KEY, httpVersion); + } + + @Override + public String toString() { + return getReplayerRequestKey().toString(); + } + } +} diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/RootReplayerContext.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/RootReplayerContext.java new file mode 100644 index 000000000..8fe51cacf --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/RootReplayerContext.java @@ -0,0 +1,94 @@ +package org.opensearch.migrations.replay.tracing; + +import io.opentelemetry.api.OpenTelemetry; +import lombok.Getter; +import org.opensearch.migrations.replay.datatypes.ISourceTrafficChannelKey; +import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; +import org.opensearch.migrations.replay.traffic.source.InputStreamOfTraffic; +import org.opensearch.migrations.tracing.RootOtelContext; + +@Getter +public class RootReplayerContext extends RootOtelContext implements IRootReplayerContext { + public static final String SCOPE_NAME = "replayer"; + + public final KafkaConsumerContexts.AsyncListeningContext.MetricInstruments asyncListeningInstruments; + public final KafkaConsumerContexts.TouchScopeContext.MetricInstruments touchInstruments; + public final KafkaConsumerContexts.PollScopeContext.MetricInstruments pollInstruments; + public final KafkaConsumerContexts.CommitScopeContext.MetricInstruments commitInstruments; + public final KafkaConsumerContexts.KafkaCommitScopeContext.MetricInstruments kafkaCommitInstruments; + + public final TrafficSourceContexts.ReadChunkContext.MetricInstruments readChunkInstruments; + public final TrafficSourceContexts.BackPressureBlockContext.MetricInstruments backPressureInstruments; + public final TrafficSourceContexts.WaitForNextSignal.MetricInstruments waitForNextSignalInstruments; + + public final ReplayContexts.ChannelKeyContext.MetricInstruments channelKeyInstruments; + public final ReplayContexts.KafkaRecordContext.MetricInstruments kafkaRecordInstruments; + public final ReplayContexts.TrafficStreamLifecycleContext.MetricInstruments trafficStreamLifecycleInstruments; + public final ReplayContexts.HttpTransactionContext.MetricInstruments httpTransactionInstruments; + public final ReplayContexts.RequestAccumulationContext.MetricInstruments requestAccumInstruments; + public final ReplayContexts.ResponseAccumulationContext.MetricInstruments responseAccumInstruments; + public final ReplayContexts.RequestTransformationContext.MetricInstruments transformationInstruments; + public final ReplayContexts.ScheduledContext.MetricInstruments scheduledInstruments; + public final ReplayContexts.TargetRequestContext.MetricInstruments targetRequestInstruments; + public final ReplayContexts.RequestSendingContext.MetricInstruments requestSendingInstruments; + public final ReplayContexts.WaitingForHttpResponseContext.MetricInstruments waitingForHttpResponseInstruments; + public final ReplayContexts.ReceivingHttpResponseContext.MetricInstruments receivingHttpInstruments; + public final ReplayContexts.TupleHandlingContext.MetricInstruments tupleHandlingInstruments; + public final ReplayContexts.SocketContext.MetricInstruments socketInstruments; + + public RootReplayerContext(OpenTelemetry sdk) { + super(SCOPE_NAME, sdk); + var meter = this.getMeterProvider().get(SCOPE_NAME); + + asyncListeningInstruments = KafkaConsumerContexts.AsyncListeningContext.makeMetrics(meter); + touchInstruments = KafkaConsumerContexts.TouchScopeContext.makeMetrics(meter); + pollInstruments = KafkaConsumerContexts.PollScopeContext.makeMetrics(meter); + commitInstruments = KafkaConsumerContexts.CommitScopeContext.makeMetrics(meter); + kafkaCommitInstruments = KafkaConsumerContexts.KafkaCommitScopeContext.makeMetrics(meter); + + readChunkInstruments = TrafficSourceContexts.ReadChunkContext.makeMetrics(meter); + backPressureInstruments = TrafficSourceContexts.BackPressureBlockContext.makeMetrics(meter); + waitForNextSignalInstruments = TrafficSourceContexts.WaitForNextSignal.makeMetrics(meter); + + channelKeyInstruments = ReplayContexts.ChannelKeyContext.makeMetrics(meter); + socketInstruments = ReplayContexts.SocketContext.makeMetrics(meter); + kafkaRecordInstruments = ReplayContexts.KafkaRecordContext.makeMetrics(meter); + trafficStreamLifecycleInstruments = ReplayContexts.TrafficStreamLifecycleContext.makeMetrics(meter); + httpTransactionInstruments = ReplayContexts.HttpTransactionContext.makeMetrics(meter); + requestAccumInstruments = ReplayContexts.RequestAccumulationContext.makeMetrics(meter); + responseAccumInstruments = ReplayContexts.ResponseAccumulationContext.makeMetrics(meter); + transformationInstruments = ReplayContexts.RequestTransformationContext.makeMetrics(meter); + scheduledInstruments = ReplayContexts.ScheduledContext.makeMetrics(meter); + targetRequestInstruments = ReplayContexts.TargetRequestContext.makeMetrics(meter); + requestSendingInstruments = ReplayContexts.RequestSendingContext.makeMetrics(meter); + waitingForHttpResponseInstruments = ReplayContexts.WaitingForHttpResponseContext.makeMetrics(meter); + receivingHttpInstruments = ReplayContexts.ReceivingHttpResponseContext.makeMetrics(meter); + tupleHandlingInstruments = ReplayContexts.TupleHandlingContext.makeMetrics(meter); + } + + @Override + public TrafficSourceContexts.ReadChunkContext createReadChunkContext() { + return new TrafficSourceContexts.ReadChunkContext(this, null); + } + + public IReplayContexts.IChannelKeyContext createChannelContext(ISourceTrafficChannelKey tsk) { + return new ReplayContexts.ChannelKeyContext(this, null, tsk); + } + + public IKafkaConsumerContexts.ICommitScopeContext createCommitContext() { + return new KafkaConsumerContexts.CommitScopeContext(this, null); + } + + public IReplayContexts.ITrafficStreamsLifecycleContext + createTrafficStreamContextForStreamSource(IReplayContexts.IChannelKeyContext channelCtx, + ITrafficStreamKey tsk) { + return new InputStreamOfTraffic.IOSTrafficStreamContext(this, channelCtx, tsk); + } + + public IReplayContexts.IKafkaRecordContext + createTrafficStreamContextForKafkaSource(IReplayContexts.IChannelKeyContext channelCtx, + String recordId, + int kafkaRecordSize) { + return new ReplayContexts.KafkaRecordContext(this, channelCtx, recordId, kafkaRecordSize); + } +} diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/TrafficSourceContexts.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/TrafficSourceContexts.java new file mode 100644 index 000000000..7f6663f21 --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/tracing/TrafficSourceContexts.java @@ -0,0 +1,118 @@ +package org.opensearch.migrations.replay.tracing; + +import io.opentelemetry.api.metrics.Meter; +import lombok.NonNull; +import org.opensearch.migrations.tracing.BaseNestedSpanContext; +import org.opensearch.migrations.tracing.CommonScopedMetricInstruments; +import org.opensearch.migrations.tracing.IInstrumentationAttributes; +import org.opensearch.migrations.tracing.IScopedInstrumentationAttributes; + +public class TrafficSourceContexts { + + private TrafficSourceContexts() {} + + public static class ReadChunkContext + extends BaseNestedSpanContext + implements ITrafficSourceContexts.IReadChunkContext + { + @Override + public ITrafficSourceContexts.IBackPressureBlockContext createBackPressureContext() { + return new TrafficSourceContexts.BackPressureBlockContext(getRootInstrumentationScope(), this); + } + + @Override + public IKafkaConsumerContexts.IPollScopeContext createPollContext() { + return new KafkaConsumerContexts.PollScopeContext(getRootInstrumentationScope(), this); + } + + @Override + public IKafkaConsumerContexts.ICommitScopeContext createCommitContext() { + return new KafkaConsumerContexts.CommitScopeContext(getRootInstrumentationScope(), this); + } + + public static class MetricInstruments extends CommonScopedMetricInstruments { + private MetricInstruments(Meter meter, String activityName) { + super(meter, activityName); + } + } + + public static @NonNull MetricInstruments makeMetrics(Meter meter) { + return new MetricInstruments(meter, ACTIVITY_NAME); + } + + public @NonNull MetricInstruments getMetrics() { + return getRootInstrumentationScope().readChunkInstruments; + } + + public ReadChunkContext(RootReplayerContext rootScope, IScopedInstrumentationAttributes enclosingScope) { + super(rootScope, enclosingScope); + initializeSpan(); + } + + } + + public static class BackPressureBlockContext + extends BaseNestedSpanContext + implements ITrafficSourceContexts.IBackPressureBlockContext + { + @Override + public ITrafficSourceContexts.IWaitForNextSignal createWaitForSignalContext() { + return new TrafficSourceContexts.WaitForNextSignal(getRootInstrumentationScope(), this); + } + + @Override + public IKafkaConsumerContexts.ITouchScopeContext createNewTouchContext() { + return new KafkaConsumerContexts.TouchScopeContext(this); + } + + @Override + public IKafkaConsumerContexts.ICommitScopeContext createCommitContext() { + return new KafkaConsumerContexts.CommitScopeContext(getRootInstrumentationScope(), this); + } + + public static class MetricInstruments extends CommonScopedMetricInstruments { + private MetricInstruments(Meter meter, String activityName) { + super(meter, activityName); + } + } + + public static @NonNull MetricInstruments makeMetrics(Meter meter) { + return new MetricInstruments(meter, ACTIVITY_NAME); + } + + public @NonNull MetricInstruments getMetrics() { + return getRootInstrumentationScope().backPressureInstruments; + } + + public BackPressureBlockContext(@NonNull RootReplayerContext rootScope, + @NonNull ITrafficSourceContexts.IReadChunkContext enclosingScope) { + super(rootScope, enclosingScope); + initializeSpan(); + } + } + + public static class WaitForNextSignal + extends BaseNestedSpanContext + implements ITrafficSourceContexts.IWaitForNextSignal { + public static class MetricInstruments extends CommonScopedMetricInstruments { + private MetricInstruments(Meter meter, String activityName) { + super(meter, activityName); + } + } + + public static @NonNull MetricInstruments makeMetrics(Meter meter) { + return new MetricInstruments(meter, ACTIVITY_NAME); + } + + public @NonNull MetricInstruments getMetrics() { + return getRootInstrumentationScope().waitForNextSignalInstruments; + } + + public WaitForNextSignal(@NonNull RootReplayerContext rootScope, + @NonNull ITrafficSourceContexts.IBackPressureBlockContext enclosingScope) { + super(rootScope, enclosingScope); + initializeSpan(); + } + } + +} diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/BlockingTrafficSource.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/BlockingTrafficSource.java index ddf838e27..9d4ce61e7 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/BlockingTrafficSource.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/BlockingTrafficSource.java @@ -1,9 +1,12 @@ package org.opensearch.migrations.replay.traffic.source; import com.google.protobuf.Timestamp; +import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.opensearch.migrations.replay.Utils; import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; +import org.opensearch.migrations.replay.tracing.ITrafficSourceContexts; +import org.opensearch.migrations.trafficcapture.protos.TrafficObservation; import org.opensearch.migrations.trafficcapture.protos.TrafficStreamUtils; import org.slf4j.event.Level; @@ -17,6 +20,7 @@ import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Supplier; /** * The BlockingTrafficSource class implements ITrafficCaptureSource and wraps another instance. @@ -41,11 +45,10 @@ public class BlockingTrafficSource implements ITrafficCaptureSource, BufferedFlo * Limit the number of readers to one at a time and only if we haven't yet maxed out our time buffer */ private final Semaphore readGate; + @Getter private final Duration bufferTimeWindow; - - public BlockingTrafficSource(ISimpleTrafficCaptureSource underlying, - Duration bufferTimeWindow) { + public BlockingTrafficSource(ISimpleTrafficCaptureSource underlying, Duration bufferTimeWindow) { this.underlyingSource = underlying; this.stopReadingAtRef = new AtomicReference<>(Instant.EPOCH); this.lastTimestampSecondsRef = new AtomicReference<>(Instant.EPOCH); @@ -77,47 +80,47 @@ public void stopReadsPast(Instant pointInTime) { } } - public Duration getBufferTimeWindow() { - return bufferTimeWindow; - } - /** * Reads the next chunk that is available before the current stopReading barrier. However, * that barrier isn't meant to be a tight barrier with immediate effect. - * - * @return */ - @Override public CompletableFuture> - readNextTrafficStreamChunk() { + readNextTrafficStreamChunk(Supplier readChunkContextSupplier) { + var readContext = readChunkContextSupplier.get(); log.info("BlockingTrafficSource::readNext"); var trafficStreamListFuture = CompletableFuture - .supplyAsync(this::blockIfNeeded, task -> new Thread(task).start()) - .thenCompose(v->{ + .supplyAsync(() -> blockIfNeeded(readContext), task -> new Thread(task).start()) + .thenCompose(v -> { log.info("BlockingTrafficSource::composing"); - return underlyingSource.readNextTrafficStreamChunk(); - }); - return trafficStreamListFuture.whenComplete((v,t)->{ + return underlyingSource.readNextTrafficStreamChunk(()->readContext); + }) + .whenComplete((v,t)->readContext.close()); + return trafficStreamListFuture.whenComplete((v, t) -> { if (t != null) { return; } - var maxLocallyObservedTimestamp = v.stream().flatMap(tswk->tswk.getStream().getSubStreamList().stream()) - .map(tso->tso.getTs()) + var maxLocallyObservedTimestamp = v.stream() + .flatMap(tswk -> tswk.getStream().getSubStreamList().stream()) + .map(TrafficObservation::getTs) .max(Comparator.comparingLong(Timestamp::getSeconds) .thenComparingInt(Timestamp::getNanos)) .map(TrafficStreamUtils::instantFromProtoTimestamp) .orElse(Instant.EPOCH); Utils.setIfLater(lastTimestampSecondsRef, maxLocallyObservedTimestamp); - log.atTrace().setMessage(()->"end of readNextTrafficStreamChunk trigger...lastTimestampSecondsRef=" - +lastTimestampSecondsRef.get()).log(); + log.atTrace().setMessage(() -> "end of readNextTrafficStreamChunk trigger...lastTimestampSecondsRef=" + + lastTimestampSecondsRef.get()).log(); }); } - private Void blockIfNeeded() { + private Void blockIfNeeded(ITrafficSourceContexts.IReadChunkContext readContext) { if (stopReadingAtRef.get().equals(Instant.EPOCH)) { return null; } - log.atInfo().setMessage(()->"stopReadingAtRef="+stopReadingAtRef+ - " lastTimestampSecondsRef="+lastTimestampSecondsRef).log(); + log.atInfo().setMessage(() -> "stopReadingAtRef=" + stopReadingAtRef + + " lastTimestampSecondsRef=" + lastTimestampSecondsRef).log(); + ITrafficSourceContexts.IBackPressureBlockContext blockContext = null; while (stopReadingAtRef.get().isBefore(lastTimestampSecondsRef.get())) { + if (blockContext == null) { + blockContext = readContext.createBackPressureContext(); + } try { log.atInfo().setMessage("blocking until signaled to read the next chunk last={} stop={}") .addArgument(lastTimestampSecondsRef.get()) @@ -125,21 +128,25 @@ private Void blockIfNeeded() { .log(); var nextTouchOp = underlyingSource.getNextRequiredTouch(); if (nextTouchOp.isEmpty()) { - log.trace("acquring readGate semaphore (w/out timeout)"); - readGate.acquire(); + log.trace("acquiring readGate semaphore (w/out timeout)"); + try (var waitContext = blockContext.createWaitForSignalContext()) { + readGate.acquire(); + } } else { var nextInstant = nextTouchOp.get(); final var nowTime = Instant.now(); var waitIntervalMs = Duration.between(nowTime, nextInstant).toMillis(); - log.atDebug().setMessage(()->"Next touch at " + nextInstant + - " ... in " + waitIntervalMs + "ms (now="+nowTime+")").log(); + log.atDebug().setMessage(() -> "Next touch at " + nextInstant + + " ... in " + waitIntervalMs + "ms (now=" + nowTime + ")").log(); if (waitIntervalMs <= 0) { - underlyingSource.touch(); + underlyingSource.touch(blockContext); } else { // if this doesn't succeed, we'll loop around & likely do a touch, then loop around again. // if it DOES succeed, we'll loop around and make sure that there's not another reason to stop - log.atTrace().setMessage(()->"acquring readGate semaphore with timeout="+waitIntervalMs).log(); - readGate.tryAcquire(waitIntervalMs, TimeUnit.MILLISECONDS); + log.atTrace().setMessage(() -> "acquring readGate semaphore with timeout=" + waitIntervalMs).log(); + try (var waitContext = blockContext.createWaitForSignalContext()) { + readGate.tryAcquire(waitIntervalMs, TimeUnit.MILLISECONDS); + } } } } catch (InterruptedException e) { @@ -148,12 +155,20 @@ private Void blockIfNeeded() { break; } } + if (blockContext != null) { + blockContext.close(); + } return null; } @Override - public void commitTrafficStream(ITrafficStreamKey trafficStreamKey) throws IOException { - underlyingSource.commitTrafficStream(trafficStreamKey); + public CommitResult commitTrafficStream(ITrafficStreamKey trafficStreamKey) throws IOException { + var commitResult = underlyingSource.commitTrafficStream(trafficStreamKey); + if (commitResult == CommitResult.AfterNextRead) { + readGate.drainPermits(); + readGate.release(); + } + return commitResult; } @Override diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/ITrafficCaptureSource.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/ITrafficCaptureSource.java index 625bde671..89353ebe0 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/ITrafficCaptureSource.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/ITrafficCaptureSource.java @@ -1,20 +1,26 @@ package org.opensearch.migrations.replay.traffic.source; import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; +import org.opensearch.migrations.replay.tracing.ITrafficSourceContexts; import java.io.Closeable; import java.io.IOException; -import java.time.Duration; import java.time.Instant; import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.function.Supplier; public interface ITrafficCaptureSource extends Closeable { - CompletableFuture> readNextTrafficStreamChunk(); + enum CommitResult { + Immediate, AfterNextRead, BlockedByOtherCommits, Ignored + } - default void commitTrafficStream(ITrafficStreamKey trafficStreamKey) throws IOException {} + CompletableFuture> + readNextTrafficStreamChunk(Supplier contextSupplier); + + CommitResult commitTrafficStream(ITrafficStreamKey trafficStreamKey) throws IOException; default void close() throws IOException {} @@ -22,7 +28,7 @@ default void close() throws IOException {} * Keep-alive call to be used by the BlockingTrafficSource to keep this connection alive if * this is required. */ - default void touch() {} + default void touch(ITrafficSourceContexts.IBackPressureBlockContext context) {} /** * @return The time that the next call to touch() must be completed for this source to stay diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/InputStreamOfTraffic.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/InputStreamOfTraffic.java index 4f9cd53c5..667b18201 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/InputStreamOfTraffic.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/InputStreamOfTraffic.java @@ -3,6 +3,13 @@ import lombok.Lombok; import lombok.extern.slf4j.Slf4j; import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; +import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamAndKey; +import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamKeyAndContext; +import org.opensearch.migrations.replay.tracing.ChannelContextManager; +import org.opensearch.migrations.replay.tracing.IReplayContexts; +import org.opensearch.migrations.replay.tracing.ITrafficSourceContexts; +import org.opensearch.migrations.replay.tracing.ReplayContexts; +import org.opensearch.migrations.replay.tracing.RootReplayerContext; import org.opensearch.migrations.trafficcapture.protos.TrafficStream; import java.io.EOFException; @@ -11,23 +18,34 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; @Slf4j public class InputStreamOfTraffic implements ISimpleTrafficCaptureSource { private final InputStream inputStream; private final AtomicInteger trafficStreamsRead = new AtomicInteger(); + private final ChannelContextManager channelContextManager; - public InputStreamOfTraffic(InputStream inputStream) { + public InputStreamOfTraffic(RootReplayerContext context, InputStream inputStream) { + this.channelContextManager = new ChannelContextManager(context); this.inputStream = inputStream; } + public static final class IOSTrafficStreamContext extends ReplayContexts.TrafficStreamLifecycleContext { + public IOSTrafficStreamContext(RootReplayerContext rootReplayerContext, + IReplayContexts.IChannelKeyContext enclosingScope, + ITrafficStreamKey trafficStreamKey) { + super(rootReplayerContext, enclosingScope, trafficStreamKey); + } + } + /** * Returns a CompletableFuture to a TrafficStream object or sets the cause exception to an * EOFException if the input has been exhausted. - * - * @return */ - public CompletableFuture> readNextTrafficStreamChunk() { + @Override + public CompletableFuture> + readNextTrafficStreamChunk(Supplier contextSupplier) { return CompletableFuture.supplyAsync(() -> { var builder = TrafficStream.newBuilder(); try { @@ -40,7 +58,12 @@ public CompletableFuture> readNextTrafficStreamChunk var ts = builder.build(); trafficStreamsRead.incrementAndGet(); log.trace("Parsed traffic stream #{}: {}", trafficStreamsRead.get(), ts); - return List.of(new TrafficStreamWithEmbeddedKey(ts)); + return List.of(new PojoTrafficStreamAndKey(ts, + PojoTrafficStreamKeyAndContext.build(ts, tsk-> { + var channelCtx = channelContextManager.retainOrCreateContext(tsk); + return channelContextManager.getGlobalContext() + .createTrafficStreamContextForStreamSource(channelCtx, tsk); + }))); }).exceptionally(e->{ var ecf = new CompletableFuture>(); ecf.completeExceptionally(e.getCause()); @@ -49,8 +72,10 @@ public CompletableFuture> readNextTrafficStreamChunk } @Override - public void commitTrafficStream(ITrafficStreamKey trafficStreamKey) { + public CommitResult commitTrafficStream(ITrafficStreamKey trafficStreamKey) { // do nothing - this datasource isn't transactional + channelContextManager.releaseContextFor(trafficStreamKey.getTrafficStreamsContext().getLogicalEnclosingScope()); + return CommitResult.Immediate; } @Override diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/TrafficStreamWithEmbeddedKey.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/TrafficStreamWithEmbeddedKey.java deleted file mode 100644 index 90cdb95cc..000000000 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/TrafficStreamWithEmbeddedKey.java +++ /dev/null @@ -1,23 +0,0 @@ -package org.opensearch.migrations.replay.traffic.source; - -import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; -import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamKey; -import org.opensearch.migrations.trafficcapture.protos.TrafficStream; - -public class TrafficStreamWithEmbeddedKey implements ITrafficStreamWithKey { - public final TrafficStream stream; - - public TrafficStreamWithEmbeddedKey(TrafficStream stream) { - this.stream = stream; - } - - @Override - public ITrafficStreamKey getKey() { - return new PojoTrafficStreamKey(stream); - } - - @Override - public TrafficStream getStream() { - return stream; - } -} diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java index a8b9452ff..bbe07ed6b 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java @@ -1,5 +1,7 @@ package org.opensearch.migrations.replay.util; +import lombok.extern.slf4j.Slf4j; + import java.util.ArrayList; import java.util.function.Consumer; @@ -21,6 +23,7 @@ * * @param */ +@Slf4j public class OnlineRadixSorter { ArrayList items; int currentOffset; @@ -31,8 +34,10 @@ public OnlineRadixSorter(int startingOffset) { } public void add(int index, T item, Consumer sortedItemVisitor) { + assert index >= currentOffset; if (currentOffset == index) { ++currentOffset; + log.atTrace().setMessage(()->"Running callback for "+index+": "+this).log(); sortedItemVisitor.accept(item); while (currentOffset < items.size()) { var nextItem = items.get(currentOffset); @@ -59,6 +64,7 @@ public boolean hasPending() { @Override public String toString() { final StringBuilder sb = new StringBuilder("OnlineRadixSorter{"); + sb.append("id=").append(System.identityHashCode(this)); sb.append("items=").append(items); sb.append(", currentOffset=").append(currentOffset); sb.append('}'); diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/TrafficChannelKeyFormatter.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/TrafficChannelKeyFormatter.java new file mode 100644 index 000000000..b0cb18148 --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/TrafficChannelKeyFormatter.java @@ -0,0 +1,24 @@ +package org.opensearch.migrations.replay.util; + +import java.util.StringJoiner; + +public class TrafficChannelKeyFormatter { + private TrafficChannelKeyFormatter() {} + + public static String format(String nodeId, String connectionId) { + return new StringJoiner(".") + .add(nodeId) + .add(connectionId) + .toString(); + } + + public static String format(String nodeId, String connectionId, int trafficStreamIndex) { + return new StringJoiner(".") + .add(nodeId) + .add(connectionId) + .add("" + trafficStreamIndex) + .toString(); + } +} + + diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/BlockingTrafficSourceTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/BlockingTrafficSourceTest.java index 3856676e5..f07249959 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/BlockingTrafficSourceTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/BlockingTrafficSourceTest.java @@ -5,11 +5,15 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; +import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamAndKey; +import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamKeyAndContext; +import org.opensearch.migrations.replay.tracing.ITrafficSourceContexts; import org.opensearch.migrations.replay.traffic.source.BlockingTrafficSource; import org.opensearch.migrations.replay.traffic.source.ISimpleTrafficCaptureSource; import org.opensearch.migrations.replay.traffic.source.ITrafficStreamWithKey; -import org.opensearch.migrations.replay.traffic.source.TrafficStreamWithEmbeddedKey; import org.opensearch.migrations.testutils.WrapWithNettyLeakDetection; +import org.opensearch.migrations.tracing.InstrumentationTest; +import org.opensearch.migrations.tracing.TestContext; import org.opensearch.migrations.trafficcapture.protos.CloseObservation; import org.opensearch.migrations.trafficcapture.protos.TrafficObservation; import org.opensearch.migrations.trafficcapture.protos.TrafficStream; @@ -25,10 +29,11 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; @Slf4j @WrapWithNettyLeakDetection(disableLeakChecks = true) -class BlockingTrafficSourceTest { +class BlockingTrafficSourceTest extends InstrumentationTest { private static final Instant sourceStartTime = Instant.EPOCH; public static final int SHIFT = 1; @@ -36,13 +41,13 @@ class BlockingTrafficSourceTest { void readNextChunkTest() throws Exception { var nStreamsToCreate = 210; var BUFFER_MILLIS = 10; - var testSource = new TestTrafficCaptureSource(nStreamsToCreate); + var testSource = new TestTrafficCaptureSource(rootContext, nStreamsToCreate); var blockingSource = new BlockingTrafficSource(testSource, Duration.ofMillis(BUFFER_MILLIS)); blockingSource.stopReadsPast(sourceStartTime.plus(Duration.ofMillis(0))); var firstChunk = new ArrayList(); for (int i = 0; i<=BUFFER_MILLIS+SHIFT; ++i) { - var nextPieceFuture = blockingSource.readNextTrafficStreamChunk(); + var nextPieceFuture = blockingSource.readNextTrafficStreamChunk(rootContext::createReadChunkContext); nextPieceFuture.get(500000, TimeUnit.MILLISECONDS) .forEach(ts->firstChunk.add(ts)); } @@ -50,9 +55,10 @@ void readNextChunkTest() throws Exception { Assertions.assertTrue(BUFFER_MILLIS+SHIFT <= firstChunk.size()); Instant lastTime = null; for (int i =SHIFT; i> readNextTrafficStreamChunk() { + public CompletableFuture> + readNextTrafficStreamChunk(Supplier readChunkContextSupplier) { if (numberOfTrafficStreamsToRead.get() <= 0) { return CompletableFuture.failedFuture(new EOFException()); } - return trafficSource.readNextTrafficStreamChunk() + return trafficSource.readNextTrafficStreamChunk(readChunkContextSupplier) .thenApply(ltswk -> { - var transformed = ltswk.stream().map(this::modifyTrafficStream).collect(Collectors.toList()); + var transformedTrafficStream = ltswk.stream().map(this::modifyTrafficStream).collect(Collectors.toList()); var oldValue = numberOfTrafficStreamsToRead.get(); - var newValue = oldValue - transformed.size(); + var newValue = oldValue - transformedTrafficStream.size(); var exchangeResult = numberOfTrafficStreamsToRead.compareAndExchange(oldValue, newValue); assert exchangeResult == oldValue : "didn't expect to be running with a race condition here"; - return transformed; + return transformedTrafficStream; }); } diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/ExhaustiveCapturedTrafficToHttpTransactionAccumulatorTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/ExhaustiveCapturedTrafficToHttpTransactionAccumulatorTest.java index 8294aec17..df9d74c94 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/ExhaustiveCapturedTrafficToHttpTransactionAccumulatorTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/ExhaustiveCapturedTrafficToHttpTransactionAccumulatorTest.java @@ -1,13 +1,12 @@ package org.opensearch.migrations.replay; -import lombok.AllArgsConstructor; -import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; -import org.opensearch.migrations.trafficcapture.protos.TrafficObservation; +import org.opensearch.migrations.tracing.InstrumentationTest; +import org.opensearch.migrations.tracing.TestContext; import org.opensearch.migrations.trafficcapture.protos.TrafficStream; import org.opensearch.migrations.trafficcapture.protos.TrafficStreamUtils; @@ -15,25 +14,22 @@ import java.util.Arrays; import java.util.HashSet; import java.util.List; -import java.util.Map; -import java.util.Optional; import java.util.Random; import java.util.StringJoiner; import java.util.TreeSet; import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.IntStream; -import java.util.stream.Stream; @Slf4j -public class ExhaustiveCapturedTrafficToHttpTransactionAccumulatorTest { +public class ExhaustiveCapturedTrafficToHttpTransactionAccumulatorTest extends InstrumentationTest { public static Arguments[] generateAllTestsAndConfirmComplete(IntStream seedStream) { + var rootContext = TestContext.noOtelTracking(); var possibilitiesLeftToTest = TrafficStreamGenerator.getPossibleTests(); var numTries = new AtomicInteger(); StringJoiner seedsThatOfferUniqueTestCases = new StringJoiner(","); - var argsArray = TrafficStreamGenerator.generateRandomTrafficStreamsAndSizes(seedStream) + var argsArray = TrafficStreamGenerator.generateRandomTrafficStreamsAndSizes(rootContext, seedStream) .takeWhile(c->!possibilitiesLeftToTest.isEmpty()) .filter(c->TrafficStreamGenerator.classifyTrafficStream(possibilitiesLeftToTest, c.trafficStreams) > 0) .flatMap(c-> { @@ -91,13 +87,13 @@ void accumulateWithAccumulatorPairAtPoint(TrafficStream[] trafficStreams, int cu // in the first pass. // // Notice that this may cause duplicates. That's by design. The system has an at-least-once guarantee. - var indicesProcessedPass1 = - SimpleCapturedTrafficToHttpTransactionAccumulatorTest.accumulateTrafficStreamsWithNewAccumulator( + var indicesProcessedPass1 = SimpleCapturedTrafficToHttpTransactionAccumulatorTest + .accumulateTrafficStreamsWithNewAccumulator(rootContext, Arrays.stream(trafficStreams).limit(cutPoint), reconstructedTransactions, requestsReceived); cutPoint = indicesProcessedPass1.isEmpty() ? 0 : indicesProcessedPass1.last(); - var indicesProcessedPass2 = - SimpleCapturedTrafficToHttpTransactionAccumulatorTest.accumulateTrafficStreamsWithNewAccumulator( - Arrays.stream(trafficStreams).skip(cutPoint), reconstructedTransactions, requestsReceived); + var indicesProcessedPass2 = SimpleCapturedTrafficToHttpTransactionAccumulatorTest + .accumulateTrafficStreamsWithNewAccumulator(rootContext, + Arrays.stream(trafficStreams).skip(cutPoint), reconstructedTransactions, requestsReceived); // three checks to do w/ the indicesProcessed sets. // Count their sum, confirm that there were not duplicates, confirm all match the input indices @@ -112,5 +108,4 @@ void accumulateWithAccumulatorPairAtPoint(TrafficStream[] trafficStreams, int cu SimpleCapturedTrafficToHttpTransactionAccumulatorTest.assertReconstructedTransactionsMatchExpectations( reconstructedTransactions, expectedRequestSizes, expectedResponseSizes); } - } diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/ExpiringTrafficStreamMapSequentialTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/ExpiringTrafficStreamMapSequentialTest.java index b74e23606..d852f67bf 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/ExpiringTrafficStreamMapSequentialTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/ExpiringTrafficStreamMapSequentialTest.java @@ -2,9 +2,10 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamKey; +import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamKeyAndContext; import org.opensearch.migrations.replay.traffic.expiration.BehavioralPolicy; import org.opensearch.migrations.replay.traffic.expiration.ExpiringTrafficStreamMap; +import org.opensearch.migrations.tracing.InstrumentationTest; import java.nio.charset.StandardCharsets; import java.time.Duration; @@ -14,11 +15,11 @@ import java.util.function.Function; import java.util.stream.Collectors; -class ExpiringTrafficStreamMapSequentialTest { +class ExpiringTrafficStreamMapSequentialTest extends InstrumentationTest { public static final String TEST_NODE_ID_STRING = "test_node_id"; - public static void testLinearExpirations(Function connectionGenerator, int window, int granularity, + public void testLinearExpirations(Function connectionGenerator, int window, int granularity, int expectedExpirationCounts[]) { var expiredAccumulations = new ArrayList(); var expiringMap = new ExpiringTrafficStreamMap(Duration.ofSeconds(window), Duration.ofSeconds(granularity), @@ -33,12 +34,16 @@ public void onExpireAccumulation(String partitionId, var expiredCountsPerLoop = new ArrayList(); for (int i=0; inew Accumulation(tsk, 0)); createdAccumulations.add(accumulation); - expiringMap.expireOldEntries(new PojoTrafficStreamKey(TEST_NODE_ID_STRING, connectionGenerator.apply(i), 0), + expiringMap.expireOldEntries(PojoTrafficStreamKeyAndContext.build(TEST_NODE_ID_STRING, + connectionGenerator.apply(i), 0, rootContext::createTrafficStreamContextForTest), accumulation, ts); - var rrPair = createdAccumulations.get(i).getOrCreateTransactionPair(new PojoTrafficStreamKey("n","c",1)); + var rrPair = createdAccumulations.get(i).getOrCreateTransactionPair( + PojoTrafficStreamKeyAndContext.build("n","c",1, + rootContext::createTrafficStreamContextForTest), Instant.EPOCH); rrPair.addResponseData(ts, ("Add"+i).getBytes(StandardCharsets.UTF_8)); expiredCountsPerLoop.add(expiredAccumulations.size()); } diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/ExpiringTrafficStreamMapUnorderedTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/ExpiringTrafficStreamMapUnorderedTest.java index b214cdc5e..d1681338a 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/ExpiringTrafficStreamMapUnorderedTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/ExpiringTrafficStreamMapUnorderedTest.java @@ -2,10 +2,11 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamKey; +import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamKeyAndContext; import org.opensearch.migrations.replay.traffic.expiration.BehavioralPolicy; import org.opensearch.migrations.replay.traffic.expiration.ExpiringTrafficStreamMap; import org.opensearch.migrations.testutils.WrapWithNettyLeakDetection; +import org.opensearch.migrations.tracing.InstrumentationTest; import java.nio.charset.StandardCharsets; import java.time.Duration; @@ -16,7 +17,7 @@ import java.util.stream.Collectors; @WrapWithNettyLeakDetection(disableLeakChecks = true) -class ExpiringTrafficStreamMapUnorderedTest { +class ExpiringTrafficStreamMapUnorderedTest extends InstrumentationTest { public static final String TEST_NODE_ID_STRING = "test_node_id"; @@ -34,21 +35,26 @@ public void onExpireAccumulation(String partitionId, }); var createdAccumulations = new ArrayList(); var expiredCountsPerLoop = new ArrayList(); - for (int i=0; inew Accumulation(tsk, 0)); - expiringMap.expireOldEntries(new PojoTrafficStreamKey(TEST_NODE_ID_STRING, connectionGenerator.apply(i), 0), accumulation, ts); + var tsk = PojoTrafficStreamKeyAndContext.build(TEST_NODE_ID_STRING, connectionGenerator.apply(i), 0, + rootContext::createTrafficStreamContextForTest); + var accumulation = expiringMap.getOrCreateWithoutExpiration(tsk, k -> new Accumulation(tsk, 0)); + expiringMap.expireOldEntries(PojoTrafficStreamKeyAndContext.build(TEST_NODE_ID_STRING, + connectionGenerator.apply(i), 0, + rootContext::createTrafficStreamContextForTest), + accumulation, ts); createdAccumulations.add(accumulation); if (accumulation != null) { - var rrPair = accumulation.getOrCreateTransactionPair(new PojoTrafficStreamKey("n","c",1)); + var rrPair = accumulation.getOrCreateTransactionPair(PojoTrafficStreamKeyAndContext.build("n", "c", 1, + rootContext::createTrafficStreamContextForTest), Instant.EPOCH); rrPair.addResponseData(ts, ("Add" + i).getBytes(StandardCharsets.UTF_8)); } expiredCountsPerLoop.add(expiredAccumulations.size()); } Assertions.assertEquals( - Arrays.stream(expectedExpirationCounts).mapToObj(i->""+i).collect(Collectors.joining()), - expiredCountsPerLoop.stream().map(i->""+i).collect(Collectors.joining())); + Arrays.stream(expectedExpirationCounts).mapToObj(i -> "" + i).collect(Collectors.joining()), + expiredCountsPerLoop.stream().map(i -> "" + i).collect(Collectors.joining())); } @Test diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/FullReplayerWithTracingChecksTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/FullReplayerWithTracingChecksTest.java new file mode 100644 index 000000000..74fbb8c49 --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/FullReplayerWithTracingChecksTest.java @@ -0,0 +1,158 @@ +package org.opensearch.migrations.replay; + +import com.google.protobuf.ByteString; +import com.google.protobuf.Timestamp; +import io.opentelemetry.sdk.testing.exporter.InMemorySpanExporter; +import io.opentelemetry.sdk.trace.data.SpanData; +import lombok.extern.slf4j.Slf4j; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import org.opensearch.migrations.replay.traffic.source.BlockingTrafficSource; +import org.opensearch.migrations.testutils.SimpleNettyHttpServer; +import org.opensearch.migrations.testutils.WrapWithNettyLeakDetection; +import org.opensearch.migrations.tracing.TestContext; +import org.opensearch.migrations.trafficcapture.protos.CloseObservation; +import org.opensearch.migrations.trafficcapture.protos.EndOfMessageIndication; +import org.opensearch.migrations.trafficcapture.protos.ReadObservation; +import org.opensearch.migrations.trafficcapture.protos.TrafficObservation; +import org.opensearch.migrations.trafficcapture.protos.TrafficStream; +import org.opensearch.migrations.trafficcapture.protos.WriteObservation; +import org.opensearch.migrations.transform.StaticAuthTransformerFactory; + +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.time.Instant; +import java.util.HashSet; +import java.util.List; +import java.util.Random; +import java.util.function.BiConsumer; +import java.util.function.Consumer; +import java.util.stream.Collectors; + +@Slf4j +@WrapWithNettyLeakDetection(disableLeakChecks = true) +public class FullReplayerWithTracingChecksTest extends FullTrafficReplayerTest { + + @Override + protected TestContext makeInstrumentationContext() { + return TestContext.withAllTracking(); + } + + @Test + public void testSingleStreamWithCloseIsCommitted() throws Throwable { + var random = new Random(1); + var httpServer = SimpleNettyHttpServer.makeServer(false, Duration.ofMillis(2), + response -> TestHttpServerContext.makeResponse(random, response)); + var trafficStreamWithJustClose = TrafficStream.newBuilder() + .setNodeId(TEST_NODE_ID) + .setConnectionId(TEST_CONNECTION_ID) + .addSubStream(TrafficObservation.newBuilder() + .setClose(CloseObservation.newBuilder().build()).build()) + .build(); + var trafficSourceSupplier = new FullTrafficReplayerTest.ArrayCursorTrafficSourceFactory(List.of(trafficStreamWithJustClose)); + TrafficReplayerRunner.runReplayerUntilSourceWasExhausted(0, + httpServer.localhostEndpoint(), new FullTrafficReplayerTest.IndexWatchingListenerFactory(), + () -> TestContext.withAllTracking(), + trafficSourceSupplier); + Assertions.assertEquals(1, trafficSourceSupplier.nextReadCursor.get()); + log.info("done"); + } + + @ParameterizedTest + @ValueSource(ints = {1,2}) + public void testStreamWithRequestsWithCloseIsCommittedOnce(int numRequests) throws Throwable { + var random = new Random(1); + var httpServer = SimpleNettyHttpServer.makeServer(false, Duration.ofMillis(2), + response->TestHttpServerContext.makeResponse(random, response)); + var baseTime = Instant.now(); + var fixedTimestamp = + Timestamp.newBuilder().setSeconds(baseTime.getEpochSecond()).setNanos(baseTime.getNano()).build(); + var tsb = TrafficStream.newBuilder().setConnectionId("C"); + for (int i=0; i(); + try (var blockingTrafficSource = new BlockingTrafficSource(trafficSource, Duration.ofMinutes(2))) { + tr.setupRunAndWaitForReplayWithShutdownChecks(Duration.ofSeconds(70), blockingTrafficSource, + new TimeShifter(10 * 1000), (t) -> { + var wasNew = tuplesReceived.add(t.getRequestKey().toString()); + Assertions.assertTrue(wasNew); + }); + } finally { + tr.shutdown(null); + } + + Assertions.assertEquals(numRequests, tuplesReceived.size()); + checkSpansForSimpleReplayedTransactions(rootContext.inMemoryInstrumentationBundle.testSpanExporter, + numRequests); + log.info("done"); + } + + /** + * This function is written like this rather than with a loop so that the backtrace will show WHICH + * key was corrupted. + */ + private void checkSpansForSimpleReplayedTransactions(InMemorySpanExporter testSpanExporter, int numRequests) { + var byName = testSpanExporter.getFinishedSpanItems().stream().collect(Collectors.groupingBy(SpanData::getName)); + BiConsumer chk = (i, k) -> { + Assertions.assertNotNull(byName.get(k)); + Assertions.assertEquals(i, byName.get(k).size()); + byName.remove(k); + }; + chk.accept(1,"channel"); + chk.accept(1,"tcpConnection"); + chk.accept(1, "trafficStreamLifetime"); + chk.accept(numRequests, "httpTransaction"); + chk.accept(numRequests, "accumulatingRequest"); + chk.accept(numRequests, "accumulatingResponse"); + chk.accept(numRequests, "transformation"); + chk.accept(numRequests, "targetTransaction"); + chk.accept(numRequests*2, "scheduled"); + chk.accept(numRequests, "requestSending"); + chk.accept(numRequests, "comparingResults"); + + Consumer chkNonZero = k-> { + Assertions.assertNotNull(byName.get(k)); + Assertions.assertFalse(byName.get(k).isEmpty()); + byName.remove(k); + }; + chkNonZero.accept("waitingForResponse"); + chkNonZero.accept("readNextTrafficStreamChunk"); + // ideally, we'd be getting these back too, but our requests are malformed, so the server closes, which + // may occur before we've started to accumulate the response. So - just ignore these, but make sure that + // there isn't anything else that we've missed. + byName.remove("receivingResponse"); + + Assertions.assertEquals("", byName.entrySet().stream() + .map(kvp->kvp.getKey()+":"+kvp.getValue()).collect(Collectors.joining())); + } +} diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/FullTrafficReplayerTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/FullTrafficReplayerTest.java index b82c4b3ef..b5bdb7e5c 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/FullTrafficReplayerTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/FullTrafficReplayerTest.java @@ -1,37 +1,28 @@ package org.opensearch.migrations.replay; -import com.google.protobuf.ByteString; -import com.google.protobuf.Timestamp; import lombok.EqualsAndHashCode; import lombok.Getter; import lombok.ToString; import lombok.extern.slf4j.Slf4j; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Tag; -import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.CsvSource; import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; -import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamWithKey; -import org.opensearch.migrations.replay.traffic.source.BlockingTrafficSource; +import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamAndKey; +import org.opensearch.migrations.replay.tracing.IReplayContexts; +import org.opensearch.migrations.replay.tracing.ITrafficSourceContexts; import org.opensearch.migrations.replay.traffic.source.ISimpleTrafficCaptureSource; import org.opensearch.migrations.replay.traffic.source.ITrafficStreamWithKey; import org.opensearch.migrations.testutils.SimpleNettyHttpServer; import org.opensearch.migrations.testutils.WrapWithNettyLeakDetection; -import org.opensearch.migrations.trafficcapture.protos.CloseObservation; -import org.opensearch.migrations.trafficcapture.protos.EndOfMessageIndication; -import org.opensearch.migrations.trafficcapture.protos.ReadObservation; -import org.opensearch.migrations.trafficcapture.protos.TrafficObservation; +import org.opensearch.migrations.tracing.InstrumentationTest; +import org.opensearch.migrations.tracing.TestContext; import org.opensearch.migrations.trafficcapture.protos.TrafficStream; import org.opensearch.migrations.trafficcapture.protos.TrafficStreamUtils; -import org.opensearch.migrations.trafficcapture.protos.WriteObservation; -import org.opensearch.migrations.transform.StaticAuthTransformerFactory; import java.io.EOFException; -import java.nio.charset.StandardCharsets; import java.time.Duration; -import java.time.Instant; -import java.util.HashSet; import java.util.List; import java.util.Optional; import java.util.PriorityQueue; @@ -39,6 +30,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; +import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -49,13 +41,13 @@ // to the test server, a shutdown will stop those work threads without letting them flush through all of their work // (since that could take a very long time) and some of the work might have been followed by resource releases. @WrapWithNettyLeakDetection(disableLeakChecks = true) -public class FullTrafficReplayerTest { +public class FullTrafficReplayerTest extends InstrumentationTest { public static final int INITIAL_STOP_REPLAYER_REQUEST_COUNT = 1; public static final String TEST_NODE_ID = "TestNodeId"; public static final String TEST_CONNECTION_ID = "testConnectionId"; - private static class IndexWatchingListenerFactory implements Supplier> { + protected static class IndexWatchingListenerFactory implements Supplier> { AtomicInteger nextStopPointRef = new AtomicInteger(INITIAL_STOP_REPLAYER_REQUEST_COUNT); @Override @@ -63,7 +55,7 @@ public Consumer get() { log.info("StopAt="+nextStopPointRef.get()); var stopPoint = nextStopPointRef.get(); return tuple -> { - var key = tuple.uniqueRequestKey; + var key = tuple.getRequestKey(); if (((TrafficStreamCursorKey) (key.getTrafficStreamKey())).arrayIndex > stopPoint) { log.error("Request received after our ingest threshold. Throwing. Discarding " + key); var nextStopPoint = stopPoint + new Random(stopPoint).nextInt(stopPoint + 1); @@ -74,78 +66,6 @@ public Consumer get() { } } - @Test - public void testSingleStreamWithCloseIsCommitted() throws Throwable { - var random = new Random(1); - var httpServer = SimpleNettyHttpServer.makeServer(false, Duration.ofMillis(2), - response->TestHttpServerContext.makeResponse(random, response)); - var trafficStreamWithJustClose = TrafficStream.newBuilder() - .setNodeId(TEST_NODE_ID) - .setConnectionId(TEST_CONNECTION_ID) - .addSubStream(TrafficObservation.newBuilder() - .setClose(CloseObservation.newBuilder().build()).build()) - .build(); - var trafficSourceSupplier = new ArrayCursorTrafficSourceFactory(List.of(trafficStreamWithJustClose)); - TrafficReplayerRunner.runReplayerUntilSourceWasExhausted(0, - httpServer.localhostEndpoint(), new IndexWatchingListenerFactory(), trafficSourceSupplier); - Assertions.assertEquals(1, trafficSourceSupplier.nextReadCursor.get()); - log.info("done"); - } - - @Test - public void testDoubleRequestWithCloseIsCommittedOnce() throws Throwable { - var random = new Random(1); - var httpServer = SimpleNettyHttpServer.makeServer(false, Duration.ofMillis(2), - response->TestHttpServerContext.makeResponse(random, response)); - var baseTime = Instant.now(); - var fixedTimestamp = - Timestamp.newBuilder().setSeconds(baseTime.getEpochSecond()).setNanos(baseTime.getNano()).build(); - var tsb = TrafficStream.newBuilder().setConnectionId("C"); - for (int i=0; i<2; ++i) { - tsb = tsb - .addSubStream(TrafficObservation.newBuilder().setTs(fixedTimestamp) - .setRead(ReadObservation.newBuilder() - .setData(ByteString.copyFrom(("GET /" + i + " HTTP/1.0\r\n") - .getBytes(StandardCharsets.UTF_8))) - .build()) - .build()) - .addSubStream(TrafficObservation.newBuilder().setTs(fixedTimestamp) - .setEndOfMessageIndicator(EndOfMessageIndication.newBuilder() - .setFirstLineByteLength(14) - .setHeadersByteLength(14) - .build()) - .build()) - .addSubStream(TrafficObservation.newBuilder().setTs(fixedTimestamp) - .setWrite(WriteObservation.newBuilder() - .setData(ByteString.copyFrom("HTTP/1.0 OK 200\r\n".getBytes(StandardCharsets.UTF_8))) - .build()) - .build()); - } - var trafficStream = tsb.addSubStream(TrafficObservation.newBuilder().setTs(fixedTimestamp) - .setClose(CloseObservation.getDefaultInstance())) - .build(); - var trafficSource = - new ArrayCursorTrafficCaptureSource(new ArrayCursorTrafficSourceFactory(List.of(trafficStream))); - var tr = new TrafficReplayer(httpServer.localhostEndpoint(), null, - new StaticAuthTransformerFactory("TEST"), null, - true, 10, 10*1024); - - var tuplesReceived = new HashSet(); - try (var blockingTrafficSource = new BlockingTrafficSource(trafficSource, Duration.ofMinutes(2))) { - tr.setupRunAndWaitForReplayWithShutdownChecks(Duration.ofSeconds(70), blockingTrafficSource, - new TimeShifter(10 * 1000), (t) -> { - var key = t.uniqueRequestKey; - var wasNew = tuplesReceived.add(key.toString()); - Assertions.assertTrue(wasNew); - }); - } finally { - tr.shutdown(null); - } - - Assertions.assertEquals(2, tuplesReceived.size()); - log.info("done"); - } - @ParameterizedTest @CsvSource(value = { "3,false", @@ -156,17 +76,21 @@ public void testDoubleRequestWithCloseIsCommittedOnce() throws Throwable { @Tag("longTest") public void fullTest(int testSize, boolean randomize) throws Throwable { var random = new Random(1); - var httpServer = SimpleNettyHttpServer.makeServer(false, Duration.ofMillis(2), - response->TestHttpServerContext.makeResponse(random,response)); - var streamAndConsumer = TrafficStreamGenerator.generateStreamAndSumOfItsTransactions(testSize, randomize); - var numExpectedRequests = streamAndConsumer.numHttpTransactions; - var trafficStreams = streamAndConsumer.stream.collect(Collectors.toList()); - log.atInfo().setMessage(()->trafficStreams.stream().map(ts->TrafficStreamUtils.summarizeTrafficStream(ts)) - .collect(Collectors.joining("\n"))).log(); + var httpServer = SimpleNettyHttpServer.makeServer(false, Duration.ofMillis(200), + response -> TestHttpServerContext.makeResponse(random, response)); + var streamAndSizes = TrafficStreamGenerator.generateStreamAndSumOfItsTransactions(TestContext.noOtelTracking(), + testSize, randomize); + var numExpectedRequests = streamAndSizes.numHttpTransactions; + var trafficStreams = streamAndSizes.stream.collect(Collectors.toList()); + log.atInfo().setMessage(() -> trafficStreams.stream().map(ts -> TrafficStreamUtils.summarizeTrafficStream(ts)) + .collect(Collectors.joining("\n"))).log(); var trafficSourceSupplier = new ArrayCursorTrafficSourceFactory(trafficStreams); - TrafficReplayerRunner.runReplayerUntilSourceWasExhausted(numExpectedRequests, - httpServer.localhostEndpoint(), new IndexWatchingListenerFactory(), trafficSourceSupplier); - Assertions.assertEquals(trafficSourceSupplier.trafficStreamsList.size(), trafficSourceSupplier.nextReadCursor.get()); + TrafficReplayerRunner.runReplayerUntilSourceWasExhausted( + numExpectedRequests, httpServer.localhostEndpoint(), new IndexWatchingListenerFactory(), + () -> TestContext.noOtelTracking(), + trafficSourceSupplier); + Assertions.assertEquals(trafficSourceSupplier.trafficStreamsList.size(), + trafficSourceSupplier.nextReadCursor.get()); log.info("done"); } @@ -179,13 +103,14 @@ private static class TrafficStreamCursorKey implements ITrafficStreamKey, Compar public final String connectionId; public final String nodeId; public final int trafficStreamIndex; + @Getter public final IReplayContexts.ITrafficStreamsLifecycleContext trafficStreamsContext; - - public TrafficStreamCursorKey(TrafficStream stream, int arrayIndex) { + public TrafficStreamCursorKey(TestContext context, TrafficStream stream, int arrayIndex) { connectionId = stream.getConnectionId(); nodeId = stream.getNodeId(); trafficStreamIndex = TrafficStreamUtils.getTrafficStreamIndex(stream); this.arrayIndex = arrayIndex; + trafficStreamsContext = context.createTrafficStreamContextForTest(this); } @Override @@ -194,7 +119,7 @@ public int compareTo(TrafficStreamCursorKey other) { } } - private static class ArrayCursorTrafficSourceFactory implements Supplier { + protected static class ArrayCursorTrafficSourceFactory implements Function { List trafficStreamsList; AtomicInteger nextReadCursor = new AtomicInteger(); @@ -202,45 +127,49 @@ public ArrayCursorTrafficSourceFactory(List trafficStreamsList) { this.trafficStreamsList = trafficStreamsList; } - public ISimpleTrafficCaptureSource get() { - var rval = new ArrayCursorTrafficCaptureSource(this); + public ISimpleTrafficCaptureSource apply(TestContext rootContext) { + var rval = new ArrayCursorTrafficCaptureSource(rootContext, this); log.info("trafficSource="+rval+" readCursor="+rval.readCursor.get()+" nextReadCursor="+ nextReadCursor.get()); return rval; } } - private static class ArrayCursorTrafficCaptureSource implements ISimpleTrafficCaptureSource { + protected static class ArrayCursorTrafficCaptureSource implements ISimpleTrafficCaptureSource { final AtomicInteger readCursor; final PriorityQueue pQueue = new PriorityQueue<>(); Integer cursorHighWatermark; ArrayCursorTrafficSourceFactory arrayCursorTrafficSourceFactory; + TestContext rootContext; - public ArrayCursorTrafficCaptureSource(ArrayCursorTrafficSourceFactory arrayCursorTrafficSourceFactory) { + public ArrayCursorTrafficCaptureSource(TestContext rootContext, + ArrayCursorTrafficSourceFactory arrayCursorTrafficSourceFactory) { var startingCursor = arrayCursorTrafficSourceFactory.nextReadCursor.get(); log.info("startingCursor = " + startingCursor); this.readCursor = new AtomicInteger(startingCursor); this.arrayCursorTrafficSourceFactory = arrayCursorTrafficSourceFactory; cursorHighWatermark = startingCursor; + this.rootContext = rootContext; } @Override - public CompletableFuture> readNextTrafficStreamChunk() { + public CompletableFuture> + readNextTrafficStreamChunk(Supplier contextSupplier) { var idx = readCursor.getAndIncrement(); log.info("reading chunk from index="+idx); if (arrayCursorTrafficSourceFactory.trafficStreamsList.size() <= idx) { return CompletableFuture.failedFuture(new EOFException()); } var stream = arrayCursorTrafficSourceFactory.trafficStreamsList.get(idx); - var key = new TrafficStreamCursorKey(stream, idx); + var key = new TrafficStreamCursorKey(rootContext, stream, idx); synchronized (pQueue) { pQueue.add(key); cursorHighWatermark = idx; } - return CompletableFuture.supplyAsync(()->List.of(new PojoTrafficStreamWithKey(stream, key))); + return CompletableFuture.supplyAsync(()->List.of(new PojoTrafficStreamAndKey(stream, key))); } @Override - public void commitTrafficStream(ITrafficStreamKey trafficStreamKey) { + public CommitResult commitTrafficStream(ITrafficStreamKey trafficStreamKey) { synchronized (pQueue) { // figure out if I need to do something more efficient later log.info("Commit called for "+trafficStreamKey+" with pQueue.size="+pQueue.size()); var incomingCursor = ((TrafficStreamCursorKey)trafficStreamKey).arrayIndex; @@ -259,6 +188,9 @@ public void commitTrafficStream(ITrafficStreamKey trafficStreamKey) { log.info("Commit called for "+trafficStreamKey+", but topCursor="+topCursor); } } + rootContext.channelContextManager.releaseContextFor( + ((TrafficStreamCursorKey) trafficStreamKey).trafficStreamsContext.getChannelKeyContext()); + return CommitResult.Immediate; } } 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..3dfb612f1 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 @@ -7,6 +7,7 @@ import org.opensearch.migrations.replay.datatypes.HttpRequestTransformationStatus; import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; import org.opensearch.migrations.testutils.WrapWithNettyLeakDetection; +import org.opensearch.migrations.tracing.InstrumentationTest; import org.opensearch.migrations.transform.StaticAuthTransformerFactory; import java.time.Duration; @@ -15,14 +16,13 @@ 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; @Slf4j @WrapWithNettyLeakDetection -public class HeaderTransformerTest { +public class HeaderTransformerTest extends InstrumentationTest { private static final String SILLY_TARGET_CLUSTER_NAME = "remoteguest"; private static final String SOURCE_CLUSTER_NAME = "localhost"; @@ -35,7 +35,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)); + rootContext.getTestConnectionRequestContext(0)); runRandomPayloadWithTransformer(transformingHandler, dummyAggregatedResponse, testPacketCapture, contentLength -> "GET / HTTP/1.1\r\n" + "HoSt: " + SOURCE_CLUSTER_NAME + "\r\n" + @@ -79,7 +79,6 @@ private void runRandomPayloadWithTransformer(HttpJsonTransformingConsumer "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, rootContext.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/KafkaRestartingTrafficReplayerTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/KafkaRestartingTrafficReplayerTest.java index ab2c5caad..9dd8164c7 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/KafkaRestartingTrafficReplayerTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/KafkaRestartingTrafficReplayerTest.java @@ -8,17 +8,16 @@ import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.clients.producer.ProducerRecord; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Tag; -import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.CsvSource; import org.opensearch.migrations.replay.kafka.KafkaTestUtils; import org.opensearch.migrations.replay.kafka.KafkaTrafficCaptureSource; import org.opensearch.migrations.replay.traffic.source.ISimpleTrafficCaptureSource; import org.opensearch.migrations.replay.traffic.source.ITrafficStreamWithKey; -import org.opensearch.migrations.replay.traffic.source.TrafficStreamWithEmbeddedKey; import org.opensearch.migrations.testutils.SimpleNettyHttpServer; +import org.opensearch.migrations.tracing.InstrumentationTest; +import org.opensearch.migrations.tracing.TestContext; import org.opensearch.migrations.trafficcapture.protos.TrafficStream; import org.opensearch.migrations.trafficcapture.protos.TrafficStreamUtils; import org.testcontainers.containers.KafkaContainer; @@ -40,7 +39,7 @@ @Slf4j @Testcontainers(disabledWithoutDocker = true) @Tag("requiresDocker") -public class KafkaRestartingTrafficReplayerTest { +public class KafkaRestartingTrafficReplayerTest extends InstrumentationTest { public static final int INITIAL_STOP_REPLAYER_REQUEST_COUNT = 1; public static final String TEST_GROUP_CONSUMER_ID = "TEST_GROUP_CONSUMER_ID"; public static final String TEST_GROUP_PRODUCER_ID = "TEST_GROUP_PRODUCER_ID"; @@ -68,7 +67,7 @@ public Consumer get() { return tuple -> { if (counter.incrementAndGet() > stopPoint) { log.warn("Request received after our ingest threshold. Throwing. Discarding " + - tuple.uniqueRequestKey); + tuple.context); var nextStopPoint = stopPoint + new Random(stopPoint).nextInt(stopPoint + 1); nextStopPointRef.compareAndSet(stopPoint, nextStopPoint); throw new TrafficReplayerRunner.FabricatedErrorToKillTheReplayer(false); @@ -89,7 +88,8 @@ public void fullTest(int testSize, boolean randomize) throws Throwable { var random = new Random(1); var httpServer = SimpleNettyHttpServer.makeServer(false, Duration.ofMillis(2), response->TestHttpServerContext.makeResponse(random, response)); - var streamAndConsumer = TrafficStreamGenerator.generateStreamAndSumOfItsTransactions(testSize, randomize); + var streamAndConsumer = + TrafficStreamGenerator.generateStreamAndSumOfItsTransactions(TestContext.noOtelTracking(), testSize, randomize); var trafficStreams = streamAndConsumer.stream.collect(Collectors.toList()); log.atInfo().setMessage(()->trafficStreams.stream().map(TrafficStreamUtils::summarizeTrafficStream) .collect(Collectors.joining("\n"))).log(); @@ -98,8 +98,9 @@ public void fullTest(int testSize, boolean randomize) throws Throwable { Streams.concat(trafficStreams.stream(), Stream.of(SENTINEL_TRAFFIC_STREAM))); TrafficReplayerRunner.runReplayerUntilSourceWasExhausted(streamAndConsumer.numHttpTransactions, httpServer.localhostEndpoint(), new CounterLimitedReceiverFactory(), - () -> new SentinelSensingTrafficSource( - new KafkaTrafficCaptureSource(buildKafkaConsumer(), TEST_TOPIC_NAME, + () -> TestContext.noOtelTracking(), + rootContext -> new SentinelSensingTrafficSource( + new KafkaTrafficCaptureSource(rootContext, buildKafkaConsumer(), TEST_TOPIC_NAME, Duration.ofMillis(DEFAULT_POLL_INTERVAL_MS)))); log.info("done"); } @@ -119,8 +120,8 @@ private void loadStreamsToKafka(KafkaConsumer kafkaConsumer, var kafkaProducer = buildKafkaProducer(); var counter = new AtomicInteger(); loadStreamsAsynchronouslyWithCloseableResource(kafkaConsumer, streams, s -> s.forEach(trafficStream -> - KafkaTestUtils.writeTrafficStreamRecord(kafkaProducer, new TrafficStreamWithEmbeddedKey(trafficStream), - TEST_TOPIC_NAME, "KEY_" + counter.incrementAndGet()))); + KafkaTestUtils.writeTrafficStreamRecord(kafkaProducer, + trafficStream, TEST_TOPIC_NAME, "KEY_" + counter.incrementAndGet()))); Thread.sleep(PRODUCER_SLEEP_INTERVAL_MS); } @@ -160,18 +161,21 @@ Producer buildKafkaProducer() { } private Supplier - loadStreamsToKafkaFromCompressedFile(KafkaConsumer kafkaConsumer, + loadStreamsToKafkaFromCompressedFile(TestContext rootCtx, + KafkaConsumer kafkaConsumer, String filename, int recordCount) throws Exception { var kafkaProducer = buildKafkaProducer(); - loadStreamsAsynchronouslyWithCloseableResource(kafkaConsumer, new V0_1TrafficCaptureSource(filename), + loadStreamsAsynchronouslyWithCloseableResource(kafkaConsumer, + new V0_1TrafficCaptureSource(rootCtx, filename), originalTrafficSource -> { try { for (int i = 0; i < recordCount; ++i) { List chunks = null; - chunks = originalTrafficSource.readNextTrafficStreamChunk().get(); + chunks = originalTrafficSource.readNextTrafficStreamChunk(rootCtx::createReadChunkContext) + .get(); for (int j = 0; j < chunks.size(); ++j) { - KafkaTestUtils.writeTrafficStreamRecord(kafkaProducer, chunks.get(j), TEST_TOPIC_NAME, - "KEY_" + i + "_" + j); + KafkaTestUtils.writeTrafficStreamRecord(kafkaProducer, chunks.get(j).getStream(), + TEST_TOPIC_NAME, "KEY_" + i + "_" + j); Thread.sleep(PRODUCER_SLEEP_INTERVAL_MS); } } @@ -179,7 +183,7 @@ Producer buildKafkaProducer() { throw Lombok.sneakyThrow(e); } }); - return () -> new KafkaTrafficCaptureSource(kafkaConsumer, TEST_TOPIC_NAME, + return () -> new KafkaTrafficCaptureSource(rootCtx, kafkaConsumer, TEST_TOPIC_NAME, Duration.ofMillis(DEFAULT_POLL_INTERVAL_MS)); } diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/ParsedHttpMessagesAsDictsTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/ParsedHttpMessagesAsDictsTest.java index 8025f52e0..056e33a64 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/ParsedHttpMessagesAsDictsTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/ParsedHttpMessagesAsDictsTest.java @@ -1,85 +1,27 @@ package org.opensearch.migrations.replay; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; -import org.opensearch.migrations.replay.datatypes.MockMetricsBuilder; -import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamKey; -import org.opensearch.migrations.replay.datatypes.PojoUniqueSourceRequestKey; +import org.opensearch.migrations.tracing.InstrumentationTest; +import org.opensearch.migrations.tracing.TestContext; import java.util.Map; import java.util.Optional; -import static org.junit.jupiter.api.Assertions.*; - -class ParsedHttpMessagesAsDictsTest { - - private static final PojoTrafficStreamKey TEST_TRAFFIC_STREAM_KEY = - new PojoTrafficStreamKey("N","C",1); +class ParsedHttpMessagesAsDictsTest extends InstrumentationTest { ParsedHttpMessagesAsDicts makeTestData() { return makeTestData(null, null); } - ParsedHttpMessagesAsDicts makeTestData(Map sourceResponse, Map targetResponse) { - return new ParsedHttpMessagesAsDicts( - Optional.empty(), Optional.ofNullable(sourceResponse), - Optional.empty(), Optional.ofNullable(targetResponse)); + @Override + protected TestContext makeInstrumentationContext() { + return TestContext.withTracking(false, true); } - String getLoggedMetrics(ParsedHttpMessagesAsDicts parsedMessage) { - var metricsBuilder = new MockMetricsBuilder(); - metricsBuilder = (MockMetricsBuilder) parsedMessage.buildStatusCodeMetrics(metricsBuilder, - new PojoUniqueSourceRequestKey(TEST_TRAFFIC_STREAM_KEY, 0)); - return metricsBuilder.getLoggedAttributes(); - } - - @Test - public void testMetricsAreRightWhenBothAreEmpty() { - var loggedMetrics = getLoggedMetrics(makeTestData()); - Assertions.assertEquals("REQUEST_ID:C.0|HTTP_STATUS_MATCH:0", loggedMetrics); - } - - @Test - public void testMetricsAreRightWhenSourceIsEmpty() { - var loggedMetrics = getLoggedMetrics(makeTestData( - null, - Map.of("Status-Code", Integer.valueOf(200)) - )); - Assertions.assertEquals("REQUEST_ID:C.0|TARGET_HTTP_STATUS:200|HTTP_STATUS_MATCH:0", loggedMetrics); - } - - @Test - public void testMetricsAreRightWhenTargetIsEmpty() { - var loggedMetrics = getLoggedMetrics(makeTestData( - Map.of("Status-Code", Integer.valueOf(200)), - null - )); - Assertions.assertEquals("REQUEST_ID:C.0|SOURCE_HTTP_STATUS:200|HTTP_STATUS_MATCH:0", loggedMetrics); - } - - @Test - public void testMetricsAreRightWhenDifferent() { - var loggedMetrics = getLoggedMetrics(makeTestData( - Map.of("Status-Code", Integer.valueOf(200)), - Map.of("Status-Code", Integer.valueOf(200)) - )); - Assertions.assertEquals("REQUEST_ID:C.0|SOURCE_HTTP_STATUS:200|TARGET_HTTP_STATUS:200|HTTP_STATUS_MATCH:1", loggedMetrics); - } - - @Test - public void testMetricsAreRightWhenMissing() { - var loggedMetrics = getLoggedMetrics(makeTestData( - Map.of("Status-Code", Integer.valueOf(200)), - Map.of("Status-Code", Integer.valueOf(404)))); - Assertions.assertEquals("REQUEST_ID:C.0|SOURCE_HTTP_STATUS:200|TARGET_HTTP_STATUS:404|HTTP_STATUS_MATCH:0", loggedMetrics); + ParsedHttpMessagesAsDicts makeTestData(Map sourceResponse, + Map targetResponse) { + return new ParsedHttpMessagesAsDicts(rootContext.getTestTupleContext(), + Optional.empty(), Optional.ofNullable(sourceResponse), + Optional.empty(), Optional.ofNullable(targetResponse)); } - @Test - public void testMetricsAreRightWithMissingStatusCode() { - var loggedMetrics = getLoggedMetrics(makeTestData( - Map.of("Sorry", "exception message..."), - Map.of("Status-Code", Integer.valueOf(404)))); - Assertions.assertEquals("REQUEST_ID:C.0|TARGET_HTTP_STATUS:404|HTTP_STATUS_MATCH:0", loggedMetrics); - } } \ No newline at end of file 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..7f01c7d56 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,13 +5,13 @@ 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; import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; import org.opensearch.migrations.testutils.SimpleHttpServer; import org.opensearch.migrations.testutils.WrapWithNettyLeakDetection; +import org.opensearch.migrations.tracing.InstrumentationTest; import java.nio.charset.StandardCharsets; import java.time.Duration; @@ -23,7 +23,7 @@ @Slf4j @WrapWithNettyLeakDetection(repetitions = 1) -class RequestSenderOrchestratorTest { +class RequestSenderOrchestratorTest extends InstrumentationTest { public static final int NUM_REQUESTS_TO_SCHEDULE = 20; public static final int NUM_REPEATS = 2; @@ -40,26 +40,27 @@ public void testThatSchedulingWorks() throws Exception { Instant lastEndTime = baseTime; var scheduledItems = new ArrayList>(); for (int i = 0; i 0); var httpMessage = HttpByteBufFormatter.parseHttpMessageFromBufs(HttpByteBufFormatter.HttpMessageType.RESPONSE, arr.responsePackets.stream().map(kvp->Unpooled.wrappedBuffer(kvp.getValue())), false); diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/ResultsToLogsConsumerTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/ResultsToLogsConsumerTest.java index 90ee03710..3ff102293 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/ResultsToLogsConsumerTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/ResultsToLogsConsumerTest.java @@ -14,12 +14,12 @@ import org.junit.jupiter.api.Test; import org.opensearch.migrations.replay.datahandlers.NettyPacketToHttpConsumerTest; import org.opensearch.migrations.replay.datatypes.HttpRequestTransformationStatus; -import org.opensearch.migrations.replay.datatypes.MockMetricsBuilder; -import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamKey; -import org.opensearch.migrations.replay.datatypes.PojoUniqueSourceRequestKey; +import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamKeyAndContext; import org.opensearch.migrations.replay.datatypes.TransformedPackets; import org.opensearch.migrations.replay.datatypes.UniqueReplayerRequestKey; import org.opensearch.migrations.testutils.WrapWithNettyLeakDetection; +import org.opensearch.migrations.tracing.InstrumentationTest; +import org.opensearch.migrations.tracing.TestContext; import java.io.IOException; import java.io.InputStream; @@ -28,15 +28,20 @@ import java.time.Instant; import java.util.ArrayList; import java.util.List; -import java.util.function.BiConsumer; +import java.util.stream.Collectors; @Slf4j @WrapWithNettyLeakDetection(repetitions = 4) -class ResultsToLogsConsumerTest { +class ResultsToLogsConsumerTest extends InstrumentationTest { private static final String NODE_ID = "n"; private static final ObjectMapper mapper = new ObjectMapper(); public static final String TEST_EXCEPTION_MESSAGE = "TEST_EXCEPTION"; + @Override + protected TestContext makeInstrumentationContext() { + return TestContext.withTracking(false, true); + } + private static class CloseableLogSetup implements AutoCloseable { List logEvents = new ArrayList<>(); AbstractAppender testAppender; @@ -76,11 +81,13 @@ public void testTupleNewWithNullKeyThrows() { @Test public void testOutputterWithNulls() throws IOException { - var emptyTuple = new SourceTargetCaptureTuple( - new UniqueReplayerRequestKey(new PojoTrafficStreamKey(NODE_ID,"c",0), 0, 0), + + var urk = new UniqueReplayerRequestKey(PojoTrafficStreamKeyAndContext.build(NODE_ID, "c", 0, + rootContext::createTrafficStreamContextForTest), 0, 0); + var emptyTuple = new SourceTargetCaptureTuple(rootContext.getTestTupleContext(), null, null, null, null, null, null); try (var closeableLogSetup = new CloseableLogSetup()) { - var consumer = new TupleParserChainConsumer(null, new ResultsToLogsConsumer()); + var consumer = new TupleParserChainConsumer(new ResultsToLogsConsumer()); consumer.accept(emptyTuple); Assertions.assertEquals(1, closeableLogSetup.logEvents.size()); var contents = closeableLogSetup.logEvents.get(0); @@ -92,12 +99,11 @@ public void testOutputterWithNulls() throws IOException { @Test public void testOutputterWithException() throws IOException { var exception = new Exception(TEST_EXCEPTION_MESSAGE); - var emptyTuple = new SourceTargetCaptureTuple( - new UniqueReplayerRequestKey(new PojoTrafficStreamKey(NODE_ID,"c",0), 0, 0), + var emptyTuple = new SourceTargetCaptureTuple(rootContext.getTestTupleContext(), null, null, null, null, exception, null); try (var closeableLogSetup = new CloseableLogSetup()) { - var consumer = new TupleParserChainConsumer(null, new ResultsToLogsConsumer()); + var consumer = new TupleParserChainConsumer(new ResultsToLogsConsumer()); consumer.accept(emptyTuple); Assertions.assertEquals(1, closeableLogSetup.logEvents.size()); var contents = closeableLogSetup.logEvents.get(0); @@ -166,7 +172,7 @@ public void testOutputterForGet() throws IOException { " \"Funtime\": \"checkIt!\",\n" + " \"content-length\": \"30\"\n" + " },\n" + - " \"connectionId\": \"c.0\"\n" + + " \"connectionId\": \"testConnection.1\"\n" + "}"; testOutputterForRequest("get_withAuthHeader.txt", EXPECTED_LOGGED_OUTPUT); } @@ -221,15 +227,16 @@ public void testOutputterForPost() throws IOException { " \"Funtime\": \"checkIt!\",\n" + " \"content-length\": \"30\"\n" + " },\n" + - " \"connectionId\": \"c.0\"\n" + + " \"connectionId\": \"testConnection.1\"\n" + "}"; testOutputterForRequest("post_formUrlEncoded_withFixedLength.txt", EXPECTED_LOGGED_OUTPUT); } - @Test - private void testOutputterForRequest(String requestResourceName, String expected) throws IOException { - var trafficStreamKey = new PojoTrafficStreamKey(NODE_ID,"c",0); - var sourcePair = new RequestResponsePacketPair(trafficStreamKey); + public void testOutputterForRequest(String requestResourceName, String expected) throws IOException { + var trafficStreamKey = PojoTrafficStreamKeyAndContext.build(NODE_ID, "c", 0, + rootContext::createTrafficStreamContextForTest); + var sourcePair = new RequestResponsePacketPair(trafficStreamKey, Instant.EPOCH, + 0, 0); var rawRequestData = loadResourceAsBytes("/requests/raw/" + requestResourceName); sourcePair.addRequestData(Instant.EPOCH, rawRequestData); var rawResponseData = NettyPacketToHttpConsumerTest.EXPECTED_RESPONSE_STRING.getBytes(StandardCharsets.UTF_8); @@ -240,31 +247,31 @@ private void testOutputterForRequest(String requestResourceName, String expected var targetResponse = new ArrayList(); targetResponse.add(rawResponseData); - var tuple = new SourceTargetCaptureTuple( - new UniqueReplayerRequestKey(trafficStreamKey, 0, 0), - sourcePair, targetRequest, targetResponse, HttpRequestTransformationStatus.SKIPPED, null, Duration.ofMillis(267)); - var streamConsumer = new ResultsToLogsConsumer(); - // we don't have an interface on MetricsLogger yet, so it's a challenge to test that directly. - // Assuming that it's going to use Slf/Log4J are really brittle. I'd rather miss a couple lines that - // should be getting tested elsewhere and be immune to those changes down the line - BiConsumer metricsHardWayCheckConsumer = (t,p) -> { - var metricsBuilder = new MockMetricsBuilder(); - metricsBuilder = (MockMetricsBuilder) p.buildStatusCodeMetrics(metricsBuilder, - new PojoUniqueSourceRequestKey(trafficStreamKey, 0)); - Assertions.assertEquals("REQUEST_ID:c.0|SOURCE_HTTP_STATUS:200|TARGET_HTTP_STATUS:200|HTTP_STATUS_MATCH:1", - metricsBuilder.getLoggedAttributes()); - }; - try (var closeableLogSetup = new CloseableLogSetup()) { - var consumer = new TupleParserChainConsumer(null, (a,b)->{ - streamConsumer.accept(a,b); - metricsHardWayCheckConsumer.accept(a,b); - }); + try (var tupleContext = rootContext.getTestTupleContext(); + var closeableLogSetup = new CloseableLogSetup()) { + var tuple = new SourceTargetCaptureTuple(tupleContext, + sourcePair, targetRequest, targetResponse, HttpRequestTransformationStatus.SKIPPED, null, Duration.ofMillis(267)); + var streamConsumer = new ResultsToLogsConsumer(); + var consumer = new TupleParserChainConsumer(streamConsumer); consumer.accept(tuple); Assertions.assertEquals(1, closeableLogSetup.logEvents.size()); var contents = closeableLogSetup.logEvents.get(0); - log.info("Output="+contents); + log.info("Output=" + contents); Assertions.assertEquals(normalizeJson(expected), normalizeJson(contents)); } + try { + Thread.sleep(500); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + var allMetricData = rootContext.inMemoryInstrumentationBundle.testMetricExporter.getFinishedMetricItems(); + var filteredMetrics = allMetricData.stream().filter(md->md.getName().startsWith("tupleResult")) + .collect(Collectors.toList()); + // TODO - find out how to verify these metrics + log.error("TODO - find out how to verify these metrics"); +// Assertions.assertEquals("REQUEST_ID:testConnection.1|SOURCE_HTTP_STATUS:200|TARGET_HTTP_STATUS:200|HTTP_STATUS_MATCH:1", +// filteredMetrics.stream().map(md->md.getName()+":"+md.getData()).collect(Collectors.joining("|"))); + } static String normalizeJson(String input) throws JsonProcessingException { diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/SentinelSensingTrafficSource.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/SentinelSensingTrafficSource.java index 5ef78d922..6d23daca3 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/SentinelSensingTrafficSource.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/SentinelSensingTrafficSource.java @@ -2,15 +2,16 @@ import lombok.extern.slf4j.Slf4j; import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; +import org.opensearch.migrations.replay.tracing.ITrafficSourceContexts; import org.opensearch.migrations.replay.traffic.source.ISimpleTrafficCaptureSource; import org.opensearch.migrations.replay.traffic.source.ITrafficStreamWithKey; import java.io.EOFException; import java.io.IOException; import java.util.List; -import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Supplier; import java.util.stream.Collectors; @Slf4j @@ -25,11 +26,12 @@ public SentinelSensingTrafficSource(ISimpleTrafficCaptureSource underlyingSource } @Override - public CompletableFuture> readNextTrafficStreamChunk() { + public CompletableFuture> + readNextTrafficStreamChunk(Supplier contextSupplier) { if (stopReadingRef.get()) { return CompletableFuture.failedFuture(new EOFException()); } - return underlyingSource.readNextTrafficStreamChunk().thenApply(v->{ + return underlyingSource.readNextTrafficStreamChunk(contextSupplier).thenApply(v->{ if (v != null) { return v.stream().takeWhile(ts->{ var isSentinel = ts.getStream().getConnectionId().equals(SENTINEL_CONNECTION_ID); @@ -45,8 +47,8 @@ public CompletableFuture> readNextTrafficStreamChunk } @Override - public void commitTrafficStream(ITrafficStreamKey trafficStreamKey) throws IOException { - underlyingSource.commitTrafficStream(trafficStreamKey); + public CommitResult commitTrafficStream(ITrafficStreamKey trafficStreamKey) throws IOException { + return underlyingSource.commitTrafficStream(trafficStreamKey); } @Override diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/SigV4SigningTransformationTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/SigV4SigningTransformationTest.java index 8d7ad7206..4d7563897 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/SigV4SigningTransformationTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/SigV4SigningTransformationTest.java @@ -1,16 +1,14 @@ package org.opensearch.migrations.replay; -import io.netty.buffer.Unpooled; -import io.netty.handler.codec.base64.Base64; import io.netty.handler.codec.http.DefaultHttpHeaders; import io.netty.util.ResourceLeakDetector; import org.junit.jupiter.api.Test; import org.opensearch.migrations.testutils.WrapWithNettyLeakDetection; +import org.opensearch.migrations.tracing.InstrumentationTest; import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; import software.amazon.awssdk.auth.credentials.AwsCredentials; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; -import java.nio.charset.StandardCharsets; import java.time.Clock; import java.time.Instant; import java.time.ZoneOffset; @@ -20,7 +18,7 @@ @WrapWithNettyLeakDetection -public class SigV4SigningTransformationTest { +public class SigV4SigningTransformationTest extends InstrumentationTest { private static class MockCredentialsProvider implements AwsCredentialsProvider { @Override @@ -52,13 +50,14 @@ public void testSignatureProperlyApplied() throws Exception { "SignedHeaders=host;x-amz-content-sha256;x-amz-date, " + "Signature=4cb1c423e6fe61216fbaa11398260af7f8daa85e74cd41428711e4df5cd70c97"); expectedRequestHeaders.add("x-amz-content-sha256", - "fc0e8e9a1f7697f510bfdd4d55b8612df8a0140b4210967efd87ee9cb7104362"); + "fc0e8e9a1f7697f510bfdd4d55b8612df8a0140b4210967efd87ee9cb7104362"); expectedRequestHeaders.add("X-Amz-Date", "19700101T000000Z"); - TestUtils.runPipelineAndValidate( + TestUtils.runPipelineAndValidate(rootContext, msg -> new SigV4Signer(mockCredentialsProvider, "es", "us-east-1", "https", - () -> Clock.fixed(Instant.EPOCH, ZoneOffset.UTC)), + () -> Clock.fixed(Instant.EPOCH, ZoneOffset.UTC)), null, stringParts, expectedRequestHeaders, referenceStringBuilder -> TestUtils.resolveReferenceString(referenceStringBuilder)); + } } diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/SimpleCapturedTrafficToHttpTransactionAccumulatorTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/SimpleCapturedTrafficToHttpTransactionAccumulatorTest.java index f9833785c..f73b2b056 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/SimpleCapturedTrafficToHttpTransactionAccumulatorTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/SimpleCapturedTrafficToHttpTransactionAccumulatorTest.java @@ -10,11 +10,14 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; -import org.opensearch.migrations.replay.datatypes.ISourceTrafficChannelKey; import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; +import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamAndKey; +import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamKeyAndContext; import org.opensearch.migrations.replay.datatypes.RawPackets; -import org.opensearch.migrations.replay.datatypes.UniqueReplayerRequestKey; -import org.opensearch.migrations.replay.traffic.source.TrafficStreamWithEmbeddedKey; +import org.opensearch.migrations.replay.tracing.IReplayContexts; +import org.opensearch.migrations.tracing.InstrumentationTest; +import org.opensearch.migrations.tracing.RootOtelContext; +import org.opensearch.migrations.tracing.TestContext; import org.opensearch.migrations.trafficcapture.IChannelConnectionCaptureSerializer; import org.opensearch.migrations.trafficcapture.InMemoryConnectionCaptureFactory; import org.opensearch.migrations.trafficcapture.protos.TrafficStream; @@ -26,7 +29,6 @@ import java.util.Arrays; import java.util.List; import java.util.SortedSet; -import java.util.StringJoiner; import java.util.TreeSet; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; @@ -49,7 +51,7 @@ * @return */ @Slf4j -public class SimpleCapturedTrafficToHttpTransactionAccumulatorTest { +public class SimpleCapturedTrafficToHttpTransactionAccumulatorTest extends InstrumentationTest { public static final int MAX_COMMANDS_IN_CONNECTION = 256; @@ -88,6 +90,12 @@ public String toString() { } } + static class TestRootContext extends RootOtelContext { + public TestRootContext() { + super(null); + } + } + public static InMemoryConnectionCaptureFactory buildSerializerFactory(int bufferSize, Runnable onClosedCallback) { return new InMemoryConnectionCaptureFactory("TEST_NODE_ID", bufferSize, onClosedCallback); } @@ -109,11 +117,12 @@ static ByteBuf makeSequentialByteBuf(int offset, int size) { return bb; } - static AtomicInteger uniqueIdCounter = new AtomicInteger(); - static TrafficStream[] makeTrafficStreams(int bufferSize, int interactionOffset, - List directives) throws Exception { + static TrafficStream[] makeTrafficStreams(int bufferSize, int interactionOffset, AtomicInteger uniqueIdCounter, + List directives, TestContext rootContext) throws Exception { var connectionFactory = buildSerializerFactory(bufferSize, ()->{}); - var offloader = connectionFactory.createOffloader("TEST_"+uniqueIdCounter.incrementAndGet()); + var tsk = PojoTrafficStreamKeyAndContext.build("n", "test_"+uniqueIdCounter.incrementAndGet(), + 0, rootContext::createTrafficStreamContextForTest); + var offloader = connectionFactory.createOffloader(rootContext.createChannelContext(tsk)); for (var directive : directives) { serializeEvent(offloader, interactionOffset++, directive); } @@ -185,11 +194,12 @@ public static Tuple2 unzipRequestResponseSizes(List collat @MethodSource("loadSimpleCombinations") void generateAndTest(String testName, int bufferSize, int skipCount, List directives, List expectedSizes) throws Exception { - var trafficStreams = Arrays.stream(makeTrafficStreams(bufferSize, 0, directives)) - .skip(skipCount); + var trafficStreams = Arrays.stream(makeTrafficStreams(bufferSize, 0, new AtomicInteger(), + directives, rootContext)).skip(skipCount); List reconstructedTransactions = new ArrayList<>(); AtomicInteger requestsReceived = new AtomicInteger(0); - accumulateTrafficStreamsWithNewAccumulator(trafficStreams, reconstructedTransactions, requestsReceived); + accumulateTrafficStreamsWithNewAccumulator(rootContext, trafficStreams, reconstructedTransactions, + requestsReceived); var splitSizes = unzipRequestResponseSizes(expectedSizes); assertReconstructedTransactionsMatchExpectations(reconstructedTransactions, splitSizes._1, splitSizes._2); Assertions.assertEquals(requestsReceived.get(), reconstructedTransactions.size()); @@ -203,21 +213,24 @@ void generateAndTest(String testName, int bufferSize, int skipCount, * @return */ static SortedSet - accumulateTrafficStreamsWithNewAccumulator(Stream trafficStreams, - List aggregations, - AtomicInteger requestsReceived) { + accumulateTrafficStreamsWithNewAccumulator(TestContext context, + Stream trafficStreams, + List aggregations, + AtomicInteger requestsReceived) { var tsIndicesReceived = new TreeSet(); CapturedTrafficToHttpTransactionAccumulator trafficAccumulator = new CapturedTrafficToHttpTransactionAccumulator(Duration.ofSeconds(30), null, new AccumulationCallbacks() { @Override - public void onRequestReceived(UniqueReplayerRequestKey key, HttpMessageAndTimestamp request) { + public void onRequestReceived(@NonNull IReplayContexts.IReplayerHttpTransactionContext ctx, + @NonNull HttpMessageAndTimestamp request) { requestsReceived.incrementAndGet(); } @Override - public void onFullDataReceived(UniqueReplayerRequestKey requestKey, RequestResponsePacketPair fullPair) { - var sourceIdx = requestKey.getSourceRequestIndex(); + public void onFullDataReceived(@NonNull IReplayContexts.IReplayerHttpTransactionContext ctx, + @NonNull RequestResponsePacketPair fullPair) { + var sourceIdx = ctx.getReplayerRequestKey().getSourceRequestIndex(); if (fullPair.completionStatus == RequestResponsePacketPair.ReconstructionStatus.CLOSED_PREMATURELY) { return; @@ -236,22 +249,28 @@ public void onFullDataReceived(UniqueReplayerRequestKey requestKey, RequestRespo @Override public void onTrafficStreamsExpired(RequestResponsePacketPair.ReconstructionStatus status, - List trafficStreamKeysBeingHeld) {} + IReplayContexts.@NonNull IChannelKeyContext ctx, + @NonNull List trafficStreamKeysBeingHeld) {} @Override - public void onConnectionClose(ISourceTrafficChannelKey key, int channelInteractionNumber, + public void onConnectionClose(int channelInteractionNumber, + @NonNull IReplayContexts.IChannelKeyContext ctx, RequestResponsePacketPair.ReconstructionStatus status, - Instant when, - List trafficStreamKeysBeingHeld) { + @NonNull Instant when, + @NonNull List trafficStreamKeysBeingHeld) { } - @Override public void onTrafficStreamIgnored(@NonNull ITrafficStreamKey tsk) { - tsIndicesReceived.add(tsk.getTrafficStreamIndex()); + @Override public void onTrafficStreamIgnored(@NonNull IReplayContexts.ITrafficStreamsLifecycleContext ctx) { + tsIndicesReceived.add(ctx.getTrafficStreamKey().getTrafficStreamIndex()); } }); var tsList = trafficStreams.collect(Collectors.toList()); trafficStreams = tsList.stream(); - trafficStreams.forEach(ts->trafficAccumulator.accept(new TrafficStreamWithEmbeddedKey(ts))); + ; + trafficStreams.forEach(ts->trafficAccumulator.accept( + new PojoTrafficStreamAndKey(ts, PojoTrafficStreamKeyAndContext.build(ts, + context::createTrafficStreamContextForTest) + ))); trafficAccumulator.close(); return tsIndicesReceived; } diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/TimeShifterTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/TimeShifterTest.java index 9d49b4267..054f4e7ea 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/TimeShifterTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/TimeShifterTest.java @@ -8,8 +8,6 @@ import java.time.Instant; import java.util.Optional; -import static org.junit.jupiter.api.Assertions.*; - @WrapWithNettyLeakDetection(disableLeakChecks = true) class TimeShifterTest { diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/TrafficReplayerRunner.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/TrafficReplayerRunner.java index e5ce4b09d..fe8866fb2 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/TrafficReplayerRunner.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/TrafficReplayerRunner.java @@ -5,10 +5,9 @@ import lombok.extern.slf4j.Slf4j; import org.junit.jupiter.api.Assertions; import org.opensearch.migrations.replay.datatypes.ISourceTrafficChannelKey; -import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; import org.opensearch.migrations.replay.traffic.source.BlockingTrafficSource; import org.opensearch.migrations.replay.traffic.source.ISimpleTrafficCaptureSource; -import org.opensearch.migrations.testutils.SimpleNettyHttpServer; +import org.opensearch.migrations.tracing.TestContext; import org.opensearch.migrations.transform.StaticAuthTransformerFactory; import org.slf4j.event.Level; import org.testcontainers.shaded.org.apache.commons.io.output.NullOutputStream; @@ -18,11 +17,11 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Optional; -import java.util.Random; import java.util.StringJoiner; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; +import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -39,7 +38,8 @@ private TrafficReplayerRunner() {} static void runReplayerUntilSourceWasExhausted(int numExpectedRequests, URI endpoint, Supplier> tupleListenerSupplier, - Supplier trafficSourceSupplier) + Supplier rootContextSupplier, + Function trafficSourceFactory) throws Throwable { AtomicInteger runNumberRef = new AtomicInteger(); var totalUniqueEverReceived = new AtomicInteger(); @@ -52,15 +52,15 @@ static void runReplayerUntilSourceWasExhausted(int numExpectedRequests, URI endp int runNumber = runNumberRef.get(); var counter = new AtomicInteger(); var tupleReceiver = tupleListenerSupplier.get(); - try { - runTrafficReplayer(trafficSourceSupplier, endpoint, (t) -> { + try (var rootContext = rootContextSupplier.get()) { + runTrafficReplayer(rootContext, ()->trafficSourceFactory.apply(rootContext), endpoint, (t) -> { if (runNumber != runNumberRef.get()) { // for an old replayer. I'm not sure why shutdown isn't blocking until all threads are dead, // but that behavior only impacts this test as far as I can tell. return; } Assertions.assertEquals(runNumber, runNumberRef.get()); - var key = t.uniqueRequestKey; + var key = t.getRequestKey(); ISourceTrafficChannelKey tsk = key.getTrafficStreamKey(); var keyString = tsk.getConnectionId() + "_" + key.getSourceRequestIndex(); var prevKeyString = tsk.getConnectionId() + "_" + (key.getSourceRequestIndex()-1); @@ -136,11 +136,12 @@ static void runReplayerUntilSourceWasExhausted(int numExpectedRequests, URI endp Assertions.assertEquals(numExpectedRequests, totalUniqueEverReceived.get()); } - private static void runTrafficReplayer(Supplier captureSourceSupplier, + private static void runTrafficReplayer(TestContext rootContext, + Supplier captureSourceSupplier, URI endpoint, Consumer tupleReceiver) throws Exception { log.info("Starting a new replayer and running it"); - var tr = new TrafficReplayer(endpoint, null, + var tr = new TrafficReplayer(rootContext, endpoint, null, new StaticAuthTransformerFactory("TEST"), null, true, 10, 10*1024); 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..9f54daa34 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 @@ -6,11 +6,11 @@ import lombok.extern.slf4j.Slf4j; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -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.IReplayContexts; import org.opensearch.migrations.replay.traffic.source.InputStreamOfTraffic; import org.opensearch.migrations.testutils.WrapWithNettyLeakDetection; +import org.opensearch.migrations.tracing.InstrumentationTest; import org.opensearch.migrations.trafficcapture.protos.CloseObservation; import org.opensearch.migrations.trafficcapture.protos.ConnectionExceptionObservation; import org.opensearch.migrations.trafficcapture.protos.EndOfMessageIndication; @@ -37,7 +37,7 @@ @Slf4j @WrapWithNettyLeakDetection -class TrafficReplayerTest { +class TrafficReplayerTest extends InstrumentationTest { public static final String TEST_NODE_ID_STRING = "test_node_id"; private static String TEST_TRAFFIC_STREAM_ID_STRING = "testId"; @@ -113,9 +113,10 @@ public void testDelimitedDeserializer() throws Exception { try (var bais = new ByteArrayInputStream(serializedChunks)) { AtomicInteger counter = new AtomicInteger(0); var allMatch = new AtomicBoolean(true); - try (var trafficProducer = new InputStreamOfTraffic(bais)) { + try (var trafficProducer = new InputStreamOfTraffic(rootContext, bais)) { while (true) { - trafficProducer.readNextTrafficStreamChunk().get().stream().forEach(ts->{ + trafficProducer.readNextTrafficStreamChunk(rootContext::createReadChunkContext).get().stream() + .forEach(ts->{ var i = counter.incrementAndGet(); var expectedStream = makeTrafficStream(timestamp.plus(i - 1, ChronoUnit.SECONDS), i); var isEqual = ts.getStream().equals(expectedStream); @@ -148,51 +149,61 @@ static byte[] synthesizeTrafficStreamsIntoByteArray(Instant timestamp, int numSt @Test public void testReader() throws Exception { - var tr = new TrafficReplayer(new URI("http://localhost:9200"), null, null, false); + var tr = new TrafficReplayer(rootContext, + new URI("http://localhost:9200"), null, null, false); List> byteArrays = new ArrayList<>(); CapturedTrafficToHttpTransactionAccumulator trafficAccumulator = new CapturedTrafficToHttpTransactionAccumulator(Duration.ofSeconds(30), null, new AccumulationCallbacks() { @Override - public void onRequestReceived(UniqueReplayerRequestKey id, HttpMessageAndTimestamp request) { + public void onRequestReceived(@NonNull IReplayContexts.IReplayerHttpTransactionContext ctx, + @NonNull 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) { - var responseBytes = fullPair.responseData.packetBytes.stream().collect(Collectors.toList()); + public void onFullDataReceived(@NonNull IReplayContexts.IReplayerHttpTransactionContext ctx, + @NonNull RequestResponsePacketPair fullPair) { + var responseBytes = new ArrayList(fullPair.responseData.packetBytes); Assertions.assertEquals(FAKE_READ_PACKET_DATA, collectBytesToUtf8String(responseBytes)); } @Override public void onTrafficStreamsExpired(RequestResponsePacketPair.ReconstructionStatus status, - List trafficStreamKeysBeingHeld) {} + IReplayContexts.@NonNull IChannelKeyContext ctx, + @NonNull List trafficStreamKeysBeingHeld) { + } @Override - public void onConnectionClose(ISourceTrafficChannelKey key, int channelInteractionNumber, + public void onConnectionClose(int channelInteractionNumber, + IReplayContexts.@NonNull IChannelKeyContext ctx, RequestResponsePacketPair.ReconstructionStatus status, - Instant when, - List trafficStreamKeysBeingHeld) { + @NonNull Instant when, + @NonNull List trafficStreamKeysBeingHeld) { } - @Override public void onTrafficStreamIgnored(@NonNull ITrafficStreamKey tsk) {} + @Override + public void onTrafficStreamIgnored(@NonNull IReplayContexts.ITrafficStreamsLifecycleContext ctx) { + + } }); var bytes = synthesizeTrafficStreamsIntoByteArray(Instant.now(), 1); try (var bais = new ByteArrayInputStream(bytes)) { - try (var trafficSource = new InputStreamOfTraffic(bais)) { + try (var trafficSource = new InputStreamOfTraffic(rootContext, bais)) { tr.pullCaptureFromSourceToAccumulator(trafficSource, trafficAccumulator); } } Assertions.assertEquals(1, byteArrays.size()); - Assertions.assertTrue(byteArrays.stream().allMatch(ba->ba.size()==2)); + Assertions.assertTrue(byteArrays.stream().allMatch(ba -> ba.size() == 2)); } @Test public void testCapturedReadsAfterCloseAreHandledAsNew() throws Exception { - var tr = new TrafficReplayer(new URI("http://localhost:9200"), null, null, false); + var tr = new TrafficReplayer(rootContext, + new URI("http://localhost:9200"), null, null, false); List> byteArrays = new ArrayList<>(); var remainingAccumulations = new AtomicInteger(); CapturedTrafficToHttpTransactionAccumulator trafficAccumulator = @@ -201,29 +212,37 @@ 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(@NonNull IReplayContexts.IReplayerHttpTransactionContext ctx, + @NonNull 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) { - var responseBytes = fullPair.responseData.packetBytes.stream().collect(Collectors.toList()); + public void onFullDataReceived(@NonNull IReplayContexts.IReplayerHttpTransactionContext ctx, + @NonNull RequestResponsePacketPair fullPair) { + var responseBytes = new ArrayList(fullPair.responseData.packetBytes); Assertions.assertEquals(FAKE_READ_PACKET_DATA, collectBytesToUtf8String(responseBytes)); } @Override public void onTrafficStreamsExpired(RequestResponsePacketPair.ReconstructionStatus status, - List trafficStreamKeysBeingHeld) {} + @NonNull IReplayContexts.IChannelKeyContext ctx, + @NonNull List trafficStreamKeysBeingHeld) { + } @Override - public void onConnectionClose(ISourceTrafficChannelKey key, int channelInteractionNumber, + public void onConnectionClose(int channelInteractionNumber, + @NonNull IReplayContexts.IChannelKeyContext ctx, RequestResponsePacketPair.ReconstructionStatus status, - Instant when, - List trafficStreamKeysBeingHeld) { + @NonNull Instant when, + @NonNull List trafficStreamKeysBeingHeld) { + } + + @Override + public void onTrafficStreamIgnored(@NonNull IReplayContexts.ITrafficStreamsLifecycleContext ctx) { } - @Override public void onTrafficStreamIgnored(@NonNull ITrafficStreamKey tsk) {} } ); byte[] serializedChunks; @@ -244,13 +263,13 @@ public void onConnectionClose(ISourceTrafficChannelKey key, int channelInteracti } try (var bais = new ByteArrayInputStream(serializedChunks)) { - try (var trafficSource = new InputStreamOfTraffic(bais)) { + try (var trafficSource = new InputStreamOfTraffic(rootContext, bais)) { tr.pullCaptureFromSourceToAccumulator(trafficSource, trafficAccumulator); } } trafficAccumulator.close(); Assertions.assertEquals(2, byteArrays.size()); - Assertions.assertTrue(byteArrays.stream().allMatch(ba->ba.size()==2)); + Assertions.assertTrue(byteArrays.stream().allMatch(ba -> ba.size() == 2)); Assertions.assertEquals(0, remainingAccumulations.get()); } diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/TrafficStreamGenerator.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/TrafficStreamGenerator.java index abd0f17e2..421eba8b2 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/TrafficStreamGenerator.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/TrafficStreamGenerator.java @@ -4,6 +4,7 @@ import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import org.opensearch.migrations.testutils.StreamInterleaver; +import org.opensearch.migrations.tracing.TestContext; import org.opensearch.migrations.trafficcapture.protos.TrafficObservation; import org.opensearch.migrations.trafficcapture.protos.TrafficStream; import org.opensearch.migrations.trafficcapture.protos.TrafficStreamUtils; @@ -209,9 +210,10 @@ private static void fillCommandsAndSizes(Random r, double cancelRequestLikelihoo } @SneakyThrows - private static TrafficStream[] fillCommandsAndSizesForSeed(long rSeed, - ArrayList commands, - ArrayList sizes) { + private static TrafficStream[] + fillCommandsAndSizesForSeed(TestContext rootContext, long rSeed, AtomicInteger uniqueIdCounter, + ArrayList commands, + ArrayList sizes) { var r2 = new Random(rSeed); var bufferSize = r2.nextInt(MAX_BUFFER_SIZE-MIN_BUFFER_SIZE) + MIN_BUFFER_SIZE; final var bufferBound = (int)(Math.abs(r2.nextGaussian()) * ((MAX_BUFFER_SIZE_MULTIPLIER * bufferSize)))+1; @@ -221,7 +223,8 @@ private static TrafficStream[] fillCommandsAndSizesForSeed(long rSeed, .log(); var flushLikelihood = Math.pow(r2.nextDouble(),2.0); fillCommandsAndSizes(r2, flushLikelihood/4, flushLikelihood, bufferBound, commands, sizes); - return SimpleCapturedTrafficToHttpTransactionAccumulatorTest.makeTrafficStreams(bufferSize, (int) rSeed, commands); + return SimpleCapturedTrafficToHttpTransactionAccumulatorTest.makeTrafficStreams(bufferSize, (int) rSeed, + uniqueIdCounter, commands, rootContext); } /** @@ -266,12 +269,12 @@ public static class StreamAndExpectedSizes { } static StreamAndExpectedSizes - generateStreamAndSumOfItsTransactions(int count, boolean randomize) { + generateStreamAndSumOfItsTransactions(TestContext rootContext, int count, boolean randomize) { var generatedCases = count > 0 ? - generateRandomTrafficStreamsAndSizes(IntStream.range(0,count)) : - generateAllIndicativeRandomTrafficStreamsAndSizes(); + generateRandomTrafficStreamsAndSizes(rootContext, IntStream.range(0, count)) : + generateAllIndicativeRandomTrafficStreamsAndSizes(rootContext); var testCaseArr = generatedCases.toArray(RandomTrafficStreamAndTransactionSizes[]::new); - log.atInfo().setMessage(()-> + log.atInfo().setMessage(()-> "test case array = \n" + Arrays.stream(testCaseArr) .flatMap(tc->Arrays.stream(tc.trafficStreams).map(TrafficStreamUtils::summarizeTrafficStream)) .collect(Collectors.joining("\n"))) @@ -286,11 +289,12 @@ public static class StreamAndExpectedSizes { } public static Stream - generateRandomTrafficStreamsAndSizes(IntStream seedStream) { + generateRandomTrafficStreamsAndSizes(TestContext rootContext, IntStream seedStream) { + var uniqueIdCounter = new AtomicInteger(); return seedStream.mapToObj(rSeed->{ var commands = new ArrayList(); var sizes = new ArrayList(); - var trafficStreams = fillCommandsAndSizesForSeed(rSeed, commands, sizes); + var trafficStreams = fillCommandsAndSizesForSeed(rootContext, rSeed, uniqueIdCounter, commands, sizes); var splitSizes = SimpleCapturedTrafficToHttpTransactionAccumulatorTest.unzipRequestResponseSizes(sizes); return new RandomTrafficStreamAndTransactionSizes(rSeed, trafficStreams, @@ -298,8 +302,9 @@ public static class StreamAndExpectedSizes { }).filter(o->o!=null); } - public static Stream generateAllIndicativeRandomTrafficStreamsAndSizes() { - return generateRandomTrafficStreamsAndSizes( + public static Stream + generateAllIndicativeRandomTrafficStreamsAndSizes(TestContext rootContext) { + return generateRandomTrafficStreamsAndSizes(rootContext, RANDOM_GENERATOR_SEEDS_FOR_SUFFICIENT_TRAFFIC_VARIANCE.stream().mapToInt(i->i)); } } diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/UtilsTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/UtilsTest.java index f46c33efd..03b9b0fd6 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/UtilsTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/UtilsTest.java @@ -3,6 +3,7 @@ import lombok.extern.slf4j.Slf4j; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import org.opensearch.migrations.Utils; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -17,7 +18,7 @@ public void testFoldLeft() { .collect(Collectors.joining()); var foldedValue = - IntStream.range('A','F').mapToObj(c->(char)c+"").collect(Utils.foldLeft("", (a,b) -> a+b)); + IntStream.range('A','F').mapToObj(c->(char)c+"").collect(Utils.foldLeft("", (a, b) -> a+b)); log.info("stream concatenated value: " + foldedValue); Assertions.assertEquals(groundTruth, foldedValue); diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/V0_1TrafficCaptureSource.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/V0_1TrafficCaptureSource.java index e350bc0a0..d05f4c010 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/V0_1TrafficCaptureSource.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/V0_1TrafficCaptureSource.java @@ -1,22 +1,20 @@ package org.opensearch.migrations.replay; +import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamAndKey; +import org.opensearch.migrations.replay.tracing.RootReplayerContext; import org.opensearch.migrations.replay.traffic.source.ITrafficStreamWithKey; -import org.opensearch.migrations.replay.traffic.source.InputStreamOfTraffic; -import org.opensearch.migrations.replay.traffic.source.TrafficStreamWithEmbeddedKey; import org.opensearch.migrations.trafficcapture.protos.TrafficStream; -import java.io.FileInputStream; import java.io.IOException; import java.util.HashMap; import java.util.Optional; -import java.util.zip.GZIPInputStream; public class V0_1TrafficCaptureSource extends CompressedFileTrafficCaptureSource { protected final HashMap connectionProgressMap; - public V0_1TrafficCaptureSource(String filename) throws IOException { - super(filename); + public V0_1TrafficCaptureSource(RootReplayerContext context, String filename) throws IOException { + super(context, filename); connectionProgressMap = new HashMap<>(); } @@ -32,9 +30,10 @@ protected ITrafficStreamWithKey modifyTrafficStream(ITrafficStreamWithKey stream outgoingBuilder.setNumberOfThisLastChunk(incoming.getNumberOfThisLastChunk()); } var progress = connectionProgressMap.get(incoming.getConnectionId()); + var key = streamWithKey.getKey(); if (progress == null) { progress = new Progress(); - progress.lastWasRead = streamWithKey.getKey().getTrafficStreamIndex() != 0; + progress.lastWasRead = key.getTrafficStreamIndex() != 0; connectionProgressMap.put(incoming.getConnectionId(), progress); } outgoingBuilder.setLastObservationWasUnterminatedRead(progress.lastWasRead); @@ -44,7 +43,7 @@ protected ITrafficStreamWithKey modifyTrafficStream(ITrafficStreamWithKey stream if (incoming.hasNumberOfThisLastChunk()) { connectionProgressMap.remove(incoming.getConnectionId()); } - return new TrafficStreamWithEmbeddedKey(outgoingBuilder.build()); + return new PojoTrafficStreamAndKey(outgoingBuilder.build(), key); } private static class Progress { 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..c79986c89 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 @@ -16,12 +16,9 @@ import org.opensearch.migrations.replay.PacketToTransformingHttpHandlerFactory; import org.opensearch.migrations.replay.ReplayEngine; import org.opensearch.migrations.replay.RequestSenderOrchestrator; -import org.opensearch.migrations.replay.TestRequestKey; import org.opensearch.migrations.replay.TimeShifter; import org.opensearch.migrations.replay.TrafficReplayer; 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.traffic.source.BufferedFlowController; import org.opensearch.migrations.testutils.HttpFirstLine; import org.opensearch.migrations.testutils.PortFinder; @@ -29,6 +26,8 @@ import org.opensearch.migrations.testutils.SimpleHttpClientForTesting; import org.opensearch.migrations.testutils.SimpleHttpServer; import org.opensearch.migrations.testutils.WrapWithNettyLeakDetection; +import org.opensearch.migrations.tracing.InstrumentationTest; +import org.opensearch.migrations.tracing.TestContext; import javax.net.ssl.SSLException; import java.io.IOException; @@ -47,7 +46,7 @@ @Slf4j @WrapWithNettyLeakDetection -public class NettyPacketToHttpConsumerTest { +public class NettyPacketToHttpConsumerTest extends InstrumentationTest { public static final String SERVER_RESPONSE_BODY = "I should be decrypted tester!\n"; @@ -77,8 +76,8 @@ public class NettyPacketToHttpConsumerTest { @BeforeAll public static void setupTestServer() throws PortFinder.ExceededMaxPortAssigmentAttemptException { testServers = Map.of( - false, SimpleHttpServer.makeServer(false, NettyPacketToHttpConsumerTest::makeContext), - true, SimpleHttpServer.makeServer(true, NettyPacketToHttpConsumerTest::makeContext)); + false, SimpleHttpServer.makeServer(false, NettyPacketToHttpConsumerTest::makeResponseContext), + true, SimpleHttpServer.makeServer(true, NettyPacketToHttpConsumerTest::makeResponseContext)); } @AfterAll @@ -92,6 +91,11 @@ public static void tearDownTestServer() throws Exception { }); } + @Override + protected TestContext makeInstrumentationContext() { + return TestContext.withTracking(false, true); + } + @Test public void testThatTestSetupIsCorrect() throws IOException, NoSuchAlgorithmException, KeyStoreException, KeyManagementException @@ -112,7 +116,7 @@ private SimpleHttpResponse makeTestRequestViaClient(SimpleHttpClientForTesting c "User-Agent", "UnitTest").entrySet().stream()); } - private static SimpleHttpResponse makeContext(HttpFirstLine request) { + private static SimpleHttpResponse makeResponseContext(HttpFirstLine request) { var headers = Map.of( "Content-Type", "text/plain", "Funtime", "checkIt!", @@ -128,9 +132,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, + rootContext.getTestConnectionRequestContext(0)); nphc.consumeBytes((EXPECTED_REQUEST_STRING).getBytes(StandardCharsets.UTF_8)); var aggregatedResponse = nphc.finalizeRequest().get(); var responseBytePackets = aggregatedResponse.getCopyOfPackets(); @@ -145,10 +151,9 @@ public void testHttpResponseIsSuccessfullyCaptured(boolean useTls) throws Except @ParameterizedTest @ValueSource(booleans = {false, true}) public void testThatConnectionsAreKeptAliveAndShared(boolean useTls) - throws SSLException, ExecutionException, InterruptedException - { + throws SSLException, ExecutionException, InterruptedException { var testServer = testServers.get(useTls); - var sslContext = !testServer.localhostEndpoint().getScheme().toLowerCase().equals("https") ? null : + var sslContext = !testServer.localhostEndpoint().getScheme().equalsIgnoreCase("https") ? null : SslContextBuilder.forClient().trustManager(InsecureTrustManagerFactory.INSTANCE).build(); var transformingHttpHandlerFactory = new PacketToTransformingHttpHandlerFactory( new TransformationLoader().getTransformerFactoryLoader(null), null); @@ -158,15 +163,13 @@ public void testThatConnectionsAreKeptAliveAndShared(boolean useTls) new RequestSenderOrchestrator( new ClientConnectionPool(testServer.localhostEndpoint(), sslContext, 1)), new TestFlowController(), timeShifter); - for (int j=0; j<2; ++j) { + 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 = rootContext.getTestConnectionRequestContext("TEST_" + i, j); var requestFinishFuture = TrafficReplayer.transformAndSendRequest(transformingHttpHandlerFactory, - sendingFactory, Instant.now(), Instant.now(), - new UniqueReplayerRequestKey(trafficStreamKey, 0, i), - ()->Stream.of(EXPECTED_REQUEST_STRING.getBytes(StandardCharsets.UTF_8))); - log.info("requestFinishFuture="+requestFinishFuture); + sendingFactory, ctx, Instant.now(), Instant.now(), + () -> Stream.of(EXPECTED_REQUEST_STRING.getBytes(StandardCharsets.UTF_8))); + log.info("requestFinishFuture=" + requestFinishFuture); var aggregatedResponse = requestFinishFuture.get(); log.debug("Got aggregated response=" + aggregatedResponse); Assertions.assertNull(aggregatedResponse.getError()); @@ -181,8 +184,24 @@ public void testThatConnectionsAreKeptAliveAndShared(boolean useTls) var stopFuture = sendingFactory.closeConnectionsAndShutdown(); log.info("waiting for factory to shutdown: " + stopFuture); stopFuture.get(); - Assertions.assertEquals(2, sendingFactory.getNumConnectionsCreated()); - Assertions.assertEquals(2, sendingFactory.getNumConnectionsClosed()); + } + + @ParameterizedTest + @ValueSource(booleans = {false, true}) + @WrapWithNettyLeakDetection(repetitions = 1) + public void testMetricCountsFor_testThatConnectionsAreKeptAliveAndShared(boolean useTls) throws Exception { + testThatConnectionsAreKeptAliveAndShared(useTls); + Thread.sleep(200); // let metrics settle down + var allMetricData = rootContext.inMemoryInstrumentationBundle.testMetricExporter.getFinishedMetricItems(); + long tcpOpenConnectionCount = allMetricData.stream().filter(md->md.getName().startsWith("tcpConnectionCount")) + .reduce((a,b)->b).get().getLongSumData().getPoints().stream().reduce((a,b)->b).get().getValue(); + long connectionsOpenedCount = allMetricData.stream().filter(md->md.getName().startsWith("connectionsOpened")) + .reduce((a,b)->b).get().getLongSumData().getPoints().stream().reduce((a,b)->b).get().getValue(); + long connectionsClosedCount = allMetricData.stream().filter(md->md.getName().startsWith("connectionsClosed")) + .reduce((a,b)->b).get().getLongSumData().getPoints().stream().reduce((a,b)->b).get().getValue(); + Assertions.assertEquals(2, tcpOpenConnectionCount); + Assertions.assertEquals(2, connectionsOpenedCount); + Assertions.assertEquals(2, connectionsClosedCount); } private static String normalizeMessage(String s) { 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..f3ee45482 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 @@ -4,10 +4,10 @@ import org.junit.jupiter.api.Test; import org.opensearch.migrations.replay.AggregatedRawResponse; import org.opensearch.migrations.replay.TestCapturePacketToHttpHandler; -import org.opensearch.migrations.replay.TestRequestKey; import org.opensearch.migrations.replay.TransformationLoader; import org.opensearch.migrations.replay.datatypes.HttpRequestTransformationStatus; import org.opensearch.migrations.testutils.WrapWithNettyLeakDetection; +import org.opensearch.migrations.tracing.InstrumentationTest; import org.opensearch.migrations.transform.IJsonTransformer; import org.opensearch.migrations.transform.JsonCompositeTransformer; import org.opensearch.migrations.transform.RemovingAuthTransformerFactory; @@ -18,7 +18,7 @@ import java.util.Map; @WrapWithNettyLeakDetection -class HttpJsonTransformingConsumerTest { +class HttpJsonTransformingConsumerTest extends InstrumentationTest { @Test public void testPassThroughSinglePacketPost() throws Exception { final var dummyAggregatedResponse = @@ -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, + rootContext.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, + rootContext.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, + rootContext.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, rootContext.getTestConnectionRequestContext(0)); byte[] testBytes; try (var sampleStream = HttpJsonTransformingConsumer.class.getResourceAsStream( "/requests/raw/post_formUrlEncoded_withFixedLength.txt")) { @@ -131,6 +130,4 @@ private void walkMaps(Object o) { Assertions.assertInstanceOf(NettyJsonBodyAccumulateHandler.IncompleteJsonBodyException.class, returnedResponse.error); } - - } \ No newline at end of file diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datatypes/MockMetricsBuilder.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datatypes/MockMetricsBuilder.java deleted file mode 100644 index e1d2c2ed1..000000000 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datatypes/MockMetricsBuilder.java +++ /dev/null @@ -1,24 +0,0 @@ -package org.opensearch.migrations.replay.datatypes; - -import org.opensearch.migrations.coreutils.MetricsAttributeKey; -import org.opensearch.migrations.coreutils.MetricsLogBuilder; - -import java.util.StringJoiner; - -public class MockMetricsBuilder extends MetricsLogBuilder { - StringJoiner attributeLogger = new StringJoiner("|"); - - public MockMetricsBuilder() { - super(null); - } - - @Override - public MetricsLogBuilder setAttribute(MetricsAttributeKey key, Object value) { - attributeLogger.add(key + ":" + value); - return this; - } - - public String getLoggedAttributes() { - return attributeLogger.toString(); - } -} diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaCommitsWorkBetweenLongPolls.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaCommitsWorkBetweenLongPollsTest.java similarity index 85% rename from TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaCommitsWorkBetweenLongPolls.java rename to TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaCommitsWorkBetweenLongPollsTest.java index cbc0bed4d..ab2771053 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaCommitsWorkBetweenLongPolls.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaCommitsWorkBetweenLongPollsTest.java @@ -10,8 +10,7 @@ import org.junit.jupiter.api.Test; import org.opensearch.migrations.replay.traffic.source.BlockingTrafficSource; import org.opensearch.migrations.replay.traffic.source.ITrafficStreamWithKey; -import org.opensearch.migrations.replay.traffic.source.TrafficStreamWithEmbeddedKey; -import org.opensearch.migrations.trafficcapture.protos.TrafficStream; +import org.opensearch.migrations.tracing.InstrumentationTest; import org.testcontainers.containers.KafkaContainer; import org.testcontainers.junit.jupiter.Container; import org.testcontainers.junit.jupiter.Testcontainers; @@ -20,14 +19,12 @@ import java.time.Duration; import java.time.Instant; import java.util.List; -import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.locks.Condition; @Slf4j @Testcontainers(disabledWithoutDocker = true) @Tag("requiresDocker") -public class KafkaCommitsWorkBetweenLongPolls { +public class KafkaCommitsWorkBetweenLongPollsTest extends InstrumentationTest { private static final long DEFAULT_POLL_INTERVAL_MS = 1000; private static final int NUM_RUNS = 5; public static final String TEST_TOPIC_NAME = "test-topic"; @@ -49,7 +46,7 @@ private KafkaConsumer buildKafkaConsumer() { @Test @Tag("longTest") public void testThatCommitsAndReadsKeepWorking() throws Exception { - var kafkaSource = new KafkaTrafficCaptureSource(buildKafkaConsumer(), + var kafkaSource = new KafkaTrafficCaptureSource(rootContext, buildKafkaConsumer(), TEST_TOPIC_NAME, Duration.ofMillis(DEFAULT_POLL_INTERVAL_MS/3)); var blockingSource = new BlockingTrafficSource(kafkaSource, Duration.ofMinutes(5)); var kafkaProducer = KafkaTestUtils.buildKafkaProducer(embeddedKafkaBroker.getBootstrapServers()); @@ -79,7 +76,7 @@ public void testThatCommitsAndReadsKeepWorking() throws Exception { for (int i=0; i kafkaProducer, int i) { var ts = KafkaTestUtils.makeTestTrafficStreamWithFixedTime(getTimeAtPoint(i), i); - KafkaTestUtils.writeTrafficStreamRecord(kafkaProducer, new TrafficStreamWithEmbeddedKey(ts), - TEST_TOPIC_NAME, ""+i); + KafkaTestUtils.writeTrafficStreamRecord(kafkaProducer, ts, TEST_TOPIC_NAME, ""+i); } } diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaKeepAliveTests.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaKeepAliveTests.java index b2319765a..c88d5220b 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaKeepAliveTests.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaKeepAliveTests.java @@ -11,6 +11,8 @@ import org.junit.jupiter.api.Test; import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; import org.opensearch.migrations.replay.traffic.source.BlockingTrafficSource; +import org.opensearch.migrations.tracing.InstrumentationTest; +import org.opensearch.migrations.tracing.TestContext; import org.testcontainers.containers.KafkaContainer; import org.testcontainers.junit.jupiter.Container; import org.testcontainers.junit.jupiter.Testcontainers; @@ -28,7 +30,7 @@ @Slf4j @Testcontainers(disabledWithoutDocker = true) @Tag("requiresDocker") -public class KafkaKeepAliveTests { +public class KafkaKeepAliveTests extends InstrumentationTest { public static final String TEST_GROUP_CONSUMER_ID = "TEST_GROUP_CONSUMER_ID"; public static final String HEARTBEAT_INTERVAL_MS_KEY = "heartbeat.interval.ms"; public static final long MAX_POLL_INTERVAL_MS = 1000; @@ -45,8 +47,8 @@ public class KafkaKeepAliveTests { // see https://docs.confluent.io/platform/current/installation/versions-interoperability.html#cp-and-apache-kafka-compatibility private final KafkaContainer embeddedKafkaBroker = new KafkaContainer(DockerImageName.parse("confluentinc/cp-kafka:7.5.0")); - private KafkaTrafficCaptureSource kafkaSource; + private KafkaTrafficCaptureSource kafkaSource; /** * Set up the test case where we've produced and received 1 message, but have not yet committed it. * Another message is in the process of being produced. @@ -68,11 +70,12 @@ private void setupTestCase() throws Exception { kafkaProperties.put(HEARTBEAT_INTERVAL_MS_KEY, HEARTBEAT_INTERVAL_MS+""); kafkaProperties.put("max.poll.records", 1); var kafkaConsumer = new KafkaConsumer(kafkaProperties); - this.kafkaSource = new KafkaTrafficCaptureSource(kafkaConsumer, testTopicName, Duration.ofMillis(MAX_POLL_INTERVAL_MS)); + this.kafkaSource = new KafkaTrafficCaptureSource(rootContext, + kafkaConsumer, testTopicName, Duration.ofMillis(MAX_POLL_INTERVAL_MS)); this.trafficSource = new BlockingTrafficSource(kafkaSource, Duration.ZERO); this.keysReceived = new ArrayList<>(); - readNextNStreams(trafficSource, keysReceived, 0, 1); + readNextNStreams(rootContext, trafficSource, keysReceived, 0, 1); KafkaTestUtils.produceKafkaRecord(testTopicName, kafkaProducer, 1, sendCompleteCount); } @@ -98,7 +101,7 @@ public void testTimeoutsDontOccurForSlowPolls() throws Exception { pollIntervalMs, TimeUnit.MILLISECONDS); // wait for 2 messages so that they include the last one produced by the async schedule call previously - readNextNStreams(trafficSource, keysReceived, 1, 2); + readNextNStreams(rootContext, trafficSource, keysReceived, 1, 2); Assertions.assertEquals(3, keysReceived.size()); // At this point, we've read all (3) messages produced , committed the first one // (all the way through to Kafka), and no commits are in-flight yet for the last two messages. @@ -110,7 +113,7 @@ public void testBlockedReadsAndBrokenCommitsDontCauseReordering() throws Excepti for (int i=0; i<2; ++i) { KafkaTestUtils.produceKafkaRecord(testTopicName, kafkaProducer, 1 + i, sendCompleteCount).get(); } - readNextNStreams(trafficSource, keysReceived, 1, 1); + readNextNStreams(rootContext, trafficSource, keysReceived, 1, 1); trafficSource.commitTrafficStream(keysReceived.get(0)); log.info("Called commitTrafficStream but waiting long enough for the client to leave the group. " + @@ -125,7 +128,7 @@ public void testBlockedReadsAndBrokenCommitsDontCauseReordering() throws Excepti log.info("re-establish a client connection so that the following commit will work"); log.atInfo().setMessage(()->"1 ..."+renderNextCommitsAsString()).log(); - readNextNStreams(trafficSource, keysReceived, 0, 1); + readNextNStreams(rootContext, trafficSource, keysReceived, 0, 1); log.atInfo().setMessage(()->"2 ..."+renderNextCommitsAsString()).log(); log.info("wait long enough to fall out of the group again"); @@ -134,16 +137,16 @@ public void testBlockedReadsAndBrokenCommitsDontCauseReordering() throws Excepti var keysReceivedUntilDrop2 = keysReceived; keysReceived = new ArrayList<>(); log.atInfo().setMessage(()->"re-establish... 3 ..."+renderNextCommitsAsString()).log(); - readNextNStreams(trafficSource, keysReceived, 0, 1); + readNextNStreams(rootContext, trafficSource, keysReceived, 0, 1); trafficSource.commitTrafficStream(keysReceivedUntilDrop1.get(1)); log.atInfo().setMessage(()->"re-establish... 4 ..."+renderNextCommitsAsString()).log(); - readNextNStreams(trafficSource, keysReceived, 1, 1); + readNextNStreams(rootContext, trafficSource, keysReceived, 1, 1); log.atInfo().setMessage(()->"5 ..."+renderNextCommitsAsString()).log(); Thread.sleep(2*MAX_POLL_INTERVAL_MS); var keysReceivedUntilDrop3 = keysReceived; keysReceived = new ArrayList<>(); - readNextNStreams(trafficSource, keysReceived, 0, 3); + readNextNStreams(rootContext, trafficSource, keysReceived, 0, 3); log.atInfo().setMessage(()->"6 ..."+kafkaSource.trackingKafkaConsumer.nextCommitsToString()).log(); trafficSource.close(); } @@ -153,11 +156,11 @@ private String renderNextCommitsAsString() { } @SneakyThrows - private static void readNextNStreams(BlockingTrafficSource kafkaSource, List keysReceived, - int from, int count) { + private static void readNextNStreams(TestContext rootContext, BlockingTrafficSource kafkaSource, + List keysReceived, int from, int count) { Assertions.assertEquals(from, keysReceived.size()); for (int i=0; i{ var tsk = ts.getKey(); log.atInfo().setMessage(()->"checking for "+tsk).log(); diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTestUtils.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTestUtils.java index a5eed831a..e7e2136f2 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTestUtils.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTestUtils.java @@ -10,7 +10,9 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.jetbrains.annotations.NotNull; import org.junit.jupiter.api.Tag; +import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamKeyAndContext; import org.opensearch.migrations.replay.traffic.source.ITrafficStreamWithKey; +import org.opensearch.migrations.replay.util.TrafficChannelKeyFormatter; import org.opensearch.migrations.trafficcapture.protos.ReadObservation; import org.opensearch.migrations.trafficcapture.protos.TrafficObservation; import org.opensearch.migrations.trafficcapture.protos.TrafficStream; @@ -78,16 +80,18 @@ static TrafficStream makeTestTrafficStreamWithFixedTime(Instant t, int i) { @SneakyThrows public static void writeTrafficStreamRecord(Producer kafkaProducer, - ITrafficStreamWithKey trafficStreamAndKey, + TrafficStream trafficStream, String TEST_TOPIC_NAME, String recordId) { while (true) { try { - var record = new ProducerRecord(TEST_TOPIC_NAME, recordId, trafficStreamAndKey.getStream().toByteArray()); - log.info("sending record with trafficStream=" + trafficStreamAndKey.getKey()); + var record = new ProducerRecord(TEST_TOPIC_NAME, recordId, trafficStream.toByteArray()); + var tsKeyStr = TrafficChannelKeyFormatter.format(trafficStream.getNodeId(), + trafficStream.getConnectionId()); + log.info("sending record with trafficStream=" + tsKeyStr); var sendFuture = kafkaProducer.send(record, (metadata, exception) -> { log.atInfo().setCause(exception).setMessage(() -> "completed send of TrafficStream with key=" + - trafficStreamAndKey.getKey() + " metadata=" + metadata).log(); + tsKeyStr + " metadata=" + metadata).log(); }); var recordMetadata = sendFuture.get(); log.info("finished publishing record... metadata=" + recordMetadata); diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSourceLongTermTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSourceLongTermTest.java index 40e15c712..81ad5d8ef 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSourceLongTermTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSourceLongTermTest.java @@ -5,6 +5,7 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; +import org.opensearch.migrations.tracing.InstrumentationTest; import org.testcontainers.containers.KafkaContainer; import org.testcontainers.junit.jupiter.Container; import org.testcontainers.junit.jupiter.Testcontainers; @@ -19,7 +20,7 @@ @Slf4j @Testcontainers(disabledWithoutDocker = true) @Tag("requiresDocker") -public class KafkaTrafficCaptureSourceLongTermTest { +public class KafkaTrafficCaptureSourceLongTermTest extends InstrumentationTest { public static final int TEST_RECORD_COUNT = 10; public static final String TEST_GROUP_CONSUMER_ID = "TEST_GROUP_CONSUMER_ID"; @@ -37,18 +38,18 @@ public void testTrafficCaptureSource() throws Exception { String testTopicName = "TEST_TOPIC"; var kafkaConsumerProps = KafkaTrafficCaptureSource.buildKafkaProperties(embeddedKafkaBroker.getBootstrapServers(), - TEST_GROUP_CONSUMER_ID, false, null); + TEST_GROUP_CONSUMER_ID, false, null); final long MAX_POLL_MS = 10000; - kafkaConsumerProps.setProperty(KafkaTrafficCaptureSource.MAX_POLL_INTERVAL_KEY, MAX_POLL_MS+""); - var kafkaConsumer = new KafkaConsumer(kafkaConsumerProps); - var kafkaTrafficCaptureSource = new KafkaTrafficCaptureSource(kafkaConsumer, testTopicName, - Duration.ofMillis(MAX_POLL_MS)); + kafkaConsumerProps.setProperty(KafkaTrafficCaptureSource.MAX_POLL_INTERVAL_KEY, MAX_POLL_MS + ""); + var kafkaConsumer = new KafkaConsumer(kafkaConsumerProps); + var kafkaTrafficCaptureSource = new KafkaTrafficCaptureSource(rootContext, + kafkaConsumer, testTopicName, Duration.ofMillis(MAX_POLL_MS)); var kafkaProducer = KafkaTestUtils.buildKafkaProducer(embeddedKafkaBroker.getBootstrapServers()); var sendCompleteCount = new AtomicInteger(0); var scheduledIterationsCount = new AtomicInteger(0); var executor = Executors.newSingleThreadScheduledExecutor(); - executor.scheduleAtFixedRate(()->{ + executor.scheduleAtFixedRate(() -> { var i = scheduledIterationsCount.getAndIncrement(); if (i >= TEST_RECORD_COUNT) { executor.shutdown(); @@ -57,26 +58,28 @@ public void testTrafficCaptureSource() throws Exception { } }, 0, PRODUCER_SLEEP_INTERVAL_MS, TimeUnit.MILLISECONDS); - for (int i=0; i { - var rogueChunk = kafkaTrafficCaptureSource.readNextTrafficStreamChunk().get(1, TimeUnit.SECONDS); - if (rogueChunk.isEmpty()) { - // TimeoutExceptions cannot be thrown by the supplier of the CompletableFuture today, BUT we - // could long-poll on the broker for longer than the timeout value supplied in the get() call above - throw new TimeoutException("read actually returned 0 items, but transforming this to a " + - "TimeoutException because either result would be valid."); - } - log.error("rogue chunk: "+ rogueChunk); + Assertions.assertThrows(TimeoutException.class, () -> { + var rogueChunk = kafkaTrafficCaptureSource.readNextTrafficStreamChunk(rootContext::createReadChunkContext) + .get(1, TimeUnit.SECONDS); + if (rogueChunk.isEmpty()) { + // TimeoutExceptions cannot be thrown by the supplier of the CompletableFuture today, BUT we + // could long-poll on the broker for longer than the timeout value supplied in the get() call above + throw new TimeoutException("read actually returned 0 items, but transforming this to a " + + "TimeoutException because either result would be valid."); + } + log.error("rogue chunk: " + rogueChunk); }); } diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSourceTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSourceTest.java index 6cec1a13b..e2b13fdca 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSourceTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSourceTest.java @@ -10,8 +10,10 @@ import org.apache.kafka.common.TopicPartition; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamKey; +import org.opensearch.migrations.replay.tracing.ChannelContextManager; +import org.opensearch.migrations.replay.tracing.ReplayContexts; import org.opensearch.migrations.replay.traffic.source.ITrafficStreamWithKey; +import org.opensearch.migrations.tracing.InstrumentationTest; import org.opensearch.migrations.trafficcapture.protos.ReadObservation; import org.opensearch.migrations.trafficcapture.protos.TrafficObservation; import org.opensearch.migrations.trafficcapture.protos.TrafficStream; @@ -32,7 +34,7 @@ import java.util.function.Supplier; @Slf4j -class KafkaTrafficCaptureSourceTest { +class KafkaTrafficCaptureSourceTest extends InstrumentationTest { public static final int NUM_READ_ITEMS_BOUND = 1000; public static final String TEST_TOPIC_NAME = "TEST_TOPIC_NAME"; @@ -43,7 +45,10 @@ public void testRecordToString() { .setNodeId("n") .setNumber(7) .build(); - var tsk = new TrafficStreamKeyWithKafkaRecordId(ts, 1, 2, 123); + var tsk = new TrafficStreamKeyWithKafkaRecordId( + k -> new ReplayContexts.KafkaRecordContext(rootContext, + new ChannelContextManager(rootContext).retainOrCreateContext(k), "", 1), + ts, 1, 2, 123); Assertions.assertEquals("n.c.7|partition=2|offset=123", tsk.toString()); } @@ -51,8 +56,8 @@ public void testRecordToString() { public void testSupplyTrafficFromSource() { int numTrafficStreams = 10; MockConsumer mockConsumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); - KafkaTrafficCaptureSource protobufConsumer = new KafkaTrafficCaptureSource(mockConsumer, TEST_TOPIC_NAME, - Duration.ofHours(1)); + KafkaTrafficCaptureSource protobufConsumer = new KafkaTrafficCaptureSource(rootContext, + mockConsumer, TEST_TOPIC_NAME, Duration.ofHours(1)); initializeMockConsumerTopic(mockConsumer); List substreamCounts = new ArrayList<>(); @@ -71,13 +76,14 @@ public void testSupplyTrafficFromSource() { var tsCount = new AtomicInteger(); Assertions.assertTimeoutPreemptively(Duration.ofSeconds(1), () -> { while (tsCount.get() < numTrafficStreams) { - protobufConsumer.readNextTrafficStreamChunk().get().stream().forEach(streamWithKey->{ - tsCount.incrementAndGet(); - log.trace("Stream has substream count: " + streamWithKey.getStream().getSubStreamCount()); - Assertions.assertInstanceOf(ITrafficStreamWithKey.class, streamWithKey); - Assertions.assertEquals(streamWithKey.getStream().getSubStreamCount(), - substreamCounts.get(foundStreamsCount.getAndIncrement())); - }); + protobufConsumer.readNextTrafficStreamChunk(rootContext::createReadChunkContext).get().stream() + .forEach(streamWithKey -> { + tsCount.incrementAndGet(); + log.trace("Stream has substream count: " + streamWithKey.getStream().getSubStreamCount()); + Assertions.assertInstanceOf(ITrafficStreamWithKey.class, streamWithKey); + Assertions.assertEquals(streamWithKey.getStream().getSubStreamCount(), + substreamCounts.get(foundStreamsCount.getAndIncrement())); + }); } }); Assertions.assertEquals(foundStreamsCount.get(), numTrafficStreams); @@ -92,8 +98,8 @@ public void testSupplyTrafficFromSource() { public void testSupplyTrafficWithUnformattedMessages() { int numTrafficStreams = 10; MockConsumer mockConsumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); - KafkaTrafficCaptureSource protobufConsumer = new KafkaTrafficCaptureSource(mockConsumer, TEST_TOPIC_NAME, - Duration.ofHours(1)); + KafkaTrafficCaptureSource protobufConsumer = new KafkaTrafficCaptureSource(rootContext, + mockConsumer, TEST_TOPIC_NAME, Duration.ofHours(1)); initializeMockConsumerTopic(mockConsumer); List substreamCounts = new ArrayList<>(); @@ -118,17 +124,17 @@ public void testSupplyTrafficWithUnformattedMessages() { // This assertion will fail the test case if not completed within its duration, as would be the case if there // were missing traffic streams. Its task currently is limited to the numTrafficStreams where it will stop the stream - var tsCount = new AtomicInteger(); Assertions.assertTimeoutPreemptively(Duration.ofSeconds(1), () -> { while (tsCount.get() < numTrafficStreams) { - protobufConsumer.readNextTrafficStreamChunk().get().stream().forEach(streamWithKey->{ - tsCount.incrementAndGet(); - log.trace("Stream has substream count: " + streamWithKey.getStream().getSubStreamCount()); - Assertions.assertInstanceOf(ITrafficStreamWithKey.class, streamWithKey); - Assertions.assertEquals(streamWithKey.getStream().getSubStreamCount(), - substreamCounts.get(foundStreamsCount.getAndIncrement())); - }); + protobufConsumer.readNextTrafficStreamChunk(rootContext::createReadChunkContext).get().stream() + .forEach(streamWithKey->{ + tsCount.incrementAndGet(); + log.trace("Stream has substream count: " + streamWithKey.getStream().getSubStreamCount()); + Assertions.assertInstanceOf(ITrafficStreamWithKey.class, streamWithKey); + Assertions.assertEquals(streamWithKey.getStream().getSubStreamCount(), + substreamCounts.get(foundStreamsCount.getAndIncrement())); + }); } }); diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/tracing/TracingTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/tracing/TracingTest.java new file mode 100644 index 000000000..d6cbd2fc2 --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/tracing/TracingTest.java @@ -0,0 +1,92 @@ +package org.opensearch.migrations.replay.tracing; + +import io.opentelemetry.sdk.metrics.data.MetricData; +import io.opentelemetry.sdk.trace.data.SpanData; +import lombok.Lombok; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.opensearch.migrations.replay.datatypes.ISourceTrafficChannelKey; +import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamKeyAndContext; +import org.opensearch.migrations.replay.datatypes.UniqueReplayerRequestKey; +import org.opensearch.migrations.tracing.InstrumentationTest; +import org.opensearch.migrations.tracing.TestContext; + +import java.time.Duration; +import java.time.Instant; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class TracingTest extends InstrumentationTest { + + @Override + protected TestContext makeInstrumentationContext() { + return TestContext.withAllTracking(); + } + + @Test + public void tracingWorks() { + var tssk = new ISourceTrafficChannelKey.PojoImpl("n", "c"); + try (var channelCtx = rootContext.createChannelContext(tssk); + var kafkaRecordCtx = + rootContext.createTrafficStreamContextForKafkaSource(channelCtx, "testRecordId", 127)) { + var tsk = PojoTrafficStreamKeyAndContext.build(tssk, 1, kafkaRecordCtx::createTrafficLifecyleContext); + try (var tskCtx = tsk.getTrafficStreamsContext()) { // made in the callback of the previous call + var urk = new UniqueReplayerRequestKey(tsk, 1, 0); + try (var httpCtx = tskCtx.createHttpTransactionContext(urk, Instant.EPOCH)) { + try (var ctx = httpCtx.createRequestAccumulationContext()) { + } + try (var ctx = httpCtx.createResponseAccumulationContext()) { + } + try (var ctx = httpCtx.createTransformationContext()) { + } + try (var ctx = httpCtx.createScheduledContext(Instant.now())) { + } + try (var targetRequestCtx = httpCtx.createTargetRequestContext()) { + try (var ctx = targetRequestCtx.createHttpSendingContext()) { + } + try (var ctx = targetRequestCtx.createWaitingForResponseContext()) { + } + try (var ctx = targetRequestCtx.createHttpReceivingContext()) { + } + } + try (var ctx = httpCtx.createTupleContext()) { + } + } + } + try (var ctx = channelCtx.createSocketContext()) { + } + } + + var recordedSpans = rootContext.inMemoryInstrumentationBundle.testSpanExporter.getFinishedSpanItems(); + var recordedMetrics = rootContext.inMemoryInstrumentationBundle.testMetricExporter.getFinishedMetricItems(); + + checkSpans(recordedSpans); + checkMetrics(recordedMetrics); + + Assertions.assertTrue(rootContext.contextTracker.getAllRemainingActiveScopes().isEmpty()); + } + + private void checkMetrics(List recordedMetrics) { + } + + private void checkSpans(List recordedSpans) { + var byName = recordedSpans.stream().collect(Collectors.groupingBy(SpanData::getName)); + var keys = Arrays.stream(IReplayContexts.ActivityNames.class.getFields()).map(f -> { + try { + return f.get(null); + } catch (Exception e) { + throw Lombok.sneakyThrow(e); + } + }).toArray(String[]::new); + Stream.of(keys).forEach(spanName -> { + Assertions.assertNotNull(byName.get(spanName)); + Assertions.assertEquals(1, byName.get(spanName).size()); + byName.remove(spanName); + }); + + Assertions.assertEquals("", byName.entrySet().stream() + .map(kvp -> kvp.getKey() + ":" + kvp.getValue()).collect(Collectors.joining())); + } +} 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 deleted file mode 100644 index 5751cb4aa..000000000 --- a/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/TestRequestKey.java +++ /dev/null @@ -1,15 +0,0 @@ -package org.opensearch.migrations.replay; - -import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamKey; -import org.opensearch.migrations.replay.datatypes.UniqueReplayerRequestKey; - -public class TestRequestKey { - - private TestRequestKey() {} - - public static final UniqueReplayerRequestKey getTestConnectionRequestId(int replayerIdx) { - return new UniqueReplayerRequestKey( - new PojoTrafficStreamKey("testNodeId", "testConnectionId", 0), - 0, replayerIdx); - } -} 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..d883e7d94 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 @@ -11,9 +11,11 @@ import io.netty.handler.codec.http.HttpRequestDecoder; import lombok.extern.slf4j.Slf4j; import org.junit.jupiter.api.Assertions; +import org.opensearch.migrations.Utils; import org.opensearch.migrations.replay.datahandlers.IPacketConsumer; import org.opensearch.migrations.replay.datahandlers.http.HttpJsonTransformingConsumer; import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; +import org.opensearch.migrations.tracing.TestContext; import org.opensearch.migrations.transform.IAuthTransformerFactory; import org.opensearch.migrations.transform.IJsonTransformer; @@ -79,7 +81,7 @@ static DiagnosticTrackableCompletableFuture chainedWriteHeadersAndD List stringParts, StringBuilder referenceStringAccumulator, IntFunction headersGenerator) { - var contentLength = stringParts.stream().mapToInt(s->s.length()).sum(); + var contentLength = stringParts.stream().mapToInt(String::length).sum(); String headers = headersGenerator.apply(contentLength) + "\r\n"; referenceStringAccumulator.append(headers); return chainedWriteHeadersAndDualWritePayloadParts(packetConsumer, stringParts, referenceStringAccumulator, headers); @@ -117,20 +119,22 @@ private static String getStringFromContent(FullHttpRequest fullRequest) throws I try (var baos = new ByteArrayOutputStream()) { var bb = fullRequest.content(); bb.readBytes(baos, bb.readableBytes()); - return new String(baos.toByteArray(), StandardCharsets.UTF_8); + return baos.toString(StandardCharsets.UTF_8); } } - static void runPipelineAndValidate(IAuthTransformerFactory authTransformer, + static void runPipelineAndValidate(TestContext rootContext, + IAuthTransformerFactory authTransformer, String extraHeaders, List stringParts, DefaultHttpHeaders expectedRequestHeaders, Function expectedOutputGenerator) throws IOException, ExecutionException, InterruptedException { - runPipelineAndValidate(x -> x, + runPipelineAndValidate(rootContext, x -> x, authTransformer, extraHeaders, stringParts, expectedRequestHeaders, expectedOutputGenerator); } - static void runPipelineAndValidate(IJsonTransformer transformer, + static void runPipelineAndValidate(TestContext rootContext, + IJsonTransformer transformer, IAuthTransformerFactory authTransformer, String extraHeaders, List stringParts, @@ -141,7 +145,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)); + rootContext.getTestConnectionRequestContext("TEST_CONNECTION", 0)); var contentLength = stringParts.stream().mapToInt(String::length).sum(); var headerString = "GET / HTTP/1.1\r\n" + diff --git a/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/tracing/InstrumentationTest.java b/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/tracing/InstrumentationTest.java new file mode 100644 index 000000000..826ada529 --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/tracing/InstrumentationTest.java @@ -0,0 +1,24 @@ +package org.opensearch.migrations.tracing; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; + +public class InstrumentationTest { + + protected TestContext rootContext; + + protected TestContext makeInstrumentationContext() { + return TestContext.noOtelTracking(); + } + + @BeforeEach + protected void initializeContext() { + rootContext = makeInstrumentationContext(); + } + + @AfterEach + protected void teardownContext() { + rootContext.close(); + rootContext = null; + } +} diff --git a/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/tracing/TestContext.java b/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/tracing/TestContext.java new file mode 100644 index 000000000..1c5927e46 --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/tracing/TestContext.java @@ -0,0 +1,88 @@ +package org.opensearch.migrations.tracing; + +import io.opentelemetry.sdk.testing.exporter.InMemoryMetricExporter; +import io.opentelemetry.sdk.testing.exporter.InMemorySpanExporter; +import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; +import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamKeyAndContext; +import org.opensearch.migrations.replay.datatypes.UniqueReplayerRequestKey; +import org.opensearch.migrations.replay.tracing.ChannelContextManager; +import org.opensearch.migrations.replay.tracing.IReplayContexts; +import org.opensearch.migrations.replay.tracing.RootReplayerContext; + +import java.time.Instant; + +public class TestContext extends RootReplayerContext implements AutoCloseable { + + public static final String TEST_NODE_ID = "testNodeId"; + public static final String DEFAULT_TEST_CONNECTION = "testConnection"; + public final InMemoryInstrumentationBundle inMemoryInstrumentationBundle; + public final ContextTracker contextTracker = new ContextTracker(); + public final ChannelContextManager channelContextManager = new ChannelContextManager(this); + private final Object channelContextManagerLock = new Object(); + + public static TestContext withTracking(boolean tracing, boolean metrics) { + return new TestContext(new InMemoryInstrumentationBundle(tracing, metrics)); + } + + public static TestContext withAllTracking() { + return withTracking(true, true); + } + + public static TestContext noOtelTracking() { + return new TestContext(new InMemoryInstrumentationBundle(null, null)); + } + + public TestContext(InMemoryInstrumentationBundle inMemoryInstrumentationBundle) { + super(inMemoryInstrumentationBundle.openTelemetrySdk); + this.inMemoryInstrumentationBundle = inMemoryInstrumentationBundle; + } + + @Override + public void onContextCreated(IScopedInstrumentationAttributes newScopedContext) { + contextTracker.onCreated(newScopedContext); + } + + @Override + public void onContextClosed(IScopedInstrumentationAttributes newScopedContext) { + contextTracker.onClosed(newScopedContext); + } + + public IReplayContexts.ITrafficStreamsLifecycleContext createTrafficStreamContextForTest(ITrafficStreamKey tsk) { + synchronized (channelContextManagerLock) { + return createTrafficStreamContextForStreamSource(channelContextManager.retainOrCreateContext(tsk), tsk); + } + } + + @Override + public void close() { + contextTracker.close(); + inMemoryInstrumentationBundle.close(); +// Assertions.assertEquals("", contextTracker.getAllRemainingActiveScopes().entrySet().stream() +// .map(kvp->kvp.getKey().toString()).collect(Collectors.joining())); + } + + + public final IReplayContexts.IReplayerHttpTransactionContext + getTestConnectionRequestContext(int replayerIdx) { + return getTestConnectionRequestContext(DEFAULT_TEST_CONNECTION, replayerIdx); + } + + public IReplayContexts.IReplayerHttpTransactionContext + getTestConnectionRequestContext(String connectionId, int replayerIdx) { + var rk = new UniqueReplayerRequestKey( + PojoTrafficStreamKeyAndContext.build(TEST_NODE_ID, connectionId, 0, + this::createTrafficStreamContextForTest), + 0, replayerIdx); + return rk.trafficStreamKey.getTrafficStreamsContext().createHttpTransactionContext(rk, Instant.EPOCH); + } + + public IReplayContexts.ITupleHandlingContext + getTestTupleContext() { + return getTestTupleContext(DEFAULT_TEST_CONNECTION, 1); + } + + public IReplayContexts.ITupleHandlingContext + getTestTupleContext(String connectionId, int replayerIdx) { + return getTestConnectionRequestContext(connectionId, replayerIdx).createTupleContext(); + } +} diff --git a/deployment/cdk/opensearch-service-migration/lib/common-utilities.ts b/deployment/cdk/opensearch-service-migration/lib/common-utilities.ts index 80ad70a64..a28fa2b61 100644 --- a/deployment/cdk/opensearch-service-migration/lib/common-utilities.ts +++ b/deployment/cdk/opensearch-service-migration/lib/common-utilities.ts @@ -77,6 +77,28 @@ export function createMSKProducerIAMPolicies(scope: Construct, region: string, a return [mskClusterConnectPolicy, mskTopicProducerPolicy] } +export function createAwsDistroForOtelPushInstrumentationPolicy(): PolicyStatement { + // see https://aws-otel.github.io/docs/setup/permissions + return new PolicyStatement( { + effect: Effect.ALLOW, + resources: ["*"], + actions: [ + "logs:PutLogEvents", + "logs:CreateLogGroup", + "logs:CreateLogStream", + "logs:DescribeLogStreams", + "logs:DescribeLogGroups", + "logs:PutRetentionPolicy", + "xray:PutTraceSegments", + "xray:PutTelemetryRecords", + "xray:GetSamplingRules", + "xray:GetSamplingTargets", + "xray:GetSamplingStatisticSummaries", + "ssm:GetParameters" + ] + }) +} + export function createDefaultECSTaskRole(scope: Construct, serviceName: string): Role { const serviceTaskRole = new Role(scope, `${serviceName}-TaskRole`, { assumedBy: new ServicePrincipal('ecs-tasks.amazonaws.com'), diff --git a/deployment/cdk/opensearch-service-migration/lib/service-stacks/migration-analytics-stack.ts b/deployment/cdk/opensearch-service-migration/lib/service-stacks/migration-analytics-stack.ts index 2a8e6c8bc..bc6d56778 100644 --- a/deployment/cdk/opensearch-service-migration/lib/service-stacks/migration-analytics-stack.ts +++ b/deployment/cdk/opensearch-service-migration/lib/service-stacks/migration-analytics-stack.ts @@ -11,6 +11,7 @@ import {Construct} from "constructs"; import {join} from "path"; import {MigrationServiceCore} from "./migration-service-core"; import {StringParameter} from "aws-cdk-lib/aws-ssm"; +import {createAwsDistroForOtelPushInstrumentationPolicy} from "../common-utilities"; export interface MigrationAnalyticsProps extends StackPropsExt { readonly vpc: IVpc, @@ -72,10 +73,14 @@ export class MigrationAnalyticsStack extends MigrationServiceCore { let securityGroups = [ SecurityGroup.fromSecurityGroupId(this, "serviceConnectSG", StringParameter.valueForStringParameter(this, `/migration/${props.stage}/${props.defaultDeployId}/serviceConnectSecurityGroupId`)), migrationAnalyticsSecurityGroup - ] + ] + + const servicePolicies = [createAwsDistroForOtelPushInstrumentationPolicy()] + this.createService({ serviceName: `otel-collector`, - dockerFilePath: join(__dirname, "../../../../../", "TrafficCapture/dockerSolution/src/main/docker/otelcol"), + dockerFilePath: join(__dirname, "../../../../../", "TrafficCapture/dockerSolution/src/main/docker/otelCollector"), + dockerImageCommand: ["--config=/etc/otel-config-aws.yaml"], securityGroups: securityGroups, taskCpuUnits: 1024, taskMemoryLimitMiB: 4096, @@ -83,6 +88,7 @@ export class MigrationAnalyticsStack extends MigrationServiceCore { serviceConnectServices: [serviceConnectServiceCollector, serviceConnectServiceHealthCheck], serviceDiscoveryEnabled: true, serviceDiscoveryPort: 4317, + taskRolePolicies: servicePolicies, environment: { "ANALYTICS_DOMAIN_ENDPOINT": analyticsDomainEndpoint },