From 85735e8810c3828e1e45e8f951c849364155ef7e Mon Sep 17 00:00:00 2001 From: Greg Schohn Date: Sat, 7 Sep 2024 09:01:18 -0400 Subject: [PATCH 01/12] Transformation UX. Add a command line flag to take the configuration via a base64 encoded string + a test sample script. Add a test/sample script to excise specific elements, when present, from a json tree and also drop a component of the URI path. Signed-off-by: Greg Schohn --- .../migrations/replay/TrafficReplayer.java | 167 +++++++++++++----- .../replay/MultipleJoltScriptsTest.java | 87 ++++++++- 2 files changed, 210 insertions(+), 44 deletions(-) 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 af0c09fd7..d34a1ea07 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 @@ -9,6 +9,7 @@ import java.time.Clock; import java.time.Duration; import java.time.Instant; +import java.util.Base64; import java.util.List; import java.util.Optional; import java.util.concurrent.Executors; @@ -89,93 +90,166 @@ public static boolean validateRequiredKafkaParams(String brokers, String topic, } public static class Parameters { - @Parameter(required = true, arity = 1, description = "URI of the target cluster/domain") + @Parameter( + required = true, + arity = 1, + description = "URI of the target cluster/domain") String targetUriString; - @Parameter(required = false, names = { - "--insecure" }, arity = 0, description = "Do not check the server's certificate") + @Parameter( + required = false, + names = {"--insecure" }, + arity = 0, description = "Do not check the server's certificate") boolean allowInsecureConnections; - @Parameter(required = false, names = { - REMOVE_AUTH_HEADER_VALUE_ARG }, arity = 0, description = "Remove the authorization header if present and do not replace it with anything. " + @Parameter( + required = false, + names = {REMOVE_AUTH_HEADER_VALUE_ARG }, + arity = 0, description = "Remove the authorization header if present and do not replace it with anything. " + "(cannot be used with other auth arguments)") boolean removeAuthHeader; - @Parameter(required = false, names = { - AUTH_HEADER_VALUE_ARG }, arity = 1, description = "Static value to use for the \"authorization\" header of each request " + @Parameter( + required = false, + names = { AUTH_HEADER_VALUE_ARG }, + arity = 1, description = "Static value to use for the \"authorization\" header of each request " + "(cannot be used with other auth arguments)") String authHeaderValue; - @Parameter(required = false, names = { - AWS_AUTH_HEADER_USER_AND_SECRET_ARG }, arity = 2, description = " pair to specify " + @Parameter( + required = false, names = { + AWS_AUTH_HEADER_USER_AND_SECRET_ARG }, + arity = 2, + description = " pair to specify " + "\"authorization\" header value for each request. " + "The USERNAME specifies the plaintext user and the SECRET_ARN specifies the ARN or " + "Secret name from AWS Secrets Manager to retrieve the password from for the password section" + "(cannot be used with other auth arguments)") List awsAuthHeaderUserAndSecret; - @Parameter(required = false, names = { - SIGV_4_AUTH_HEADER_SERVICE_REGION_ARG }, arity = 1, description = "Use AWS SigV4 to sign each request with the specified service name and region. " + @Parameter( + required = false, + names = { SIGV_4_AUTH_HEADER_SERVICE_REGION_ARG }, + arity = 1, + description = "Use AWS SigV4 to sign each request with the specified service name and region. " + "(e.g. es,us-east-1) " + "DefaultCredentialsProvider is used to resolve credentials. " + "(cannot be used with other auth arguments)") String useSigV4ServiceAndRegion; - @Parameter(required = false, names = "--transformer-config", arity = 1, description = "Configuration of message transformers. Either as a string that identifies the " + @Parameter( + required = false, + names = "--transformer-config-base64", + arity = 1, + description = "Configuration of message transformers. The same contents as --transformer-config but " + + "Base64 encoded so that the configuration is easier to pass as a command line parameter.") + String transformerConfigEncoded; + + @Parameter( + required = false, + names = "--transformer-config", + arity = 1, + description = "Configuration of message transformers. Either as a string that identifies the " + "transformer that should be run (with default settings) or as json to specify options " + "as well as multiple transformers to run in sequence. " + "For json, keys are the (simple) names of the loaded transformers and values are the " + "configuration passed to each of the transformers.") String transformerConfig; - @Parameter(required = false, names = "--transformer-config-file", arity = 1, description = "Path to the JSON configuration file of message transformers.") + + @Parameter( + required = false, + names = "--transformer-config-file", + arity = 1, + description = "Path to the JSON configuration file of message transformers.") String transformerConfigFile; - @Parameter(required = false, names = "--user-agent", arity = 1, description = "For HTTP requests to the target cluster, append this string (after \"; \") to" + @Parameter( + required = false, + names = "--user-agent", + arity = 1, + description = "For HTTP requests to the target cluster, append this string (after \"; \") to" + "the existing user-agent field or if the field wasn't present, simply use this value") String userAgent; - @Parameter(required = false, names = { - "-i", - "--input" }, arity = 1, description = "input file to read the request/response traces for the source cluster") + @Parameter( + required = false, + names = { "-i", "--input" }, + arity = 1, + description = "input file to read the request/response traces for the source cluster") String inputFilename; - @Parameter(required = false, names = { - "-t", - PACKET_TIMEOUT_SECONDS_PARAMETER_NAME }, arity = 1, description = "assume that connections were terminated after this many " + @Parameter( + required = false, + names = {"-t", PACKET_TIMEOUT_SECONDS_PARAMETER_NAME }, + arity = 1, + description = "assume that connections were terminated after this many " + "seconds of inactivity observed in the captured stream") int observedPacketConnectionTimeout = 70; - @Parameter(required = false, names = { - "--speedup-factor" }, arity = 1, description = "Accelerate the replayed communications by this factor. " + @Parameter( + required = false, + names = { "--speedup-factor" }, + arity = 1, description = "Accelerate the replayed communications by this factor. " + "This means that between each interaction will be replayed at this rate faster " + "than the original observations, provided that the replayer and target are able to keep up.") double speedupFactor = 1.0; - @Parameter(required = false, names = { - LOOKAHEAD_TIME_WINDOW_PARAMETER_NAME }, arity = 1, description = "Number of seconds of data that will be buffered.") + @Parameter( + required = false, + names = { LOOKAHEAD_TIME_WINDOW_PARAMETER_NAME }, + arity = 1, + description = "Number of seconds of data that will be buffered.") int lookaheadTimeSeconds = 300; - @Parameter(required = false, names = { - "--max-concurrent-requests" }, arity = 1, description = "Maximum number of requests at a time that can be outstanding") + @Parameter( + required = false, + names = { "--max-concurrent-requests" }, + arity = 1, + description = "Maximum number of requests at a time that can be outstanding") int maxConcurrentRequests = 1024; - @Parameter(required = false, names = { - "--num-client-threads" }, arity = 1, description = "Number of threads to use to send requests from.") + @Parameter( + required = false, + names = { "--num-client-threads" }, + arity = 1, + description = "Number of threads to use to send requests from.") int numClientThreads = 0; // https://github.com/opensearch-project/opensearch-java/blob/main/java-client/src/main/java/org/opensearch/client/transport/httpclient5/ApacheHttpClient5TransportBuilder.java#L49-L54 - @Parameter(required = false, names = { - "--target-response-timeout" }, arity = 1, description = "Seconds to wait before timing out a replayed request to the target.") + @Parameter( + required = false, + names = { "--target-response-timeout" }, + arity = 1, + description = "Seconds to wait before timing out a replayed request to the target.") int targetServerResponseTimeoutSeconds = 30; - @Parameter(required = false, names = { - "--kafka-traffic-brokers" }, arity = 1, description = "Comma-separated list of host and port pairs that are the addresses of the Kafka brokers to bootstrap with i.e. 'kafka-1:9092,kafka-2:9092'") + @Parameter( + required = false, + names = { "--kafka-traffic-brokers" }, + arity = 1, + description = "Comma-separated list of host and port pairs that are the addresses of the Kafka brokers " + + "to bootstrap with i.e. 'kafka-1:9092,kafka-2:9092'") String kafkaTrafficBrokers; - @Parameter(required = false, names = { - "--kafka-traffic-topic" }, arity = 1, description = "Topic name used to pull messages from Kafka") + @Parameter( + required = false, + names = { "--kafka-traffic-topic" }, + arity = 1, + description = "Topic name used to pull messages from Kafka") String kafkaTrafficTopic; - @Parameter(required = false, names = { - "--kafka-traffic-group-id" }, arity = 1, description = "Consumer group id that is used when pulling messages from Kafka") + @Parameter( + required = false, + names = { "--kafka-traffic-group-id" }, + arity = 1, + description = "Consumer group id that is used when pulling messages from Kafka") String kafkaTrafficGroupId; - @Parameter(required = false, names = { - "--kafka-traffic-enable-msk-auth" }, arity = 0, description = "Enables SASL properties required for connecting to MSK with IAM auth") + @Parameter( + required = false, + names = { "--kafka-traffic-enable-msk-auth" }, + arity = 0, + description = "Enables SASL properties required for connecting to MSK with IAM auth") boolean kafkaTrafficEnableMSKAuth; - @Parameter(required = false, names = { - "--kafka-traffic-property-file" }, arity = 1, description = "File path for Kafka properties file to use for additional or overriden Kafka properties") + @Parameter( + required = false, + names = { "--kafka-traffic-property-file" }, + arity = 1, + description = "File path for Kafka properties file to use for additional or overriden Kafka properties") String kafkaTrafficPropertyFile; - @Parameter(required = false, names = { - "--otelCollectorEndpoint" }, arity = 1, description = "Endpoint (host:port) for the OpenTelemetry Collector to which metrics logs should be" + @Parameter( + required = false, + names = { "--otelCollectorEndpoint" }, + arity = 1, + description = "Endpoint (host:port) for the OpenTelemetry Collector to which metrics logs should be" + "forwarded. If no value is provided, metrics will not be forwarded.") String otelCollectorEndpoint; } @@ -198,9 +272,12 @@ private static Parameters parseArgs(String[] args) { private static String getTransformerConfig(Parameters params) { if (params.transformerConfigFile != null && !params.transformerConfigFile.isBlank() + && params.transformerConfigEncoded != null + && !params.transformerConfigEncoded.isBlank() && params.transformerConfig != null && !params.transformerConfig.isBlank()) { - System.err.println("Specify either --transformer-config or --transformer-config-file, not both."); + System.err.println("Specify only one of --transformer-config-base64, --transformer-config or " + + "--transformer-config-file."); System.exit(4); } @@ -217,6 +294,10 @@ private static String getTransformerConfig(Parameters params) { return params.transformerConfig; } + if (params.transformerConfigEncoded != null && !params.transformerConfigEncoded.isBlank()) { + return new String(Base64.getDecoder().decode(params.transformerConfigEncoded)); + } + return null; } diff --git a/TrafficCapture/transformationPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/java/org/opensearch/migrations/replay/MultipleJoltScriptsTest.java b/TrafficCapture/transformationPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/java/org/opensearch/migrations/replay/MultipleJoltScriptsTest.java index 6cf42927d..2f0971ee9 100644 --- a/TrafficCapture/transformationPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/java/org/opensearch/migrations/replay/MultipleJoltScriptsTest.java +++ b/TrafficCapture/transformationPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/java/org/opensearch/migrations/replay/MultipleJoltScriptsTest.java @@ -56,4 +56,89 @@ public void testAddGzipAndCustom() throws Exception { Assertions.assertEquals("newValue", headers.get("newHeader")); } -} + @Test + public void testExciseWhenPresent() throws Exception { + var script = "[{ \"JsonJoltTransformerProvider\":\n" + + "[\n" + + " {\n" + + " \"script\": {\n" + + " \"operation\": \"shift\",\n" + + " \"spec\": {\n" + + " \"method\": \"method\",\n" + + " \"URI\": \"URI\",\n" + + " \"headers\": \"headers\",\n" + + " \"payload\": {\n" + + " \"inlinedJsonBody\": {\n" + + " \"top\": {\n" + + " \"tagToExcise\": {\n" + + " \"*\": \"payload.inlinedJsonBody.top.&\" \n" + + " },\n" + + " \"*\": \"payload.inlinedJsonBody.top.&\"\n" + + " },\n" + + " \"*\": \"payload.inlinedJsonBody.&\"\n" + + " }\n" + + " }\n" + + " }\n" + + " }\n" + + " }, \n" + + " {\n" + + " \"script\": {\n" + + " \"operation\": \"modify-overwrite-beta\",\n" + + " \"spec\": {\n" + + " \"URI\": \"=split('/extraThingToRemove',@(1,&))\"\n" + + " }\n" + + " }\n" + + " },\n" + + " {\n" + + " \"script\": {\n" + + " \"operation\": \"modify-overwrite-beta\",\n" + + " \"spec\": {\n" + + " \"URI\": \"=join('',@(1,&))\"\n" + + " }\n" + + " }\n" + + " }\n" + + "]\n" + + "}]"; + + + var excisingTransformer = new TransformationLoader().getTransformerFactoryLoader( + "testhostname", + null, + script + ); + var origDocStr = "{\n" + + " \"method\": \"PUT\",\n" + + " \"URI\": \"/oldStyleIndex/extraThingToRemove/moreStuff\",\n" + + " \"headers\": {\n" + + " \"host\": \"127.0.0.1\"\n" + + " },\n" + + " \"payload\": {\n" + + " \"inlinedJsonBody\": {\n" + + " \"top\": {\n" + + " \"tagToExcise\": {\n" + + " \"properties\": {\n" + + " \"field1\": {\n" + + " \"type\": \"text\"\n" + + " },\n" + + " \"field2\": {\n" + + " \"type\": \"keyword\"\n" + + " }\n" + + " }\n" + + " }\n" + + " }\n" + + " }\n" + + " }\n" + + "}"; + var expectedDocStr = "{\"method\":\"PUT\",\"URI\":\"/oldStyleIndex/moreStuff\",\"headers\":{\"host\":\"testhostname\"},\"payload\":{\"inlinedJsonBody\":{\"top\":{\"properties\":{\"field1\":{\"type\":\"text\"},\"field2\":{\"type\":\"keyword\"}}}}}}"; + var origDoc = parseAsMap(origDocStr); + var newDoc = excisingTransformer.transformJson(origDoc); + var newAsStr = mapper.writeValueAsString(newDoc); + Assertions.assertEquals(expectedDocStr, newAsStr); + + var secondPassDoc = excisingTransformer.transformJson(newDoc); + var secondPassDocAsStr = mapper.writeValueAsString(secondPassDoc); + Assertions.assertEquals(expectedDocStr, secondPassDocAsStr); + + Assertions.assertEquals("testhostname", ((Map) newDoc.get(JsonKeysForHttpMessage.HEADERS_KEY)).get("host")); + } +} \ No newline at end of file From 1b3794f112056334906f0aee9dbf65663ced8b43 Mon Sep 17 00:00:00 2001 From: Greg Schohn Date: Sat, 7 Sep 2024 16:03:13 -0400 Subject: [PATCH 02/12] Further improvements to transformation - most notably to support header transforms for docs w/out bodies or with malformed bodies. Signed-off-by: Greg Schohn --- .../replay/TrafficReplayerCore.java | 4 +- .../NettyPacketToHttpConsumer.java | 4 +- .../PayloadAccessFaultingMap.java | 84 ++++++++++++++----- .../http/HttpJsonTransformingConsumer.java | 5 +- .../http/NettyJsonBodyAccumulateHandler.java | 30 +++++-- .../http/NettyJsonBodyConvertHandler.java | 8 +- .../http/NettyJsonBodySerializeHandler.java | 17 ++-- .../HttpJsonTransformingConsumerTest.java | 18 ++-- .../transform/JsonKeysForHttpMessage.java | 5 ++ 9 files changed, 127 insertions(+), 48 deletions(-) diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/TrafficReplayerCore.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/TrafficReplayerCore.java index e53c03637..90e80b2ae 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/TrafficReplayerCore.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/TrafficReplayerCore.java @@ -117,7 +117,9 @@ public Consumer onRequestReceived( ); finishedAccumulatingResponseFuture.future.whenComplete( (v, t) -> log.atDebug() - .setMessage(() -> "Done receiving captured stream for " + ctx + ":" + v.requestData) + .setMessage(() -> "Done receiving captured stream for {}:{}") + .addArgument(ctx) + .addArgument(v.requestData) .log() ); 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 329cadc8b..79593d458 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 @@ -282,7 +282,9 @@ private void deactivateChannel() { try { var pipeline = channel.pipeline(); log.atDebug() - .setMessage(() -> "Resetting the pipeline for channel " + channel + "currently at: " + pipeline) + .setMessage(() -> "Resetting the pipeline for channel {} currently at: {}") + .addArgument(channel) + .addArgument(pipeline) .log(); for (var handlerName : new String[] { WRITE_COUNT_WATCHER_HANDLER_NAME, READ_COUNT_WATCHER_HANDLER_NAME }) { try { diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/PayloadAccessFaultingMap.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/PayloadAccessFaultingMap.java index ee0dd0e3c..64cb7f64d 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/PayloadAccessFaultingMap.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/PayloadAccessFaultingMap.java @@ -10,7 +10,11 @@ import org.opensearch.migrations.replay.datahandlers.http.StrictCaseInsensitiveHttpHeadersMap; import org.opensearch.migrations.transform.JsonKeysForHttpMessage; +import io.netty.buffer.ByteBuf; import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.NonNull; +import lombok.Setter; import lombok.extern.slf4j.Slf4j; /** @@ -25,7 +29,11 @@ public class PayloadAccessFaultingMap extends AbstractMap { private final boolean isJson; - private Object onlyValue; + private Object jsonValue; + private ByteBuf binaryValue; + @Getter + @Setter + private boolean disableThrowingPayloadNotLoaded; public PayloadAccessFaultingMap(StrictCaseInsensitiveHttpHeadersMap headers) { isJson = Optional.ofNullable(headers.get("content-type")) @@ -34,24 +42,48 @@ public PayloadAccessFaultingMap(StrictCaseInsensitiveHttpHeadersMap headers) { } @Override - public Object get(Object key) { - if (!JsonKeysForHttpMessage.INLINED_JSON_BODY_DOCUMENT_KEY.equals(key) || !isJson) { + public boolean containsKey(Object key) { + return (JsonKeysForHttpMessage.INLINED_JSON_BODY_DOCUMENT_KEY.equals(key) && jsonValue != null) || + (JsonKeysForHttpMessage.INLINED_BINARY_BODY_DOCUMENT_KEY.equals(key) && binaryValue != null); + } + + private Object nullOrThrow() { + if (disableThrowingPayloadNotLoaded) { return null; } - if (onlyValue == null) { - throw PayloadNotLoadedException.getInstance(); + throw PayloadNotLoadedException.getInstance(); + } + + @Override + public Object get(Object key) { + if (JsonKeysForHttpMessage.INLINED_JSON_BODY_DOCUMENT_KEY.equals(key)) { + if (jsonValue == null) { + return nullOrThrow(); + } else { + return jsonValue; + } + } else if (JsonKeysForHttpMessage.INLINED_BINARY_BODY_DOCUMENT_KEY.equals(key)) { + if (binaryValue == null) { + return nullOrThrow(); + } else { + return binaryValue; + } } else { - return onlyValue; + return null; } } @Override + @NonNull public Set> entrySet() { - if (onlyValue != null) { - return Set.of(new SimpleEntry<>(JsonKeysForHttpMessage.INLINED_JSON_BODY_DOCUMENT_KEY, onlyValue)); + if (jsonValue != null) { + return Set.of(new SimpleEntry<>(JsonKeysForHttpMessage.INLINED_JSON_BODY_DOCUMENT_KEY, jsonValue)); + } else if (binaryValue != null) { + return Set.of(new SimpleEntry<>(JsonKeysForHttpMessage.INLINED_BINARY_BODY_DOCUMENT_KEY, binaryValue)); } else { - return new AbstractSet>() { + return new AbstractSet<>() { @Override + @NonNull public Iterator> iterator() { return new Iterator<>() { private int count; @@ -65,17 +97,23 @@ public boolean hasNext() { public Entry next() { if (isJson && count == 0) { ++count; - if (onlyValue != null) { + if (jsonValue != null) { return new SimpleEntry<>( JsonKeysForHttpMessage.INLINED_JSON_BODY_DOCUMENT_KEY, - onlyValue + jsonValue + ); + } else if (binaryValue != null) { + return new SimpleEntry<>( + JsonKeysForHttpMessage.INLINED_BINARY_BODY_DOCUMENT_KEY, + binaryValue ); } else { - throw PayloadNotLoadedException.getInstance(); + if (!disableThrowingPayloadNotLoaded) { + throw PayloadNotLoadedException.getInstance(); + } } - } else { - throw new NoSuchElementException(); } + throw new NoSuchElementException(); } }; } @@ -90,19 +128,25 @@ public int size() { @Override public Object put(String key, Object value) { - if (!JsonKeysForHttpMessage.INLINED_JSON_BODY_DOCUMENT_KEY.equals(key)) { + if (JsonKeysForHttpMessage.INLINED_JSON_BODY_DOCUMENT_KEY.equals(key)) { + Object old = jsonValue; + jsonValue = value; + return old; + } else if (JsonKeysForHttpMessage.INLINED_BINARY_BODY_DOCUMENT_KEY.equals(key)) { + Object old = binaryValue; + binaryValue = (ByteBuf) value; + return old; + } else { return null; } - Object old = onlyValue; - onlyValue = value; - return old; } @Override public String toString() { - final StringBuilder sb = new StringBuilder("PayloadFaultMap{"); + final var sb = new StringBuilder("PayloadFaultMap{"); sb.append("isJson=").append(isJson); - sb.append(", onlyValue=").append(onlyValue); + sb.append(", jsonValue=").append(jsonValue); + sb.append(", binaryValue=").append(binaryValue); sb.append('}'); return sb.toString(); } 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 3e91b32e8..df3247cb0 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 @@ -125,9 +125,8 @@ public TrackedFuture> finalizeRequest() { } } catch (Exception e) { this.transformationContext.addCaughtException(e); - log.atLevel( - e instanceof NettyJsonBodyAccumulateHandler.IncompleteJsonBodyException ? Level.DEBUG : Level.WARN - ).setMessage("Caught IncompleteJsonBodyException when sending the end of content").setCause(e).log(); + log.atWarn().setCause(e) + .setMessage("Caught IncompleteJsonBodyException when sending the end of content").log(); return redriveWithoutTransformation(pipelineOrchestrator.packetReceiver, e); } finally { channel.finishAndReleaseAll(); 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 7998717a5..b3ba5893d 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 @@ -1,9 +1,13 @@ package org.opensearch.migrations.replay.datahandlers.http; +import java.nio.charset.StandardCharsets; + import org.opensearch.migrations.replay.datahandlers.JsonAccumulator; import org.opensearch.migrations.replay.tracing.IReplayContexts; import org.opensearch.migrations.transform.JsonKeysForHttpMessage; +import io.netty.buffer.CompositeByteBuf; +import io.netty.buffer.Unpooled; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelInboundHandlerAdapter; import io.netty.handler.codec.http.HttpContent; @@ -22,15 +26,16 @@ public class NettyJsonBodyAccumulateHandler extends ChannelInboundHandlerAdapter private final IReplayContexts.IRequestTransformationContext context; - public static class IncompleteJsonBodyException extends NoContentException {} - JsonAccumulator jsonAccumulator; HttpJsonMessageWithFaultingPayload capturedHttpJsonMessage; + Object parsedJsonObject; + CompositeByteBuf accumulatedBody; @SneakyThrows public NettyJsonBodyAccumulateHandler(IReplayContexts.IRequestTransformationContext context) { this.context = context; this.jsonAccumulator = new JsonAccumulator(); + this.accumulatedBody = Unpooled.compositeBuffer(); } @Override @@ -38,14 +43,21 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception if (msg instanceof HttpJsonMessageWithFaultingPayload) { capturedHttpJsonMessage = (HttpJsonMessageWithFaultingPayload) msg; } else if (msg instanceof HttpContent) { - var jsonObject = jsonAccumulator.consumeByteBuffer(((HttpContent) msg).content().nioBuffer()); - if (jsonObject != null) { - capturedHttpJsonMessage.payload() - .put(JsonKeysForHttpMessage.INLINED_JSON_BODY_DOCUMENT_KEY, jsonObject); - context.onJsonPayloadParseSucceeded(); + var contentBuf = ((HttpContent) msg).content(); + accumulatedBody.addComponent(true, contentBuf.retainedDuplicate()); + parsedJsonObject = jsonAccumulator.consumeByteBuffer(contentBuf.nioBuffer()); + if (msg instanceof LastHttpContent) { + if (parsedJsonObject != null) { + capturedHttpJsonMessage.payload() + .put(JsonKeysForHttpMessage.INLINED_JSON_BODY_DOCUMENT_KEY, parsedJsonObject); + context.onJsonPayloadParseSucceeded(); + accumulatedBody.release(); + accumulatedBody = null; + } else { + capturedHttpJsonMessage.payload() + .put(JsonKeysForHttpMessage.INLINED_BINARY_BODY_DOCUMENT_KEY, accumulatedBody); + } ctx.fireChannelRead(capturedHttpJsonMessage); - } else if (msg instanceof LastHttpContent) { - throw new IncompleteJsonBodyException(); } } else { super.channelRead(ctx, msg); diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettyJsonBodyConvertHandler.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettyJsonBodyConvertHandler.java index bad57f1da..d0838fc82 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettyJsonBodyConvertHandler.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettyJsonBodyConvertHandler.java @@ -1,5 +1,6 @@ package org.opensearch.migrations.replay.datahandlers.http; +import org.opensearch.migrations.replay.datahandlers.PayloadAccessFaultingMap; import org.opensearch.migrations.transform.IJsonTransformer; import io.netty.channel.ChannelHandlerContext; @@ -15,7 +16,12 @@ public NettyJsonBodyConvertHandler(IJsonTransformer transformer) { @Override public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { if (msg instanceof HttpJsonMessageWithFaultingPayload) { - var output = transformer.transformJson((HttpJsonMessageWithFaultingPayload) msg); + var httpMsg = (HttpJsonMessageWithFaultingPayload) msg; + if (httpMsg.payload() instanceof PayloadAccessFaultingMap) { + // no reason for transforms to fault if there wasn't a body in the message + ((PayloadAccessFaultingMap) httpMsg.payload()).setDisableThrowingPayloadNotLoaded(true); + } + var output = transformer.transformJson(httpMsg); var newHttpJson = new HttpJsonMessageWithFaultingPayload(output); ctx.fireChannelRead(newHttpJson); } else { diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettyJsonBodySerializeHandler.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettyJsonBodySerializeHandler.java index 6bf8df96c..c71a1dd09 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettyJsonBodySerializeHandler.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettyJsonBodySerializeHandler.java @@ -6,9 +6,11 @@ import org.opensearch.migrations.replay.datahandlers.JsonEmitter; import org.opensearch.migrations.transform.JsonKeysForHttpMessage; +import io.netty.buffer.ByteBuf; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelInboundHandlerAdapter; import io.netty.handler.codec.http.DefaultHttpContent; +import io.netty.handler.codec.http.HttpContent; import io.netty.handler.codec.http.LastHttpContent; import lombok.extern.slf4j.Slf4j; @@ -22,12 +24,17 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception var jsonMessage = (HttpJsonMessageWithFaultingPayload) msg; var payload = jsonMessage.payload(); jsonMessage.setPayloadFaultMap(null); - var payloadContents = (Map) payload.get( - JsonKeysForHttpMessage.INLINED_JSON_BODY_DOCUMENT_KEY - ); ctx.fireChannelRead(msg); - if (payloadContents != null) { - serializePayload(ctx, payloadContents); + if (payload.containsKey(JsonKeysForHttpMessage.INLINED_JSON_BODY_DOCUMENT_KEY)) { + serializePayload(ctx, (Map) payload.get(JsonKeysForHttpMessage.INLINED_JSON_BODY_DOCUMENT_KEY)); + } else { + if (payload.containsKey(JsonKeysForHttpMessage.INLINED_BINARY_BODY_DOCUMENT_KEY)) { + var rawBody = (ByteBuf) payload.get(JsonKeysForHttpMessage.INLINED_BINARY_BODY_DOCUMENT_KEY); + if (rawBody.readableBytes() > 0) { + ctx.fireChannelRead(new DefaultHttpContent(rawBody)); + } + } + ctx.fireChannelRead(LastHttpContent.EMPTY_LAST_CONTENT); } } else { super.channelRead(ctx, msg); 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 2b35377f2..fe85e97c3 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 @@ -125,12 +125,14 @@ public void testRemoveAuthHeadersWorks() throws Exception { } @Test - public void testPartialBodyThrowsAndIsRedriven() throws Exception { + public void testPartialBodyIsPassedThrough() throws Exception { final var dummyAggregatedResponse = new AggregatedRawResponse(17, null, null, null); var testPacketCapture = new TestCapturePacketToHttpHandler(Duration.ofMillis(100), dummyAggregatedResponse); var complexTransformer = new JsonCompositeTransformer(new IJsonTransformer() { @Override public Map transformJson(Map incomingJson) { + ((Map) incomingJson.get("headers")) + .put("extraKey", "extraValue"); // just walk everything - that's enough to touch the payload and throw walkMaps(incomingJson); return incomingJson; @@ -162,13 +164,13 @@ private void walkMaps(Object o) { } transformingHandler.consumeBytes(testBytes); var returnedResponse = transformingHandler.finalizeRequest().get(); - Assertions.assertEquals(new String(testBytes, StandardCharsets.UTF_8), testPacketCapture.getCapturedAsString()); - Assertions.assertArrayEquals(testBytes, testPacketCapture.getBytesCaptured()); - Assertions.assertEquals(HttpRequestTransformationStatus.ERROR, returnedResponse.transformationStatus); - Assertions.assertInstanceOf( - NettyJsonBodyAccumulateHandler.IncompleteJsonBodyException.class, - returnedResponse.error - ); + var expectedString = new String(testBytes, StandardCharsets.UTF_8) + .replace("\r\n\r\n","\r\nextraKey: extraValue\r\n\r\n"); + Assertions.assertEquals(expectedString, testPacketCapture.getCapturedAsString()); + Assertions.assertArrayEquals(expectedString.getBytes(StandardCharsets.UTF_8), + testPacketCapture.getBytesCaptured()); + Assertions.assertEquals(HttpRequestTransformationStatus.COMPLETED, returnedResponse.transformationStatus); + Assertions.assertNull(returnedResponse.error); } public static List sliceRandomChunks(byte[] bytes, int numChunks) { diff --git a/TrafficCapture/transformationPlugins/jsonMessageTransformers/jsonMessageTransformerInterface/src/main/java/org/opensearch/migrations/transform/JsonKeysForHttpMessage.java b/TrafficCapture/transformationPlugins/jsonMessageTransformers/jsonMessageTransformerInterface/src/main/java/org/opensearch/migrations/transform/JsonKeysForHttpMessage.java index 8a88f7f92..70ff96f76 100644 --- a/TrafficCapture/transformationPlugins/jsonMessageTransformers/jsonMessageTransformerInterface/src/main/java/org/opensearch/migrations/transform/JsonKeysForHttpMessage.java +++ b/TrafficCapture/transformationPlugins/jsonMessageTransformers/jsonMessageTransformerInterface/src/main/java/org/opensearch/migrations/transform/JsonKeysForHttpMessage.java @@ -16,4 +16,9 @@ private JsonKeysForHttpMessage() {} * the payload object will be an empty map. */ public static final String INLINED_JSON_BODY_DOCUMENT_KEY = "inlinedJsonBody"; + /** + * This maps to a ByteBuf that is owned by the caller. + * Any consumers should retain if they need to access it later. This may be UTF8, UTF16 encoded, or something else. + */ + public static final String INLINED_BINARY_BODY_DOCUMENT_KEY = "inlinedBinaryBody"; } From b39eb8e8598db020ca6747a6768cacee4424c803 Mon Sep 17 00:00:00 2001 From: Greg Schohn Date: Sun, 8 Sep 2024 10:04:31 -0400 Subject: [PATCH 03/12] Support for nd-json payloads. These are deserialized into a list of json objects and are accessed via a new top-level payload key so that they're distinguishable from a single json object. Any leftover bytes are now appended after json objects. Both the single-json and ndjson-list (mutually exclusive) and the leftover bytes are all accessible for transformations. I've also added a transformation script to the replayer's command for docker-compose to exercise json body transformations. Signed-off-by: Greg Schohn --- .../src/main/docker/docker-compose.yml | 2 +- .../replay/datahandlers/JsonAccumulator.java | 23 +++- .../PayloadAccessFaultingMap.java | 115 +----------------- .../http/HttpJsonTransformingConsumer.java | 1 - .../http/NettyJsonBodyAccumulateHandler.java | 45 +++++-- .../http/NettyJsonBodySerializeHandler.java | 50 ++++++-- .../datahandlers/JsonAccumulatorTest.java | 2 +- .../HttpJsonTransformingConsumerTest.java | 79 ++++++++++++ .../replay/MultipleJoltScriptsTest.java | 2 +- .../transform/JsonKeysForHttpMessage.java | 4 + .../transform/TypeMappingsExcisionTest.java | 2 +- 11 files changed, 186 insertions(+), 139 deletions(-) diff --git a/TrafficCapture/dockerSolution/src/main/docker/docker-compose.yml b/TrafficCapture/dockerSolution/src/main/docker/docker-compose.yml index 04c432a7c..aab6e6ef4 100644 --- a/TrafficCapture/dockerSolution/src/main/docker/docker-compose.yml +++ b/TrafficCapture/dockerSolution/src/main/docker/docker-compose.yml @@ -78,7 +78,7 @@ services: condition: service_started opensearchtarget: condition: service_started - command: /bin/sh -c "/runJavaWithClasspath.sh org.opensearch.migrations.replay.TrafficReplayer --speedup-factor 2 https://opensearchtarget:9200 --auth-header-value Basic\\ YWRtaW46bXlTdHJvbmdQYXNzd29yZDEyMyE= --insecure --kafka-traffic-brokers kafka:9092 --kafka-traffic-topic logging-traffic-topic --kafka-traffic-group-id logging-group-default --otelCollectorEndpoint http://otel-collector:4317" + command: /bin/sh -c "/runJavaWithClasspath.sh org.opensearch.migrations.replay.TrafficReplayer --speedup-factor 2 https://opensearchtarget:9200 --auth-header-value Basic\\ YWRtaW46bXlTdHJvbmdQYXNzd29yZDEyMyE= --insecure --kafka-traffic-brokers kafka:9092 --kafka-traffic-topic logging-traffic-topic --kafka-traffic-group-id logging-group-default --otelCollectorEndpoint http://otel-collector:4317 --transformer-config-base64 W3sgIkpzb25Kb2x0VHJhbnNmb3JtZXJQcm92aWRlciI6ClsKICB7CiAgICAic2NyaXB0IjogewogICAgICAib3BlcmF0aW9uIjogInNoaWZ0IiwKICAgICAgInNwZWMiOiB7CiAgICAgICAgIm1ldGhvZCI6ICJtZXRob2QiLAogICAgICAgICJVUkkiOiAiVVJJIiwKICAgICAgICAiaGVhZGVycyI6ICJoZWFkZXJzIiwKICAgICAgICAicGF5bG9hZCI6IHsKICAgICAgICAgICJpbmxpbmVkSnNvbkJvZHkiOiB7CiAgICAgICAgICAgICJ0b3AiOiB7CiAgICAgICAgICAgICAgInRhZ1RvRXhjaXNlIjogewogICAgICAgICAgICAgICAgIioiOiAicGF5bG9hZC5pbmxpbmVkSnNvbkJvZHkudG9wLiYiIAogICAgICAgICAgICAgIH0sCiAgICAgICAgICAgICAgIioiOiAicGF5bG9hZC5pbmxpbmVkSnNvbkJvZHkudG9wLiYiCiAgICAgICAgICAgIH0sCiAgICAgICAgICAiKiI6ICJwYXlsb2FkLmlubGluZWRKc29uQm9keS4mIgogICAgICAgICAgfQogICAgICAgIH0KICAgICAgfQogICAgfQogIH0sIAogewogICAic2NyaXB0IjogewogICAgICJvcGVyYXRpb24iOiAibW9kaWZ5LW92ZXJ3cml0ZS1iZXRhIiwKICAgICAic3BlYyI6IHsKICAgICAgICJVUkkiOiAiPXNwbGl0KCcvZXh0cmFUaGluZ1RvUmVtb3ZlJyxAKDEsJikpIgogICAgIH0KICB9CiB9LAogewogICAic2NyaXB0IjogewogICAgICJvcGVyYXRpb24iOiAibW9kaWZ5LW92ZXJ3cml0ZS1iZXRhIiwKICAgICAic3BlYyI6IHsKICAgICAgICJVUkkiOiAiPWpvaW4oJycsQCgxLCYpKSIKICAgICB9CiAgfQogfQpdCn1dCg==" opensearchtarget: image: 'opensearchproject/opensearch:2.15.0' diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/JsonAccumulator.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/JsonAccumulator.java index 34672dfa5..72c3fce4e 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/JsonAccumulator.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/JsonAccumulator.java @@ -12,6 +12,7 @@ import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.async.ByteBufferFeeder; +import lombok.Getter; import lombok.extern.slf4j.Slf4j; /** @@ -30,28 +31,42 @@ public class JsonAccumulator { * Name in the stack. */ private final Deque jsonObjectStack; + private final ByteBufferFeeder feeder; + @Getter + private long totalBytesFullyConsumed; public JsonAccumulator() throws IOException { jsonObjectStack = new ArrayDeque<>(); JsonFactory factory = new JsonFactory(); parser = factory.createNonBlockingByteBufferParser(); + feeder = (ByteBufferFeeder) parser.getNonBlockingInputFeeder(); } protected Map createMap() { return new LinkedHashMap<>(); } + public boolean hasPartialValues() { + return !jsonObjectStack.isEmpty(); + } + /** * Returns the top-level object once it has been fully constructed or null if more input is still required. * @param byteBuffer * @return * @throws IOException */ - public Object consumeByteBuffer(ByteBuffer byteBuffer) throws IOException { - ByteBufferFeeder feeder = (ByteBufferFeeder) parser.getNonBlockingInputFeeder(); + public Object consumeByteBufferForSingleObject(ByteBuffer byteBuffer) throws IOException { + consumeByteBuffer(byteBuffer); + return getNextTopLevelObject(); + } + + public void consumeByteBuffer(ByteBuffer byteBuffer) throws IOException { log.trace("Consuming bytes: " + byteBuffer.toString()); feeder.feedInput(byteBuffer); - + } + + public Object getNextTopLevelObject() throws IOException { while (!parser.isClosed()) { var token = parser.nextToken(); if (token == null) { @@ -71,6 +86,7 @@ public Object consumeByteBuffer(ByteBuffer byteBuffer) throws IOException { var array = ((ArrayList) jsonObjectStack.pop()).toArray(); pushCompletedValue(array); if (jsonObjectStack.isEmpty()) { + totalBytesFullyConsumed = parser.currentLocation().getByteOffset(); return array; } break; @@ -81,6 +97,7 @@ public Object consumeByteBuffer(ByteBuffer byteBuffer) throws IOException { case END_OBJECT: { var popped = jsonObjectStack.pop(); if (jsonObjectStack.isEmpty()) { + totalBytesFullyConsumed = parser.currentLocation().getByteOffset(); return popped; } else { pushCompletedValue(popped); diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/PayloadAccessFaultingMap.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/PayloadAccessFaultingMap.java index 64cb7f64d..801e9e1a4 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/PayloadAccessFaultingMap.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/PayloadAccessFaultingMap.java @@ -1,19 +1,12 @@ package org.opensearch.migrations.replay.datahandlers; -import java.util.AbstractMap; -import java.util.AbstractSet; -import java.util.Iterator; -import java.util.NoSuchElementException; import java.util.Optional; -import java.util.Set; +import java.util.TreeMap; import org.opensearch.migrations.replay.datahandlers.http.StrictCaseInsensitiveHttpHeadersMap; -import org.opensearch.migrations.transform.JsonKeysForHttpMessage; -import io.netty.buffer.ByteBuf; import lombok.EqualsAndHashCode; import lombok.Getter; -import lombok.NonNull; import lombok.Setter; import lombok.extern.slf4j.Slf4j; @@ -26,11 +19,9 @@ */ @EqualsAndHashCode(callSuper = false) @Slf4j -public class PayloadAccessFaultingMap extends AbstractMap { +public class PayloadAccessFaultingMap extends TreeMap { private final boolean isJson; - private Object jsonValue; - private ByteBuf binaryValue; @Getter @Setter private boolean disableThrowingPayloadNotLoaded; @@ -41,12 +32,6 @@ public PayloadAccessFaultingMap(StrictCaseInsensitiveHttpHeadersMap headers) { .orElse(false); } - @Override - public boolean containsKey(Object key) { - return (JsonKeysForHttpMessage.INLINED_JSON_BODY_DOCUMENT_KEY.equals(key) && jsonValue != null) || - (JsonKeysForHttpMessage.INLINED_BINARY_BODY_DOCUMENT_KEY.equals(key) && binaryValue != null); - } - private Object nullOrThrow() { if (disableThrowingPayloadNotLoaded) { return null; @@ -56,98 +41,10 @@ private Object nullOrThrow() { @Override public Object get(Object key) { - if (JsonKeysForHttpMessage.INLINED_JSON_BODY_DOCUMENT_KEY.equals(key)) { - if (jsonValue == null) { - return nullOrThrow(); - } else { - return jsonValue; - } - } else if (JsonKeysForHttpMessage.INLINED_BINARY_BODY_DOCUMENT_KEY.equals(key)) { - if (binaryValue == null) { - return nullOrThrow(); - } else { - return binaryValue; - } - } else { - return null; + var value = super.get(key); + if (value == null && !disableThrowingPayloadNotLoaded) { + throw PayloadNotLoadedException.getInstance(); } - } - - @Override - @NonNull - public Set> entrySet() { - if (jsonValue != null) { - return Set.of(new SimpleEntry<>(JsonKeysForHttpMessage.INLINED_JSON_BODY_DOCUMENT_KEY, jsonValue)); - } else if (binaryValue != null) { - return Set.of(new SimpleEntry<>(JsonKeysForHttpMessage.INLINED_BINARY_BODY_DOCUMENT_KEY, binaryValue)); - } else { - return new AbstractSet<>() { - @Override - @NonNull - public Iterator> iterator() { - return new Iterator<>() { - private int count; - - @Override - public boolean hasNext() { - return count == 0 && isJson; - } - - @Override - public Entry next() { - if (isJson && count == 0) { - ++count; - if (jsonValue != null) { - return new SimpleEntry<>( - JsonKeysForHttpMessage.INLINED_JSON_BODY_DOCUMENT_KEY, - jsonValue - ); - } else if (binaryValue != null) { - return new SimpleEntry<>( - JsonKeysForHttpMessage.INLINED_BINARY_BODY_DOCUMENT_KEY, - binaryValue - ); - } else { - if (!disableThrowingPayloadNotLoaded) { - throw PayloadNotLoadedException.getInstance(); - } - } - } - throw new NoSuchElementException(); - } - }; - } - - @Override - public int size() { - return isJson ? 1 : 0; - } - }; - } - } - - @Override - public Object put(String key, Object value) { - if (JsonKeysForHttpMessage.INLINED_JSON_BODY_DOCUMENT_KEY.equals(key)) { - Object old = jsonValue; - jsonValue = value; - return old; - } else if (JsonKeysForHttpMessage.INLINED_BINARY_BODY_DOCUMENT_KEY.equals(key)) { - Object old = binaryValue; - binaryValue = (ByteBuf) value; - return old; - } else { - return null; - } - } - - @Override - public String toString() { - final var sb = new StringBuilder("PayloadFaultMap{"); - sb.append("isJson=").append(isJson); - sb.append(", jsonValue=").append(jsonValue); - sb.append(", binaryValue=").append(binaryValue); - sb.append('}'); - return sb.toString(); + return value; } } 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 df3247cb0..1891cabf0 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 @@ -20,7 +20,6 @@ import io.netty.channel.embedded.EmbeddedChannel; import io.netty.handler.codec.http.HttpRequestDecoder; import lombok.extern.slf4j.Slf4j; -import org.slf4j.event.Level; /** * This class implements a packet consuming interface by using an EmbeddedChannel to write individual 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 b3ba5893d..71e11bf09 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 @@ -1,6 +1,7 @@ package org.opensearch.migrations.replay.datahandlers.http; -import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; import org.opensearch.migrations.replay.datahandlers.JsonAccumulator; import org.opensearch.migrations.replay.tracing.IReplayContexts; @@ -28,14 +29,18 @@ public class NettyJsonBodyAccumulateHandler extends ChannelInboundHandlerAdapter JsonAccumulator jsonAccumulator; HttpJsonMessageWithFaultingPayload capturedHttpJsonMessage; - Object parsedJsonObject; + List parsedJsonObjects; CompositeByteBuf accumulatedBody; @SneakyThrows public NettyJsonBodyAccumulateHandler(IReplayContexts.IRequestTransformationContext context) { this.context = context; this.jsonAccumulator = new JsonAccumulator(); - this.accumulatedBody = Unpooled.compositeBuffer(); + // use 1024 (as opposed to the default of 16) because we really don't ever want the hit of a consolidation. + // For this buffer to continue to be used, we are far-off the happy-path. + // Consolidating will likely burn more cycles + this.accumulatedBody = Unpooled.compositeBuffer(1024); + this.parsedJsonObjects = new ArrayList<>(); } @Override @@ -45,17 +50,37 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception } else if (msg instanceof HttpContent) { var contentBuf = ((HttpContent) msg).content(); accumulatedBody.addComponent(true, contentBuf.retainedDuplicate()); - parsedJsonObject = jsonAccumulator.consumeByteBuffer(contentBuf.nioBuffer()); + var nioBuf = contentBuf.nioBuffer(); + jsonAccumulator.consumeByteBuffer(nioBuf); + Object nextObj; + while ((nextObj = jsonAccumulator.getNextTopLevelObject()) != null) { + parsedJsonObjects.add(nextObj); + } if (msg instanceof LastHttpContent) { - if (parsedJsonObject != null) { + if (!parsedJsonObjects.isEmpty()) { + var payload = capturedHttpJsonMessage.payload(); + if (parsedJsonObjects.size() > 1) { + payload.put(JsonKeysForHttpMessage.INLINED_NDJSON_BODIES_DOCUMENT_KEY, parsedJsonObjects); + } else { + payload.put(JsonKeysForHttpMessage.INLINED_JSON_BODY_DOCUMENT_KEY, parsedJsonObjects.get(0)); + } + if (!jsonAccumulator.hasPartialValues()) { + context.onJsonPayloadParseSucceeded(); + } + } + if (jsonAccumulator.hasPartialValues()) { + if (jsonAccumulator.getTotalBytesFullyConsumed() > Integer.MAX_VALUE) { + throw new IndexOutOfBoundsException("JSON contents were too large " + + jsonAccumulator.getTotalBytesFullyConsumed() + " for a single composite ByteBuf"); + } + // skip the contents that were already parsed and included in the payload as parsed json + accumulatedBody.readerIndex((int) jsonAccumulator.getTotalBytesFullyConsumed()); + // and pass the remaining stream capturedHttpJsonMessage.payload() - .put(JsonKeysForHttpMessage.INLINED_JSON_BODY_DOCUMENT_KEY, parsedJsonObject); - context.onJsonPayloadParseSucceeded(); + .put(JsonKeysForHttpMessage.INLINED_BINARY_BODY_DOCUMENT_KEY, accumulatedBody); + } else { accumulatedBody.release(); accumulatedBody = null; - } else { - capturedHttpJsonMessage.payload() - .put(JsonKeysForHttpMessage.INLINED_BINARY_BODY_DOCUMENT_KEY, accumulatedBody); } ctx.fireChannelRead(capturedHttpJsonMessage); } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettyJsonBodySerializeHandler.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettyJsonBodySerializeHandler.java index c71a1dd09..45cb7c52b 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettyJsonBodySerializeHandler.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettyJsonBodySerializeHandler.java @@ -1,16 +1,16 @@ package org.opensearch.migrations.replay.datahandlers.http; import java.io.IOException; -import java.util.Map; +import java.util.List; import org.opensearch.migrations.replay.datahandlers.JsonEmitter; import org.opensearch.migrations.transform.JsonKeysForHttpMessage; import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelInboundHandlerAdapter; import io.netty.handler.codec.http.DefaultHttpContent; -import io.netty.handler.codec.http.HttpContent; import io.netty.handler.codec.http.LastHttpContent; import lombok.extern.slf4j.Slf4j; @@ -26,28 +26,54 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception jsonMessage.setPayloadFaultMap(null); ctx.fireChannelRead(msg); if (payload.containsKey(JsonKeysForHttpMessage.INLINED_JSON_BODY_DOCUMENT_KEY)) { - serializePayload(ctx, (Map) payload.get(JsonKeysForHttpMessage.INLINED_JSON_BODY_DOCUMENT_KEY)); - } else { - if (payload.containsKey(JsonKeysForHttpMessage.INLINED_BINARY_BODY_DOCUMENT_KEY)) { - var rawBody = (ByteBuf) payload.get(JsonKeysForHttpMessage.INLINED_BINARY_BODY_DOCUMENT_KEY); - if (rawBody.readableBytes() > 0) { - ctx.fireChannelRead(new DefaultHttpContent(rawBody)); - } + serializePayload(ctx, payload.get(JsonKeysForHttpMessage.INLINED_JSON_BODY_DOCUMENT_KEY)); + } else if (payload.containsKey(JsonKeysForHttpMessage.INLINED_NDJSON_BODIES_DOCUMENT_KEY)) { + serializePayloadList(ctx, + (List) payload.get(JsonKeysForHttpMessage.INLINED_NDJSON_BODIES_DOCUMENT_KEY), + !payload.containsKey(JsonKeysForHttpMessage.INLINED_BINARY_BODY_DOCUMENT_KEY)); + } + if (payload.containsKey(JsonKeysForHttpMessage.INLINED_BINARY_BODY_DOCUMENT_KEY)) { + var rawBody = (ByteBuf) payload.get(JsonKeysForHttpMessage.INLINED_BINARY_BODY_DOCUMENT_KEY); + if (rawBody.readableBytes() > 0) { + ctx.fireChannelRead(new DefaultHttpContent(rawBody)); } - ctx.fireChannelRead(LastHttpContent.EMPTY_LAST_CONTENT); } + ctx.fireChannelRead(LastHttpContent.EMPTY_LAST_CONTENT); } else { super.channelRead(ctx, msg); } } - private void serializePayload(ChannelHandlerContext ctx, Map payload) throws IOException { + private static final ByteBuf NEWLINE = Unpooled.unreleasableBuffer(Unpooled.wrappedBuffer(new byte[]{'\n'})); + + private void serializePayloadList(ChannelHandlerContext ctx, List payloadList, boolean addLastNewline) + throws IOException + { + var it = payloadList.iterator(); + while (it.hasNext()) { + var payload = it.next(); + try (var jsonEmitter = new JsonEmitter(ctx.alloc())) { + var pac = jsonEmitter.getChunkAndContinuations(payload, NUM_BYTES_TO_ACCUMULATE_BEFORE_FIRING); + while (true) { + ctx.fireChannelRead(new DefaultHttpContent(pac.partialSerializedContents)); + if (pac.nextSupplier == null) { + break; + } + pac = pac.nextSupplier.get(); + } + if (addLastNewline || it.hasNext()) { + ctx.fireChannelRead(new DefaultHttpContent(NEWLINE.duplicate())); + } + } + } + } + + private void serializePayload(ChannelHandlerContext ctx, Object payload) throws IOException{ try (var jsonEmitter = new JsonEmitter(ctx.alloc())) { var pac = jsonEmitter.getChunkAndContinuations(payload, NUM_BYTES_TO_ACCUMULATE_BEFORE_FIRING); while (true) { ctx.fireChannelRead(new DefaultHttpContent(pac.partialSerializedContents)); if (pac.nextSupplier == null) { - ctx.fireChannelRead(LastHttpContent.EMPTY_LAST_CONTENT); break; } pac = pac.nextSupplier.get(); diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datahandlers/JsonAccumulatorTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datahandlers/JsonAccumulatorTest.java index cc9683e1c..49a40c92e 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datahandlers/JsonAccumulatorTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datahandlers/JsonAccumulatorTest.java @@ -31,7 +31,7 @@ static Object readJson(byte[] testFileBytes, int chunkBound) throws IOException var chunkSize = Math.min(r.nextInt(chunkBound), chunkByteBuffer.remaining()); chunkByteBuffer.limit(chunkSize + i); i += chunkSize; - var completedObject = jsonParser.consumeByteBuffer(chunkByteBuffer); + var completedObject = jsonParser.consumeByteBufferForSingleObject(chunkByteBuffer); if (completedObject != null) { Assertions.assertEquals(testFileBytes.length, i); return completedObject; 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 fe85e97c3..82539288c 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 @@ -24,11 +24,24 @@ import org.opensearch.migrations.tracing.InstrumentationTest; import org.opensearch.migrations.transform.IJsonTransformer; import org.opensearch.migrations.transform.JsonCompositeTransformer; +import org.opensearch.migrations.transform.JsonKeysForHttpMessage; import org.opensearch.migrations.transform.RemovingAuthTransformerFactory; +import io.netty.buffer.ByteBuf; + @WrapWithNettyLeakDetection class HttpJsonTransformingConsumerTest extends InstrumentationTest { + private final static String NDJSON_TEST_REQUEST = ( + "POST /test HTTP/1.1\r\n" + + "Host: foo.example\r\n" + + "Content-Type: application/json\r\n" + + "Content-Length: 97\r\n" + + "\r\n" + + "{\"index\":{\"_index\":\"test\",\"_id\":\"2\"}}\n" + + "{\"field1\":\"value1\"}\n" + + "{\"delete\":{\"_index\":\"test\",\"_id\":\"1\"}}\n"); + private static Stream provideTestParameters() { Integer[] attemptedChunks = { 1, 2, 4, 8, 100, 1000, Integer.MAX_VALUE }; Boolean[] transformationOptions = { true, false }; @@ -131,6 +144,9 @@ public void testPartialBodyIsPassedThrough() throws Exception { var complexTransformer = new JsonCompositeTransformer(new IJsonTransformer() { @Override public Map transformJson(Map incomingJson) { + var payload = (Map) incomingJson.get("payload"); + Assertions.assertNull(payload.get(JsonKeysForHttpMessage.INLINED_NDJSON_BODIES_DOCUMENT_KEY)); + Assertions.assertNull(payload.get(JsonKeysForHttpMessage.INLINED_JSON_BODY_DOCUMENT_KEY)); ((Map) incomingJson.get("headers")) .put("extraKey", "extraValue"); // just walk everything - that's enough to touch the payload and throw @@ -173,6 +189,69 @@ private void walkMaps(Object o) { Assertions.assertNull(returnedResponse.error); } + @Test + public void testNewlineDelimitedJsonBodyIsHandled() throws Exception { + final var dummyAggregatedResponse = new AggregatedRawResponse(19, null, null, null); + var testPacketCapture = new TestCapturePacketToHttpHandler(Duration.ofMillis(100), dummyAggregatedResponse); + var sizeCalculatingTransformer = new JsonCompositeTransformer(incomingJson -> { + var payload = (Map) incomingJson.get("payload"); + Assertions.assertNull(payload.get(JsonKeysForHttpMessage.INLINED_JSON_BODY_DOCUMENT_KEY)); + Assertions.assertNull(payload.get(JsonKeysForHttpMessage.INLINED_BINARY_BODY_DOCUMENT_KEY)); + var list = (List) payload.get(JsonKeysForHttpMessage.INLINED_NDJSON_BODIES_DOCUMENT_KEY); + ((Map) incomingJson.get("headers")) + .put("listSize", ""+list.size()); + return incomingJson; + }); + var transformingHandler = new HttpJsonTransformingConsumer( + sizeCalculatingTransformer, + null, + testPacketCapture, + rootContext.getTestConnectionRequestContext(0) + ); + + transformingHandler.consumeBytes(NDJSON_TEST_REQUEST.getBytes(StandardCharsets.UTF_8)); + var returnedResponse = transformingHandler.finalizeRequest().get(); + var expectedString = NDJSON_TEST_REQUEST.replace("\r\n\r\n","\r\nlistSize: 3\r\n\r\n"); + Assertions.assertEquals(expectedString, testPacketCapture.getCapturedAsString()); + Assertions.assertEquals(HttpRequestTransformationStatus.COMPLETED, returnedResponse.transformationStatus); + Assertions.assertNull(returnedResponse.error); + } + + @Test + public void testPartialNewlineDelimitedJsonBodyIsHandled() throws Exception { + final var dummyAggregatedResponse = new AggregatedRawResponse(19, null, null, null); + var testPacketCapture = new TestCapturePacketToHttpHandler(Duration.ofMillis(100), dummyAggregatedResponse); + var sizeCalculatingTransformer = new JsonCompositeTransformer(incomingJson -> { + var payload = (Map) incomingJson.get("payload"); + Assertions.assertNull(payload.get(JsonKeysForHttpMessage.INLINED_JSON_BODY_DOCUMENT_KEY)); + Assertions.assertNotNull(payload.get(JsonKeysForHttpMessage.INLINED_BINARY_BODY_DOCUMENT_KEY)); + var list = (List) payload.get(JsonKeysForHttpMessage.INLINED_NDJSON_BODIES_DOCUMENT_KEY); + var leftoverBytes = (ByteBuf) payload.get(JsonKeysForHttpMessage.INLINED_BINARY_BODY_DOCUMENT_KEY); + var headers = (Map) incomingJson.get("headers"); + headers.put("listSize", "" + list.size()); + headers.put("leftover", "" + leftoverBytes.readableBytes()); + return incomingJson; + }); + var transformingHandler = new HttpJsonTransformingConsumer( + sizeCalculatingTransformer, + null, + testPacketCapture, + rootContext.getTestConnectionRequestContext(0) + ); + + var testString = NDJSON_TEST_REQUEST + .replace("Content-Length: 97", "Content-Length: 87") + .substring(0, NDJSON_TEST_REQUEST.length()-10); + var testBytes = testString.getBytes(StandardCharsets.UTF_8); + transformingHandler.consumeBytes(testBytes); + var returnedResponse = transformingHandler.finalizeRequest().get(); + var expectedString = new String(testBytes, StandardCharsets.UTF_8) + .replace("\r\n\r\n","\r\nlistSize: 2\r\nleftover: 30\r\n\r\n"); + Assertions.assertEquals(expectedString, testPacketCapture.getCapturedAsString()); + Assertions.assertEquals(HttpRequestTransformationStatus.COMPLETED, returnedResponse.transformationStatus); + Assertions.assertNull(returnedResponse.error); + } + public static List sliceRandomChunks(byte[] bytes, int numChunks) { Random random = new Random(0); List chunkSizes = new ArrayList<>(numChunks); diff --git a/TrafficCapture/transformationPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/java/org/opensearch/migrations/replay/MultipleJoltScriptsTest.java b/TrafficCapture/transformationPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/java/org/opensearch/migrations/replay/MultipleJoltScriptsTest.java index 2f0971ee9..c41e86363 100644 --- a/TrafficCapture/transformationPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/java/org/opensearch/migrations/replay/MultipleJoltScriptsTest.java +++ b/TrafficCapture/transformationPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/java/org/opensearch/migrations/replay/MultipleJoltScriptsTest.java @@ -141,4 +141,4 @@ public void testExciseWhenPresent() throws Exception { Assertions.assertEquals("testhostname", ((Map) newDoc.get(JsonKeysForHttpMessage.HEADERS_KEY)).get("host")); } -} \ No newline at end of file +} diff --git a/TrafficCapture/transformationPlugins/jsonMessageTransformers/jsonMessageTransformerInterface/src/main/java/org/opensearch/migrations/transform/JsonKeysForHttpMessage.java b/TrafficCapture/transformationPlugins/jsonMessageTransformers/jsonMessageTransformerInterface/src/main/java/org/opensearch/migrations/transform/JsonKeysForHttpMessage.java index 70ff96f76..f5ff837c2 100644 --- a/TrafficCapture/transformationPlugins/jsonMessageTransformers/jsonMessageTransformerInterface/src/main/java/org/opensearch/migrations/transform/JsonKeysForHttpMessage.java +++ b/TrafficCapture/transformationPlugins/jsonMessageTransformers/jsonMessageTransformerInterface/src/main/java/org/opensearch/migrations/transform/JsonKeysForHttpMessage.java @@ -16,6 +16,10 @@ private JsonKeysForHttpMessage() {} * the payload object will be an empty map. */ public static final String INLINED_JSON_BODY_DOCUMENT_KEY = "inlinedJsonBody"; + /** + * for the type application + */ + public static final String INLINED_NDJSON_BODIES_DOCUMENT_KEY = "inlinedJsonSequenceBodies"; /** * This maps to a ByteBuf that is owned by the caller. * Any consumers should retain if they need to access it later. This may be UTF8, UTF16 encoded, or something else. diff --git a/TrafficCapture/transformationPlugins/jsonMessageTransformers/openSearch23PlusTargetTransformerProvider/src/test/java/org/opensearch/migrations/transform/TypeMappingsExcisionTest.java b/TrafficCapture/transformationPlugins/jsonMessageTransformers/openSearch23PlusTargetTransformerProvider/src/test/java/org/opensearch/migrations/transform/TypeMappingsExcisionTest.java index 5eb218337..4f9fd36e2 100644 --- a/TrafficCapture/transformationPlugins/jsonMessageTransformers/openSearch23PlusTargetTransformerProvider/src/test/java/org/opensearch/migrations/transform/TypeMappingsExcisionTest.java +++ b/TrafficCapture/transformationPlugins/jsonMessageTransformers/openSearch23PlusTargetTransformerProvider/src/test/java/org/opensearch/migrations/transform/TypeMappingsExcisionTest.java @@ -51,7 +51,7 @@ private static Map parseJsonFromResourceName(String resourceName var isr = new InputStreamReader(resourceStream, StandardCharsets.UTF_8) ) { var expectedBytes = CharStreams.toString(isr).getBytes(StandardCharsets.UTF_8); - return (Map) jsonAccumulator.consumeByteBuffer(ByteBuffer.wrap(expectedBytes)); + return (Map) jsonAccumulator.consumeByteBufferForSingleObject(ByteBuffer.wrap(expectedBytes)); } } From 799aac6dcdf078089ab1ba4b79970947c7df33e1 Mon Sep 17 00:00:00 2001 From: Greg Schohn Date: Sun, 8 Sep 2024 11:49:40 -0400 Subject: [PATCH 04/12] Bring back the PayloadAccessFaultingMap as an AbstractMap that implements its own EntrySet... Accesses to the TreeMap derived object weren't causing faults from transformations, so the it never looked like anybody was trying to access payload contents, causing the transforming pipeline to NOT parse the json or pass the bodies to the transformers. Also, when there is no payload at all, we'll pass that through as an empty byte array. It might be better to pass no payload at all, but that will be a slightly greater change. At the moment, this change allows the OpenSearch Benchmark test run traffic to completely move over to the target. Signed-off-by: Greg Schohn --- .../PayloadAccessFaultingMap.java | 46 +++++++++++++++++-- .../http/NettyJsonBodyAccumulateHandler.java | 2 +- 2 files changed, 42 insertions(+), 6 deletions(-) diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/PayloadAccessFaultingMap.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/PayloadAccessFaultingMap.java index 801e9e1a4..763cbd868 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/PayloadAccessFaultingMap.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/PayloadAccessFaultingMap.java @@ -1,12 +1,18 @@ package org.opensearch.migrations.replay.datahandlers; +import java.util.AbstractMap; +import java.util.AbstractSet; +import java.util.Iterator; +import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.TreeMap; import org.opensearch.migrations.replay.datahandlers.http.StrictCaseInsensitiveHttpHeadersMap; import lombok.EqualsAndHashCode; import lombok.Getter; +import lombok.NonNull; import lombok.Setter; import lombok.extern.slf4j.Slf4j; @@ -19,24 +25,54 @@ */ @EqualsAndHashCode(callSuper = false) @Slf4j -public class PayloadAccessFaultingMap extends TreeMap { +public class PayloadAccessFaultingMap extends AbstractMap { private final boolean isJson; + TreeMap underlyingMap; @Getter @Setter private boolean disableThrowingPayloadNotLoaded; public PayloadAccessFaultingMap(StrictCaseInsensitiveHttpHeadersMap headers) { + underlyingMap = new TreeMap<>(); isJson = Optional.ofNullable(headers.get("content-type")) .map(list -> list.stream().anyMatch(s -> s.startsWith("application/json"))) .orElse(false); } - private Object nullOrThrow() { - if (disableThrowingPayloadNotLoaded) { - return null; + @Override + @NonNull + public Set> entrySet() { + if (underlyingMap.isEmpty() && !disableThrowingPayloadNotLoaded) { + return new AbstractSet<>() { + @Override + @NonNull + public Iterator> iterator() { + return new Iterator<>() { + @Override + public boolean hasNext() { + throw PayloadNotLoadedException.getInstance(); + } + + @Override + public Map.Entry next() { + throw PayloadNotLoadedException.getInstance(); + } + }; + } + + @Override + public int size() { + throw PayloadNotLoadedException.getInstance(); + } + }; + } else { + return underlyingMap.entrySet(); } - throw PayloadNotLoadedException.getInstance(); + } + @Override + public Object put(String key, Object value) { + return underlyingMap.put(key, value); } @Override 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 71e11bf09..786ea30a2 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 @@ -68,7 +68,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception context.onJsonPayloadParseSucceeded(); } } - if (jsonAccumulator.hasPartialValues()) { + if (jsonAccumulator.hasPartialValues() || parsedJsonObjects.isEmpty()) { if (jsonAccumulator.getTotalBytesFullyConsumed() > Integer.MAX_VALUE) { throw new IndexOutOfBoundsException("JSON contents were too large " + jsonAccumulator.getTotalBytesFullyConsumed() + " for a single composite ByteBuf"); From 4d2702486e96c6a60d6907f4ccf9767f321a5c83 Mon Sep 17 00:00:00 2001 From: Greg Schohn Date: Sun, 8 Sep 2024 13:50:08 -0400 Subject: [PATCH 05/12] Touchup README, make a unit test a better example, and remove transformations from docker-compose as the default. On the last point, I'll continue to look into differences that could be causing e2e tests to fail. Signed-off-by: Greg Schohn --- .../src/main/docker/docker-compose.yml | 2 +- TrafficCapture/trafficReplayer/README.md | 27 ++++++++++++------- .../replay/MultipleJoltScriptsTest.java | 17 ++++++------ 3 files changed, 28 insertions(+), 18 deletions(-) diff --git a/TrafficCapture/dockerSolution/src/main/docker/docker-compose.yml b/TrafficCapture/dockerSolution/src/main/docker/docker-compose.yml index aab6e6ef4..3a3212b83 100644 --- a/TrafficCapture/dockerSolution/src/main/docker/docker-compose.yml +++ b/TrafficCapture/dockerSolution/src/main/docker/docker-compose.yml @@ -78,7 +78,7 @@ services: condition: service_started opensearchtarget: condition: service_started - command: /bin/sh -c "/runJavaWithClasspath.sh org.opensearch.migrations.replay.TrafficReplayer --speedup-factor 2 https://opensearchtarget:9200 --auth-header-value Basic\\ YWRtaW46bXlTdHJvbmdQYXNzd29yZDEyMyE= --insecure --kafka-traffic-brokers kafka:9092 --kafka-traffic-topic logging-traffic-topic --kafka-traffic-group-id logging-group-default --otelCollectorEndpoint http://otel-collector:4317 --transformer-config-base64 W3sgIkpzb25Kb2x0VHJhbnNmb3JtZXJQcm92aWRlciI6ClsKICB7CiAgICAic2NyaXB0IjogewogICAgICAib3BlcmF0aW9uIjogInNoaWZ0IiwKICAgICAgInNwZWMiOiB7CiAgICAgICAgIm1ldGhvZCI6ICJtZXRob2QiLAogICAgICAgICJVUkkiOiAiVVJJIiwKICAgICAgICAiaGVhZGVycyI6ICJoZWFkZXJzIiwKICAgICAgICAicGF5bG9hZCI6IHsKICAgICAgICAgICJpbmxpbmVkSnNvbkJvZHkiOiB7CiAgICAgICAgICAgICJ0b3AiOiB7CiAgICAgICAgICAgICAgInRhZ1RvRXhjaXNlIjogewogICAgICAgICAgICAgICAgIioiOiAicGF5bG9hZC5pbmxpbmVkSnNvbkJvZHkudG9wLiYiIAogICAgICAgICAgICAgIH0sCiAgICAgICAgICAgICAgIioiOiAicGF5bG9hZC5pbmxpbmVkSnNvbkJvZHkudG9wLiYiCiAgICAgICAgICAgIH0sCiAgICAgICAgICAiKiI6ICJwYXlsb2FkLmlubGluZWRKc29uQm9keS4mIgogICAgICAgICAgfQogICAgICAgIH0KICAgICAgfQogICAgfQogIH0sIAogewogICAic2NyaXB0IjogewogICAgICJvcGVyYXRpb24iOiAibW9kaWZ5LW92ZXJ3cml0ZS1iZXRhIiwKICAgICAic3BlYyI6IHsKICAgICAgICJVUkkiOiAiPXNwbGl0KCcvZXh0cmFUaGluZ1RvUmVtb3ZlJyxAKDEsJikpIgogICAgIH0KICB9CiB9LAogewogICAic2NyaXB0IjogewogICAgICJvcGVyYXRpb24iOiAibW9kaWZ5LW92ZXJ3cml0ZS1iZXRhIiwKICAgICAic3BlYyI6IHsKICAgICAgICJVUkkiOiAiPWpvaW4oJycsQCgxLCYpKSIKICAgICB9CiAgfQogfQpdCn1dCg==" + command: /bin/sh -c "/runJavaWithClasspath.sh org.opensearch.migrations.replay.TrafficReplayer --speedup-factor 2 https://opensearchtarget:9200 --auth-header-value Basic\\ YWRtaW46bXlTdHJvbmdQYXNzd29yZDEyMyE= --insecure --kafka-traffic-brokers kafka:9092 --kafka-traffic-topic logging-traffic-topic --kafka-traffic-group-id logging-group-default --otelCollectorEndpoint http://otel-collector:4317" #--transformer-config-base64 W3sgIkpzb25Kb2x0VHJhbnNmb3JtZXJQcm92aWRlciI6ClsKICB7CiAgICAic2NyaXB0IjogewogICAgICAib3BlcmF0aW9uIjogInNoaWZ0IiwKICAgICAgInNwZWMiOiB7CiAgICAgICAgIm1ldGhvZCI6ICJtZXRob2QiLAogICAgICAgICJVUkkiOiAiVVJJIiwKICAgICAgICAiaGVhZGVycyI6ICJoZWFkZXJzIiwKICAgICAgICAicGF5bG9hZCI6IHsKICAgICAgICAgICJpbmxpbmVkSnNvbkJvZHkiOiB7CiAgICAgICAgICAgICJ0b3AiOiB7CiAgICAgICAgICAgICAgInRhZ1RvRXhjaXNlIjogewogICAgICAgICAgICAgICAgIioiOiAicGF5bG9hZC5pbmxpbmVkSnNvbkJvZHkudG9wLiYiIAogICAgICAgICAgICAgIH0sCiAgICAgICAgICAgICAgIioiOiAicGF5bG9hZC5pbmxpbmVkSnNvbkJvZHkudG9wLiYiCiAgICAgICAgICAgIH0sCiAgICAgICAgICAiKiI6ICJwYXlsb2FkLmlubGluZWRKc29uQm9keS4mIgogICAgICAgICAgfQogICAgICAgIH0KICAgICAgfQogICAgfQogIH0sIAogewogICAic2NyaXB0IjogewogICAgICJvcGVyYXRpb24iOiAibW9kaWZ5LW92ZXJ3cml0ZS1iZXRhIiwKICAgICAic3BlYyI6IHsKICAgICAgICJVUkkiOiAiPXNwbGl0KCcvZXh0cmFUaGluZ1RvUmVtb3ZlJyxAKDEsJikpIgogICAgIH0KICB9CiB9LAogewogICAic2NyaXB0IjogewogICAgICJvcGVyYXRpb24iOiAibW9kaWZ5LW92ZXJ3cml0ZS1iZXRhIiwKICAgICAic3BlYyI6IHsKICAgICAgICJVUkkiOiAiPWpvaW4oJycsQCgxLCYpKSIKICAgICB9CiAgfQogfQpdCn1dCg==" opensearchtarget: image: 'opensearchproject/opensearch:2.15.0' diff --git a/TrafficCapture/trafficReplayer/README.md b/TrafficCapture/trafficReplayer/README.md index 0b74e2425..fa7834c99 100644 --- a/TrafficCapture/trafficReplayer/README.md +++ b/TrafficCapture/trafficReplayer/README.md @@ -80,12 +80,20 @@ Transformations are performed via a simple interface defined by [IJsonTransformer](../transformationPlugins/jsonMessageTransformers/jsonMessageTransformerInterface/src/main/java/org/opensearch/migrations/transform/IJsonTransformer.java) ('transformer'). They are loaded dynamically and are designed to allow for easy extension of the TrafficReplayer to support a diverse set of needs. -The input to the transformer will be an HTTP message represented as a json-like `Map` with +The input to the transformer is an HTTP message represented as a json-like `Map` with top-level key-value pairs defined in [JsonKeysForHttpMessage.java](../transformationPlugins/jsonMessageTransformers/jsonMessageTransformerInterface/src/main/java/org/opensearch/migrations/transform/JsonKeysForHttpMessage.java). -Only bodies that are json-formatted will be accessible, and they will be accessible as a fully-parsed Map (at -the keypath `'payload'->'inlinedJsonBody'`). Transformers have the option to rewrite none, or any of the keys and -values within the original message. The transformer can return either the original message or a completely new message. +Bodies that are json-formatted will be accessible via the path `payload.inlinedJsonBody` and they will be accessible +as a fully-parsed Map. Newline-delimited json (ndjson) sequences will be accessible via +`payload.inlinedJsonSequenceBodies` as a List of json Maps. These two payload entries are mutually exclusive. +Any additional bytes that follow a json object (or all of the bytes if there wasn't a json object at all) will +be available as a ByteBuf in `payload.inlinedBinaryBody`. + +Transformers have the option to rewrite none, or any of the keys and values within the original message. +The transformer can return either the original message or a completely new message. Notice that one json payload +could be broken into multiple ndjson entries or vice-versa by changing the payload key and supplying an appropriately +typed object as its value (e.g. a single Map or a List of Maps respectively for `inlinedJsonBody` and +`inlinedJsonSequenceBodies`). Transformers may be used simultaneously from concurrent threads over the lifetime of the replayer. However, a message will only be processed by one transformer at a time. @@ -108,10 +116,10 @@ The name is defined by the `IJsonTransformerProvider::getName()`, which unless o (e.g. 'JsonJoltTransformerProvider'). The value corresponding to that key is then passed to instantiate an IJsonTransformer object. -The base [jsonJoltMessageTransformerProvider](../transformationPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider) -package includes [JsonCompositeTransformer.java] +The jsonMessageTransformerInterface package includes [JsonCompositeTransformer.java] (../transformationPlugins/jsonMessageTransformers/jsonMessageTransformerInterface/src/main/java/org/opensearch/migrations/transform/JsonCompositeTransformer.java), -which run transformers in serial. That composite transformer is also utilized by the TrafficReplayer to combine the +which runs configured transformers in serial. +That composite transformer is also utilized by the TrafficReplayer to combine the list of loaded transformations with a transformer to rewrite the 'Host' header. That host transformation changes the host header of every HTTP message to use the target domain-name rather than the source's. That will be run after all loaded/specified transformations. @@ -140,8 +148,9 @@ To run only one transformer without any configuration, the `--transformer-config be set to the name of the transformer (e.g. 'JsonTransformerForOpenSearch23PlusTargetTransformerProvider', without quotes or any json surrounding it). -The user can also specify a file to read the transformations from using the `--transformer-config-file`, but can't use -both transformer options. +The user can also specify a file to read the transformations from using the `--transformer-config-file`. Users can +also pass the script as an argument via `--transformer-config-base64`. Each of the `transformer-config` options +is mutually exclusive. Some simple transformations are included to change headers to add compression or to force an HTTP message payload to be chunked. Another transformer, [JsonTypeMappingTransformer.java](../transformationPlugins/jsonMessageTransformers/openSearch23PlusTargetTransformerProvider/src/main/java/org/opensearch/migrations/transform/JsonTypeMappingTransformer.java), diff --git a/TrafficCapture/transformationPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/java/org/opensearch/migrations/replay/MultipleJoltScriptsTest.java b/TrafficCapture/transformationPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/java/org/opensearch/migrations/replay/MultipleJoltScriptsTest.java index c41e86363..5477903c5 100644 --- a/TrafficCapture/transformationPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/java/org/opensearch/migrations/replay/MultipleJoltScriptsTest.java +++ b/TrafficCapture/transformationPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/java/org/opensearch/migrations/replay/MultipleJoltScriptsTest.java @@ -58,15 +58,13 @@ public void testAddGzipAndCustom() throws Exception { @Test public void testExciseWhenPresent() throws Exception { - var script = "[{ \"JsonJoltTransformerProvider\":\n" + + var script = + "[{ \"JsonJoltTransformerProvider\":\n" + "[\n" + " {\n" + " \"script\": {\n" + " \"operation\": \"shift\",\n" + " \"spec\": {\n" + - " \"method\": \"method\",\n" + - " \"URI\": \"URI\",\n" + - " \"headers\": \"headers\",\n" + " \"payload\": {\n" + " \"inlinedJsonBody\": {\n" + " \"top\": {\n" + @@ -75,9 +73,11 @@ public void testExciseWhenPresent() throws Exception { " },\n" + " \"*\": \"payload.inlinedJsonBody.top.&\"\n" + " },\n" + - " \"*\": \"payload.inlinedJsonBody.&\"\n" + - " }\n" + - " }\n" + + " \"*\": \"payload.inlinedJsonBody.&\"\n" + + " },\n" + + " \"*\": \"payload.&\"\n" + + " },\n" + + " \"*\": \"&\"\n" + " }\n" + " }\n" + " }, \n" + @@ -108,6 +108,7 @@ public void testExciseWhenPresent() throws Exception { ); var origDocStr = "{\n" + " \"method\": \"PUT\",\n" + + " \"protocol\": \"HTTP/1.0\",\n" + " \"URI\": \"/oldStyleIndex/extraThingToRemove/moreStuff\",\n" + " \"headers\": {\n" + " \"host\": \"127.0.0.1\"\n" + @@ -129,7 +130,7 @@ public void testExciseWhenPresent() throws Exception { " }\n" + " }\n" + "}"; - var expectedDocStr = "{\"method\":\"PUT\",\"URI\":\"/oldStyleIndex/moreStuff\",\"headers\":{\"host\":\"testhostname\"},\"payload\":{\"inlinedJsonBody\":{\"top\":{\"properties\":{\"field1\":{\"type\":\"text\"},\"field2\":{\"type\":\"keyword\"}}}}}}"; + var expectedDocStr = "{\"method\":\"PUT\",\"protocol\":\"HTTP/1.0\",\"URI\":\"/oldStyleIndex/moreStuff\",\"headers\":{\"host\":\"testhostname\"},\"payload\":{\"inlinedJsonBody\":{\"top\":{\"properties\":{\"field1\":{\"type\":\"text\"},\"field2\":{\"type\":\"keyword\"}}}}}}"; var origDoc = parseAsMap(origDocStr); var newDoc = excisingTransformer.transformJson(origDoc); var newAsStr = mapper.writeValueAsString(newDoc); From 8c315393e791c8c5b36dd56f2f472835149161e6 Mon Sep 17 00:00:00 2001 From: Andre Kurait Date: Wed, 18 Sep 2024 15:42:32 -0500 Subject: [PATCH 06/12] Use Same Kafka Version Signed-off-by: Andre Kurait --- .../proxyserver/testcontainers/KafkaContainerTestBase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/testcontainers/KafkaContainerTestBase.java b/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/testcontainers/KafkaContainerTestBase.java index 033352b0b..62f196212 100644 --- a/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/testcontainers/KafkaContainerTestBase.java +++ b/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/testcontainers/KafkaContainerTestBase.java @@ -6,7 +6,7 @@ public class KafkaContainerTestBase extends TestContainerTestBase { private static final KafkaContainer kafka = new KafkaContainer( - DockerImageName.parse("confluentinc/cp-kafka:latest") + DockerImageName.parse("confluentinc/cp-kafka:7.5.0") ); public KafkaContainer getContainer() { From c48b6c42ddce43cc0d6adcd851d7519baf9fe845 Mon Sep 17 00:00:00 2001 From: Greg Schohn Date: Thu, 19 Sep 2024 08:00:23 -0400 Subject: [PATCH 07/12] PR Feedback + fixing one remaining transformation bug (see below). There were some refcount bugfixes for partial json streams and command line parsing. More importantly, there was a test (testMalformedPayload_andTypeMappingUri_IsPassedThrough) that sent malformed JSON in a payload. The test didn't do much and a comment admitted that we weren't likely doing the right thing. Moving a buffer to Unpooled caused the test to fail w/ a leak. Upon closer inspection, errors while parsing JSON didn't cause the payload contents to be dumped into the inlinedBinaryBody key of the payload. Now that content is captured and passed forward (plus, refCounted appropriately, though still as an Unpooled to minimize risk from transformations [see code comment]). In the case of this existing test, which has been moved to HttpJsonTransformingConsumerTest, the JsonTransformerForOpenSearch23PlusTargetTransformerProvider that the test uses throws an exception when the json payload isn't present. That was also not being handled properly. Now it is handled by marking the transformation as an error w/ the exception wrapped in a "TransformationException". In that case, the transformation status is marked as an error and the contents of the entire message will be empty. It feels more appropriate that if a transformation threw to be as conservative as possible and not put anything on the wire, in case some part of it was sensitive. Lastly, I refactored the AggregatedRawResponse into a base class for AggregatedRawResult. We use that class to accumulate transformation results. It was extremely confusing to see an HTTP response and the word response on so many of the fields. Now the word 'result' is used and the usage in various contexts makes more sense. Signed-off-by: Greg Schohn --- .../replay/AggregatedRawResponse.java | 88 ++------------- .../replay/AggregatedRawResult.java | 105 ++++++++++++++++++ .../replay/SourceTargetCaptureTuple.java | 4 +- .../migrations/replay/TrafficReplayer.java | 14 ++- .../replay/datahandlers/JsonAccumulator.java | 4 +- .../http/HttpJsonTransformingConsumer.java | 21 +++- .../http/NettyJsonBodyAccumulateHandler.java | 57 ++++++++-- .../http/NettyJsonBodyConvertHandler.java | 15 ++- .../http/NettyJsonBodySerializeHandler.java | 5 +- .../http/TransformationException.java | 11 ++ .../replay/HeaderTransformerTest.java | 52 +-------- .../replay/RequestSenderOrchestratorTest.java | 4 +- .../NettyPacketToHttpConsumerTest.java | 2 +- .../HttpJsonTransformingConsumerTest.java | 56 ++++++++++ 14 files changed, 278 insertions(+), 160 deletions(-) create mode 100644 TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/AggregatedRawResult.java create mode 100644 TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/TransformationException.java diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/AggregatedRawResponse.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/AggregatedRawResponse.java index e83d1e6b2..7cc561cc8 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/AggregatedRawResponse.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/AggregatedRawResponse.java @@ -3,38 +3,17 @@ import java.time.Duration; import java.time.Instant; import java.util.AbstractMap; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.stream.Stream; -import org.opensearch.migrations.replay.datatypes.ByteBufList; - -import io.netty.buffer.ByteBuf; -import io.netty.buffer.Unpooled; import io.netty.handler.codec.http.HttpResponse; import lombok.Getter; import lombok.extern.slf4j.Slf4j; +@Getter @Slf4j -public class AggregatedRawResponse { +public class AggregatedRawResponse extends AggregatedRawResult { - @Getter protected final HttpResponse rawResponse; - @Getter - protected final int responseSizeInBytes; - @Getter - protected final Duration responseDuration; - protected final ArrayList> responsePackets; - @Getter - protected final Throwable error; - - public static Builder builder(Instant i) { - return new Builder(i); - } public AggregatedRawResponse( HttpResponse rawResponse, @@ -43,82 +22,35 @@ public AggregatedRawResponse( List> responsePackets, Throwable error ) { + super(responseSizeInBytes, responseDuration, responsePackets, error); this.rawResponse = rawResponse; - this.responseSizeInBytes = responseSizeInBytes; - this.responseDuration = responseDuration; - this.responsePackets = responsePackets == null ? null : new ArrayList<>(responsePackets); - this.error = error; } - public byte[][] getCopyOfPackets() { - return responsePackets.stream() - .map(Map.Entry::getValue) - .map(x -> Arrays.copyOf(x, x.length)) - .toArray(byte[][]::new); - } - - public ByteBuf getResponseAsByteBuf() { - return responsePackets == null ? Unpooled.EMPTY_BUFFER : - ByteBufList.asCompositeByteBufRetained(responsePackets.stream() - .map(Map.Entry::getValue).map(Unpooled::wrappedBuffer)) - .asReadOnly(); - } - public static class Builder { - private final ArrayList> receiptTimeAndResponsePackets; - private final Instant requestSendTime; + public static class Builder extends AggregatedRawResult.Builder { protected HttpResponse rawResponse; - protected Throwable error; public Builder(Instant requestSendTime) { - receiptTimeAndResponsePackets = new ArrayList<>(); - this.requestSendTime = requestSendTime; - rawResponse = null; + super(requestSendTime); } public AggregatedRawResponse build() { - var totalBytes = receiptTimeAndResponsePackets.stream().mapToInt(kvp -> kvp.getValue().length).sum(); return new AggregatedRawResponse( rawResponse, - totalBytes, - Duration.between(requestSendTime, Instant.now()), + getTotalBytes(), + Duration.between(startTime, Instant.now()), receiptTimeAndResponsePackets, error ); } - public AggregatedRawResponse.Builder addResponsePacket(byte[] packet) { - return addResponsePacket(packet, Instant.now()); - } - - public AggregatedRawResponse.Builder addHttpParsedResponseObject(HttpResponse r) { + public Builder addHttpParsedResponseObject(HttpResponse r) { this.rawResponse = r; return this; } - - public AggregatedRawResponse.Builder addErrorCause(Throwable t) { - error = t; - return this; - } - - public AggregatedRawResponse.Builder addResponsePacket(byte[] packet, Instant timestamp) { - receiptTimeAndResponsePackets.add(new AbstractMap.SimpleEntry<>(timestamp, packet)); - return this; - } - } - - Stream> getReceiptTimeAndResponsePackets() { - return Optional.ofNullable(this.responsePackets).stream().flatMap(Collection::stream); } - @Override - public String toString() { - final StringBuilder sb = new StringBuilder("IResponseSummary{"); - sb.append("responseSizeInBytes=").append(responseSizeInBytes); - sb.append(", responseDuration=").append(responseDuration); - sb.append(", # of responsePackets=") - .append((this.responsePackets == null ? "-1" : "" + this.responsePackets.size())); - sb.append('}'); - return sb.toString(); + public static Builder builder(Instant i) { + return new Builder(i); } } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/AggregatedRawResult.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/AggregatedRawResult.java new file mode 100644 index 000000000..a62cd21d0 --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/AggregatedRawResult.java @@ -0,0 +1,105 @@ +package org.opensearch.migrations.replay; + +import java.time.Duration; +import java.time.Instant; +import java.util.AbstractMap; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +import org.opensearch.migrations.replay.datatypes.ByteBufList; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import lombok.Getter; + +public class AggregatedRawResult { + @Getter + protected final int sizeInBytes; + @Getter + protected final Duration duration; + protected final ArrayList> packets; + @Getter + protected final Throwable error; + + public static class Builder> { + protected final ArrayList> receiptTimeAndResponsePackets; + protected final Instant startTime; + protected Throwable error; + + public Builder(Instant startTime) { + receiptTimeAndResponsePackets = new ArrayList<>(); + this.startTime = startTime; + } + + public AggregatedRawResult build() { + var totalBytes = getTotalBytes(); + return new AggregatedRawResult( + totalBytes, + Duration.between(startTime, Instant.now()), + receiptTimeAndResponsePackets, + error + ); + } + + protected int getTotalBytes() { + return receiptTimeAndResponsePackets.stream().mapToInt(kvp -> kvp.getValue().length).sum(); + } + + public B addErrorCause(Throwable t) { + error = t; + return (B) this; + } + + public B addResponsePacket(byte[] packet) { + return (B) addResponsePacket(packet, Instant.now()); + } + + public B addResponsePacket(byte[] packet, Instant timestamp) { + receiptTimeAndResponsePackets.add(new AbstractMap.SimpleEntry<>(timestamp, packet)); + return (B) this; + } + } + + public AggregatedRawResult(int sizeInBytes, + Duration duration, + List> packets, + Throwable error) + { + this.sizeInBytes = sizeInBytes; + this.duration = duration; + this.packets = packets == null ? null : new ArrayList<>(packets); + this.error = error; + } + + public static Builder builder(Instant i) { + return new Builder<>(i); + } + + public byte[][] getCopyOfPackets() { + return packets.stream() + .map(Map.Entry::getValue) + .map(x -> Arrays.copyOf(x, x.length)) + .toArray(byte[][]::new); + } + + public ByteBuf getResponseAsByteBuf() { + return packets == null ? Unpooled.EMPTY_BUFFER : + ByteBufList.asCompositeByteBufRetained(packets.stream() + .map(Map.Entry::getValue).map(Unpooled::wrappedBuffer)) + .asReadOnly(); + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder("IResponseSummary{"); + sb.append("responseSizeInBytes=").append(sizeInBytes); + sb.append(", responseDuration=").append(duration); + sb.append(", # of responsePackets=") + .append((this.packets == null ? "-1" : "" + this.packets.size())); + sb.append('}'); + return sb.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 0e0b0e378..5e3f0f9ff 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 @@ -70,8 +70,8 @@ public SourceTargetCaptureTuple( transformedTargetRequestAndResponseList.getTransformationStatus(); this.responseList = transformedTargetRequestAndResponseList == null ? List.of() : transformedTargetRequestAndResponseList.responses().stream() - .map(arr -> new Response(arr.responsePackets.stream().map(AbstractMap.SimpleEntry::getValue) - .collect(Collectors.toList()), arr.error, arr.responseDuration)) + .map(arr -> new Response(arr.packets.stream().map(AbstractMap.SimpleEntry::getValue) + .collect(Collectors.toList()), arr.error, arr.duration)) .collect(Collectors.toList()); this.topLevelErrorCause = topLevelErrorCause; } 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 ea44b0646..084aa2801 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 @@ -270,13 +270,15 @@ private static Parameters parseArgs(String[] args) { } } + private static int isConfigured(String s) { + return (s == null || s.isBlank()) ? 0 : 1; + } + private static String getTransformerConfig(Parameters params) { - if (params.transformerConfigFile != null - && !params.transformerConfigFile.isBlank() - && params.transformerConfigEncoded != null - && !params.transformerConfigEncoded.isBlank() - && params.transformerConfig != null - && !params.transformerConfig.isBlank()) { + var configuredCount = isConfigured(params.transformerConfigFile) + + isConfigured(params.transformerConfigEncoded) + + isConfigured(params.transformerConfig); + if (configuredCount > 1) { System.err.println("Specify only one of --transformer-config-base64, --transformer-config or " + "--transformer-config-file."); System.exit(4); diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/JsonAccumulator.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/JsonAccumulator.java index 72c3fce4e..f78c0b555 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/JsonAccumulator.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/JsonAccumulator.java @@ -62,7 +62,7 @@ public Object consumeByteBufferForSingleObject(ByteBuffer byteBuffer) throws IOE } public void consumeByteBuffer(ByteBuffer byteBuffer) throws IOException { - log.trace("Consuming bytes: " + byteBuffer.toString()); + log.atTrace().setMessage(() -> "Consuming bytes: {}").addArgument(() -> byteBuffer.toString()).log(); feeder.feedInput(byteBuffer); } @@ -74,7 +74,7 @@ public Object getNextTopLevelObject() throws IOException { break; } - log.trace(this + " ... adding token=" + token); + log.atTrace().setMessage(() -> "{} ... adding token={}").addArgument(this).addArgument(token).log(); switch (token) { case FIELD_NAME: jsonObjectStack.push(parser.getText()); 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 56b326a68..81424e89a 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 @@ -47,6 +47,7 @@ public class HttpJsonTransformingConsumer implements IPacketFinalizingConsume private final RequestPipelineOrchestrator pipelineOrchestrator; private final EmbeddedChannel channel; private IReplayContexts.IRequestTransformationContext transformationContext; + private Exception lastConsumeException; /** * Roughly try to keep track of how big each data chunk was that came into the transformer. These values @@ -82,10 +83,8 @@ public HttpJsonTransformingConsumer( private NettySendByteBufsToPacketHandlerHandler getOffloadingHandler() { return Optional.ofNullable(channel) - .map( - c -> (NettySendByteBufsToPacketHandlerHandler) c.pipeline() - .get(RequestPipelineOrchestrator.OFFLOADING_HANDLER_NAME) - ) + .map(c -> (NettySendByteBufsToPacketHandlerHandler) + c.pipeline().get(RequestPipelineOrchestrator.OFFLOADING_HANDLER_NAME)) .orElse(null); } @@ -112,13 +111,19 @@ public TrackedFuture consumeBytes(ByteBuf nextRequestPacket) { .map( cf -> cf.thenAccept(x -> channel.writeInbound(nextRequestPacket)), () -> "HttpJsonTransformingConsumer sending bytes to its EmbeddedChannel" - ); + ) + .whenComplete((v,t) -> { + if (t instanceof Exception) { this.lastConsumeException = (Exception) t; } + }, () -> ""); } public TrackedFuture> finalizeRequest() { var offloadingHandler = getOffloadingHandler(); try { channel.checkException(); + if (lastConsumeException != null) { + throw lastConsumeException; + } if (getHttpRequestDecoderHandler() == null) { // LastHttpContent won't be sent channel.writeInbound(new EndOfInput()); // so send our own version of 'EOF' } @@ -179,7 +184,11 @@ private TrackedFuture> redriveWithoutTrans r -> new TransformedOutputAndResult<>(r, makeStatusForRedrive(reason)), () -> "redrive final packaging" ).whenComplete((v, t) -> { - transformationContext.onTransformSkip(); + if (t != null || (v != null && v.transformationStatus.isError())) { + transformationContext.onTransformFailure(); + } else { + transformationContext.onTransformSkip(); + } transformationContext.close(); }, () -> "HttpJsonTransformingConsumer.redriveWithoutTransformation().map()"); } 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 786ea30a2..395ca84b0 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 @@ -3,6 +3,8 @@ import java.util.ArrayList; import java.util.List; +import com.fasterxml.jackson.core.JacksonException; + import org.opensearch.migrations.replay.datahandlers.JsonAccumulator; import org.opensearch.migrations.replay.tracing.IReplayContexts; import org.opensearch.migrations.transform.JsonKeysForHttpMessage; @@ -13,7 +15,9 @@ import io.netty.channel.ChannelInboundHandlerAdapter; import io.netty.handler.codec.http.HttpContent; import io.netty.handler.codec.http.LastHttpContent; +import io.netty.util.ReferenceCountUtil; import lombok.SneakyThrows; +import lombok.extern.slf4j.Slf4j; /** * This accumulates HttpContent messages through a JsonAccumulator and eventually fires off a @@ -23,6 +27,7 @@ * This handler currently has undefined behavior if multiple json objects are within the stream of * HttpContent messages. This will also NOT fire a */ +@Slf4j public class NettyJsonBodyAccumulateHandler extends ChannelInboundHandlerAdapter { private final IReplayContexts.IRequestTransformationContext context; @@ -31,16 +36,37 @@ public class NettyJsonBodyAccumulateHandler extends ChannelInboundHandlerAdapter HttpJsonMessageWithFaultingPayload capturedHttpJsonMessage; List parsedJsonObjects; CompositeByteBuf accumulatedBody; + boolean jsonWasInvalid; @SneakyThrows public NettyJsonBodyAccumulateHandler(IReplayContexts.IRequestTransformationContext context) { this.context = context; this.jsonAccumulator = new JsonAccumulator(); + this.parsedJsonObjects = new ArrayList<>(); + } + + @Override + public void handlerAdded(ChannelHandlerContext ctx) throws Exception { // use 1024 (as opposed to the default of 16) because we really don't ever want the hit of a consolidation. // For this buffer to continue to be used, we are far-off the happy-path. // Consolidating will likely burn more cycles - this.accumulatedBody = Unpooled.compositeBuffer(1024); - this.parsedJsonObjects = new ArrayList<>(); + // + // Use Unpooled rather than the context allocator (`ctx.alloc()`) because this is the buffer that will + // be passed into a transformation if there are bytes that aren't json/ndjson formatted. + // A transformation may attempt to do manipulations or replacements of this raw ByteBuf. It may also + // throw an exception. In the interest of keeping that contract as simple as possible, just use an + // Unpooled object so that the GC can take care of this when it needs to and we won't impact the rest of + // the system. Lastly, this handler is parsing JSON - one more alloc on the GC isn't going to be + // noticeable in many cases! + accumulatedBody = Unpooled.compositeBuffer(1024); + super.handlerAdded(ctx); + } + + @Override + public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { + ReferenceCountUtil.release(accumulatedBody); + accumulatedBody = null; + super.handlerRemoved(ctx); } @Override @@ -50,11 +76,20 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception } else if (msg instanceof HttpContent) { var contentBuf = ((HttpContent) msg).content(); accumulatedBody.addComponent(true, contentBuf.retainedDuplicate()); - var nioBuf = contentBuf.nioBuffer(); - jsonAccumulator.consumeByteBuffer(nioBuf); - Object nextObj; - while ((nextObj = jsonAccumulator.getNextTopLevelObject()) != null) { - parsedJsonObjects.add(nextObj); + try { + if (!jsonWasInvalid) { + var nioBuf = contentBuf.nioBuffer(); + jsonAccumulator.consumeByteBuffer(nioBuf); + Object nextObj; + while ((nextObj = jsonAccumulator.getNextTopLevelObject()) != null) { + parsedJsonObjects.add(nextObj); + } + } + } catch (JacksonException e) { + log.atInfo().setCause(e).setMessage(() -> "Error parsing json body. " + + "Will pass all payload bytes directly as a ByteBuf within the payload map").log(); + jsonWasInvalid = true; + parsedJsonObjects.clear(); } if (msg instanceof LastHttpContent) { if (!parsedJsonObjects.isEmpty()) { @@ -74,10 +109,14 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception jsonAccumulator.getTotalBytesFullyConsumed() + " for a single composite ByteBuf"); } // skip the contents that were already parsed and included in the payload as parsed json - accumulatedBody.readerIndex((int) jsonAccumulator.getTotalBytesFullyConsumed()); // and pass the remaining stream + var jsonBodyByteLength = jsonWasInvalid ? 0 : (int) jsonAccumulator.getTotalBytesFullyConsumed(); + assert accumulatedBody.readerIndex() == 0 : + "Didn't expect the reader index to advance since this is an internal object"; capturedHttpJsonMessage.payload() - .put(JsonKeysForHttpMessage.INLINED_BINARY_BODY_DOCUMENT_KEY, accumulatedBody); + .put(JsonKeysForHttpMessage.INLINED_BINARY_BODY_DOCUMENT_KEY, + accumulatedBody.retainedSlice(jsonBodyByteLength, + accumulatedBody.readableBytes() - jsonBodyByteLength)); } else { accumulatedBody.release(); accumulatedBody = null; diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettyJsonBodyConvertHandler.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettyJsonBodyConvertHandler.java index d0838fc82..6c41b9a56 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettyJsonBodyConvertHandler.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettyJsonBodyConvertHandler.java @@ -2,10 +2,14 @@ import org.opensearch.migrations.replay.datahandlers.PayloadAccessFaultingMap; import org.opensearch.migrations.transform.IJsonTransformer; +import org.opensearch.migrations.transform.JsonKeysForHttpMessage; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelInboundHandlerAdapter; +import io.netty.util.ReferenceCountUtil; +import lombok.extern.slf4j.Slf4j; +@Slf4j public class NettyJsonBodyConvertHandler extends ChannelInboundHandlerAdapter { private final IJsonTransformer transformer; @@ -21,8 +25,15 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception // no reason for transforms to fault if there wasn't a body in the message ((PayloadAccessFaultingMap) httpMsg.payload()).setDisableThrowingPayloadNotLoaded(true); } - var output = transformer.transformJson(httpMsg); - var newHttpJson = new HttpJsonMessageWithFaultingPayload(output); + HttpJsonMessageWithFaultingPayload newHttpJson; + try { + var output = transformer.transformJson(httpMsg); + newHttpJson = new HttpJsonMessageWithFaultingPayload(output); + } catch (Exception e) { + var remainingBytes = httpMsg.payload().get(JsonKeysForHttpMessage.INLINED_BINARY_BODY_DOCUMENT_KEY); + ReferenceCountUtil.release(remainingBytes); // release because we're not passing it along for cleanup + throw new TransformationException(e); + } ctx.fireChannelRead(newHttpJson); } else { super.channelRead(ctx, msg); diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettyJsonBodySerializeHandler.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettyJsonBodySerializeHandler.java index 45cb7c52b..9f9b5ed91 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettyJsonBodySerializeHandler.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettyJsonBodySerializeHandler.java @@ -12,6 +12,7 @@ import io.netty.channel.ChannelInboundHandlerAdapter; import io.netty.handler.codec.http.DefaultHttpContent; import io.netty.handler.codec.http.LastHttpContent; +import io.netty.util.ReferenceCountUtil; import lombok.extern.slf4j.Slf4j; @Slf4j @@ -36,6 +37,8 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception var rawBody = (ByteBuf) payload.get(JsonKeysForHttpMessage.INLINED_BINARY_BODY_DOCUMENT_KEY); if (rawBody.readableBytes() > 0) { ctx.fireChannelRead(new DefaultHttpContent(rawBody)); + } else { + ReferenceCountUtil.release(rawBody); } } ctx.fireChannelRead(LastHttpContent.EMPTY_LAST_CONTENT); @@ -62,7 +65,7 @@ private void serializePayloadList(ChannelHandlerContext ctx, List payloa pac = pac.nextSupplier.get(); } if (addLastNewline || it.hasNext()) { - ctx.fireChannelRead(new DefaultHttpContent(NEWLINE.duplicate())); + ctx.fireChannelRead(new DefaultHttpContent(NEWLINE.retainedDuplicate())); } } } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/TransformationException.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/TransformationException.java new file mode 100644 index 000000000..4ce7c91ac --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/TransformationException.java @@ -0,0 +1,11 @@ +package org.opensearch.migrations.replay.datahandlers.http; + +public class TransformationException extends RuntimeException { + public TransformationException(Throwable cause) { + super(cause); + } + + public TransformationException(String message, Throwable cause) { + super(message, cause); + } +} 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 074633cfb..fdd69645f 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 @@ -23,7 +23,7 @@ import lombok.extern.slf4j.Slf4j; @Slf4j -@WrapWithNettyLeakDetection +@WrapWithNettyLeakDetection(repetitions = 2) public class HeaderTransformerTest extends InstrumentationTest { private static final String SILLY_TARGET_CLUSTER_NAME = "remoteguest"; @@ -128,54 +128,4 @@ public void testMalformedPayloadIsPassedThrough() throws Exception { + "authorization: Basic YWRtaW46YWRtaW4=\r\n" ); } - - /** - * Fixing this one will involve some thought. Where should we unwind to? I would say probably all - * the way back to the HttpTransformer. - * @throws Exception - */ - @Test - public void testMalformedPayload_andTypeMappingUri_IsPassedThrough() throws Exception { - var referenceStringBuilder = new StringBuilder(); - // mock object. values don't matter at all - not what we're testing - final var dummyAggregatedResponse = new AggregatedRawResponse(null, 12, Duration.ZERO, List.of(), null); - var testPacketCapture = new TestCapturePacketToHttpHandler(Duration.ofMillis(100), dummyAggregatedResponse); - - var transformingHandler = new HttpJsonTransformingConsumer<>( - new TransformationLoader().getTransformerFactoryLoader( - SILLY_TARGET_CLUSTER_NAME, - null, - "[{\"JsonTransformerForOpenSearch23PlusTargetTransformerProvider\":\"\"}]" - ), - 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) - .collect(Collectors.toList()); - - TrackedFuture allConsumesFuture = TestUtils.chainedDualWriteHeaderAndPayloadParts( - transformingHandler, - stringParts, - referenceStringBuilder, - contentLength -> "PUT /foo HTTP/1.1\r\n" - + "HoSt: " - + SOURCE_CLUSTER_NAME - + "\r\n" - + "content-type: application/json\r\n" - + "content-length: " - + contentLength - + "\r\n" - ); - - var finalizationFuture = allConsumesFuture.thenCompose( - v -> transformingHandler.finalizeRequest(), - () -> "HeaderTransformTest.testMalformedPayload_andTypeMappingUri_IsPassedThrough" - ); - Assertions.assertThrows(Exception.class, () -> finalizationFuture.get()); - } } 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 3819dec65..ad82de81e 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 @@ -258,8 +258,8 @@ public void testThatSchedulingWorks() throws Exception { var cf = scheduledItems.get(i); var arr = cf.get(); Assertions.assertNull(arr.error); - Assertions.assertTrue(arr.responseSizeInBytes > 0); - var packetBytesArr = arr.responsePackets.stream() + Assertions.assertTrue(arr.sizeInBytes > 0); + var packetBytesArr = arr.packets.stream() .map(SimpleEntry::getValue) .collect(Collectors.toList()); try ( 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 352fc23d1..a2bac2b18 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 @@ -260,7 +260,7 @@ private void testPeerResets( var br = new BufferedReader(isr) ) { Assertions.assertEquals("", Optional.ofNullable(br.readLine()).orElse("")); - Assertions.assertEquals(0, result.getResponseSizeInBytes()); + Assertions.assertEquals(0, result.getSizeInBytes()); } if (withServerReadTimeout) { log.trace( 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 1a33f65cd..ea1fcb28f 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 @@ -8,6 +8,8 @@ import java.util.List; import java.util.Map; import java.util.Random; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import java.util.stream.Stream; import org.junit.jupiter.api.Assertions; @@ -19,8 +21,10 @@ import org.opensearch.migrations.replay.AggregatedRawResponse; import org.opensearch.migrations.replay.TestCapturePacketToHttpHandler; +import org.opensearch.migrations.replay.TestUtils; import org.opensearch.migrations.replay.TransformationLoader; import org.opensearch.migrations.replay.datatypes.HttpRequestTransformationStatus; +import org.opensearch.migrations.replay.util.TrackedFuture; import org.opensearch.migrations.testutils.WrapWithNettyLeakDetection; import org.opensearch.migrations.tracing.InstrumentationTest; import org.opensearch.migrations.transform.IJsonTransformer; @@ -262,6 +266,58 @@ public void testPartialNewlineDelimitedJsonBodyIsHandled() throws Exception { Assertions.assertNull(returnedResponse.transformationStatus.getException()); } + @Test + public void testMalformedPayload_andThrowingTransformation_IsPassedThrough() throws Exception { + final String HOST_NAME = "foo.example"; + var referenceStringBuilder = new StringBuilder(); + // mock object. values don't matter at all - not what we're testing + final var dummyAggregatedResponse = new AggregatedRawResponse(null, 12, Duration.ZERO, List.of(), null); + var testPacketCapture = new TestCapturePacketToHttpHandler(Duration.ofMillis(100), dummyAggregatedResponse); + + var transformingHandler = new HttpJsonTransformingConsumer<>( + new TransformationLoader().getTransformerFactoryLoader( + HOST_NAME, + null, + "[{\"JsonTransformerForOpenSearch23PlusTargetTransformerProvider\":\"\"}]" + ), + 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) + .collect(Collectors.toList()); + + TrackedFuture allConsumesFuture = TestUtils.chainedDualWriteHeaderAndPayloadParts( + transformingHandler, + stringParts, + referenceStringBuilder, + contentLength -> "PUT /foo HTTP/1.1\r\n" + + "HoSt: " + HOST_NAME + "\r\n" + + "content-type: application/json\r\n" + + "content-length: " + + contentLength + + "\r\n" + ); + + var finalizationFuture = allConsumesFuture.getDeferredFutureThroughHandle( + (v,t) -> transformingHandler.finalizeRequest(), + () -> "HeaderTransformTest.testMalformedPayload_andTypeMappingUri_IsPassedThrough" + ); + var outputAndResult = finalizationFuture.get(); + Assertions.assertInstanceOf(TransformationException.class, + TrackedFuture.unwindPossibleCompletionException(outputAndResult.transformationStatus.getException()), + "It's acceptable for now that the OpenSearch upgrade transformation can't handle non-json " + + "content. If that Transform wants to handle this on its own, we'll need to use another transform " + + "configuration so that it throws and we can do this test."); + var combinedOutputBuf = outputAndResult.transformedOutput.getResponseAsByteBuf(); + Assertions.assertTrue(combinedOutputBuf.readableBytes() == 0); + combinedOutputBuf.release(); + } + public static List sliceRandomChunks(byte[] bytes, int numChunks) { Random random = new Random(0); List chunkSizes = new ArrayList<>(numChunks); From cd54b1a41429b134a199794929347b273c9f2554 Mon Sep 17 00:00:00 2001 From: Andre Kurait Date: Thu, 19 Sep 2024 10:42:14 -0500 Subject: [PATCH 08/12] Use shared constants for test docker image names Signed-off-by: Andre Kurait --- .../testcontainers/HttpdContainerTestBase.java | 6 ++++-- .../testcontainers/KafkaContainerTestBase.java | 7 +++---- .../opensearch/migrations/PruferTreeGeneratorTest.java | 1 + .../e2etests/KafkaRestartingTrafficReplayerTest.java | 6 ++---- .../migrations/replay/http/retries/HttpRetryTest.java | 6 ++---- .../kafka/KafkaCommitsWorkBetweenLongPollsTest.java | 6 ++---- .../migrations/replay/kafka/KafkaKeepAliveTests.java | 6 ++---- .../kafka/KafkaTrafficCaptureSourceLongTermTest.java | 6 ++---- .../migrations/testutils/SharedDockerImageNames.java | 9 +++++++++ 9 files changed, 27 insertions(+), 26 deletions(-) create mode 100644 testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/testutils/SharedDockerImageNames.java diff --git a/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/testcontainers/HttpdContainerTestBase.java b/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/testcontainers/HttpdContainerTestBase.java index 4e60dd53a..47e61b3c7 100644 --- a/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/testcontainers/HttpdContainerTestBase.java +++ b/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/testcontainers/HttpdContainerTestBase.java @@ -1,11 +1,13 @@ package org.opensearch.migrations.trafficcapture.proxyserver.testcontainers; +import org.opensearch.migrations.testutils.SharedDockerImageNames; + import org.testcontainers.containers.GenericContainer; public class HttpdContainerTestBase extends TestContainerTestBase> { - private static final GenericContainer httpd = new GenericContainer("httpd:alpine").withExposedPorts(80); // Container - // Port + private static final GenericContainer httpd = new GenericContainer(SharedDockerImageNames.HTTPD) + .withExposedPorts(80); // Container Port public GenericContainer getContainer() { return httpd; diff --git a/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/testcontainers/KafkaContainerTestBase.java b/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/testcontainers/KafkaContainerTestBase.java index 62f196212..862fc9f2c 100644 --- a/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/testcontainers/KafkaContainerTestBase.java +++ b/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/testcontainers/KafkaContainerTestBase.java @@ -1,13 +1,12 @@ package org.opensearch.migrations.trafficcapture.proxyserver.testcontainers; +import org.opensearch.migrations.testutils.SharedDockerImageNames; + import org.testcontainers.containers.KafkaContainer; -import org.testcontainers.utility.DockerImageName; public class KafkaContainerTestBase extends TestContainerTestBase { - private static final KafkaContainer kafka = new KafkaContainer( - DockerImageName.parse("confluentinc/cp-kafka:7.5.0") - ); + private static final KafkaContainer kafka = new KafkaContainer(SharedDockerImageNames.KAFKA); public KafkaContainer getContainer() { return kafka; diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/PruferTreeGeneratorTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/PruferTreeGeneratorTest.java index aeedb6625..65ed5325e 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/PruferTreeGeneratorTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/PruferTreeGeneratorTest.java @@ -12,6 +12,7 @@ import org.opensearch.migrations.testutils.WrapWithNettyLeakDetection; + @WrapWithNettyLeakDetection(disableLeakChecks = true) public class PruferTreeGeneratorTest { diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/KafkaRestartingTrafficReplayerTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/KafkaRestartingTrafficReplayerTest.java index 77ad252b0..659f55feb 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/KafkaRestartingTrafficReplayerTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/KafkaRestartingTrafficReplayerTest.java @@ -30,6 +30,7 @@ import org.opensearch.migrations.replay.traffic.generator.ExhaustiveTrafficStreamGenerator; import org.opensearch.migrations.replay.traffic.source.ISimpleTrafficCaptureSource; import org.opensearch.migrations.replay.traffic.source.ITrafficStreamWithKey; +import org.opensearch.migrations.testutils.SharedDockerImageNames; import org.opensearch.migrations.testutils.SimpleNettyHttpServer; import org.opensearch.migrations.testutils.WrapWithNettyLeakDetection; import org.opensearch.migrations.tracing.InstrumentationTest; @@ -43,7 +44,6 @@ import org.testcontainers.containers.KafkaContainer; import org.testcontainers.junit.jupiter.Container; import org.testcontainers.junit.jupiter.Testcontainers; -import org.testcontainers.utility.DockerImageName; @Slf4j @Testcontainers(disabledWithoutDocker = true) @@ -65,9 +65,7 @@ public class KafkaRestartingTrafficReplayerTest extends InstrumentationTest { @Container // 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 final KafkaContainer embeddedKafkaBroker = new KafkaContainer(SharedDockerImageNames.KAFKA); private static class CounterLimitedReceiverFactory implements Supplier> { AtomicInteger nextStopPointRef = new AtomicInteger(INITIAL_STOP_REPLAYER_REQUEST_COUNT); diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/http/retries/HttpRetryTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/http/retries/HttpRetryTest.java index feff8bc01..5a48e599b 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/http/retries/HttpRetryTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/http/retries/HttpRetryTest.java @@ -25,6 +25,7 @@ import org.opensearch.migrations.replay.datatypes.TransformedOutputAndResult; import org.opensearch.migrations.replay.util.TextTrackedFuture; import org.opensearch.migrations.replay.util.TrackedFuture; +import org.opensearch.migrations.testutils.SharedDockerImageNames; import org.opensearch.migrations.testutils.SimpleHttpResponse; import org.opensearch.migrations.testutils.SimpleHttpServer; import org.opensearch.migrations.testutils.ToxiProxyWrapper; @@ -45,9 +46,6 @@ @Slf4j @WrapWithNettyLeakDetection(repetitions = 1) public class HttpRetryTest { - - public static final String HTTPD_IMAGE = "httpd:alpine"; - private ByteBufList makeRequest() { return new ByteBufList(Unpooled.wrappedBuffer(TestHttpServerContext.getRequestStringForSimpleGet("/") .getBytes(StandardCharsets.UTF_8))); @@ -215,7 +213,7 @@ public void testMalformedResponseFailuresNeverGiveUp() throws Exception { var executor = Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory("HttpRetryTest")); try (var rootContext = TestContext.withAllTracking(); var network = Network.newNetwork(); - var server = new GenericContainer<>(HTTPD_IMAGE) + var server = new GenericContainer<>(SharedDockerImageNames.HTTPD) .withNetwork(network) .withNetworkAliases(SERVERNAME_ALIAS) .waitingFor(Wait.forHttp("/").forStatusCode(200)).withStartupTimeout(Duration.ofMinutes(5)); diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaCommitsWorkBetweenLongPollsTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaCommitsWorkBetweenLongPollsTest.java index 6c3f01de3..d6f03c8b8 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaCommitsWorkBetweenLongPollsTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaCommitsWorkBetweenLongPollsTest.java @@ -13,6 +13,7 @@ import org.opensearch.migrations.replay.traffic.source.BlockingTrafficSource; import org.opensearch.migrations.replay.traffic.source.ITrafficStreamWithKey; +import org.opensearch.migrations.testutils.SharedDockerImageNames; import org.opensearch.migrations.tracing.InstrumentationTest; import lombok.Lombok; @@ -21,7 +22,6 @@ import org.testcontainers.containers.KafkaContainer; import org.testcontainers.junit.jupiter.Container; import org.testcontainers.junit.jupiter.Testcontainers; -import org.testcontainers.utility.DockerImageName; @Slf4j @Testcontainers(disabledWithoutDocker = true) @@ -33,9 +33,7 @@ public class KafkaCommitsWorkBetweenLongPollsTest extends InstrumentationTest { @Container // 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 final KafkaContainer embeddedKafkaBroker = new KafkaContainer(SharedDockerImageNames.KAFKA); @SneakyThrows private KafkaConsumer buildKafkaConsumer() { 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 17ad53141..304b02d46 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 @@ -18,6 +18,7 @@ import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; import org.opensearch.migrations.replay.traffic.source.BlockingTrafficSource; +import org.opensearch.migrations.testutils.SharedDockerImageNames; import org.opensearch.migrations.tracing.InstrumentationTest; import org.opensearch.migrations.tracing.TestContext; @@ -27,7 +28,6 @@ import org.testcontainers.containers.KafkaContainer; import org.testcontainers.junit.jupiter.Container; import org.testcontainers.junit.jupiter.Testcontainers; -import org.testcontainers.utility.DockerImageName; @Slf4j @Testcontainers(disabledWithoutDocker = true) @@ -48,9 +48,7 @@ public class KafkaKeepAliveTests extends InstrumentationTest { @Container // 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 final KafkaContainer embeddedKafkaBroker = new KafkaContainer(SharedDockerImageNames.KAFKA); private KafkaTrafficCaptureSource kafkaSource; 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 8aef4ed69..6c5647ab9 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 @@ -11,13 +11,13 @@ import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; +import org.opensearch.migrations.testutils.SharedDockerImageNames; import org.opensearch.migrations.tracing.InstrumentationTest; import lombok.extern.slf4j.Slf4j; import org.testcontainers.containers.KafkaContainer; import org.testcontainers.junit.jupiter.Container; import org.testcontainers.junit.jupiter.Testcontainers; -import org.testcontainers.utility.DockerImageName; @Slf4j @Testcontainers(disabledWithoutDocker = true) @@ -31,9 +31,7 @@ public class KafkaTrafficCaptureSourceLongTermTest extends InstrumentationTest { @Container // 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 final KafkaContainer embeddedKafkaBroker = new KafkaContainer(SharedDockerImageNames.KAFKA); @Test @Tag("isolatedTest") diff --git a/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/testutils/SharedDockerImageNames.java b/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/testutils/SharedDockerImageNames.java new file mode 100644 index 000000000..cd5fc5880 --- /dev/null +++ b/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/testutils/SharedDockerImageNames.java @@ -0,0 +1,9 @@ +package org.opensearch.migrations.testutils; + +import org.testcontainers.utility.DockerImageName; + +public interface SharedDockerImageNames { + DockerImageName KAFKA = DockerImageName.parse("confluentinc/cp-kafka:7.5.0"); + DockerImageName HTTPD = DockerImageName.parse("httpd:alpine"); + +} From 802ae21ca2e95bc2ce2b795aa32b4fa6e9ca4553 Mon Sep 17 00:00:00 2001 From: Tanner Lewis Date: Thu, 19 Sep 2024 12:48:52 -0400 Subject: [PATCH 09/12] Slim down and update CDK README (#946) * Update README for CDK * Make domain name by default unique to a stage * Update error message for stage name * Update deployment/cdk/opensearch-service-migration/README.md Signed-off-by: Tanner Lewis Co-authored-by: Peter Nied --- .../src/main/docker/otelCollector/README.md | 24 ++ .../opensearch-service-migration/README.md | 216 ++---------------- .../default-values.json | 1 - .../lib/stack-composer.ts | 12 +- 4 files changed, 56 insertions(+), 197 deletions(-) create mode 100644 TrafficCapture/dockerSolution/src/main/docker/otelCollector/README.md diff --git a/TrafficCapture/dockerSolution/src/main/docker/otelCollector/README.md b/TrafficCapture/dockerSolution/src/main/docker/otelCollector/README.md new file mode 100644 index 000000000..6fe5cf26c --- /dev/null +++ b/TrafficCapture/dockerSolution/src/main/docker/otelCollector/README.md @@ -0,0 +1,24 @@ +## Monitoring Progress via Instrumentation + +The replayer and capture proxy (if started with the `--otelCollectorEndpoint` argument) emit metrics through an +otel-collector endpoint, which is deployed within Migrations Assistant tasks as a sidecar container. The +otel-collectors will publish metrics and traces to Amazon CloudWatch and AWS X-Ray. + +Some of these metrics will show simple progress, such as bytes or records transmitted. Other records can show higher +level information, such the number of responses with status codes that match vs those that don't. To observe those, +search for `statusCodesMatch` in the CloudWatch Console. That's emitted as an attribute along with the method and +the source/target status code (rounded down to the last hundred; i.e. a status code of 201 has a 200 attribute). + +Other metrics will show latencies, the number of requests, unique connections at a time and more. Low-level and +high-level metrics are being improved and added. For the latest information, see the +[README.md](../../../../../../coreUtilities/README.md). + +Along with metrics, traces are emitted by the replayer and the proxy (when proxy is run with metrics enabled, e.g. by +launching with --otelCollectorEndpoint set to the otel-collector sidecar). Traces will include very granular data for +each connection, including how long the TCP connections are open, how long the source and target clusters took to send +a response, as well as other internal details that can explain the progress of each request. + +Notice that traces for the replayer will show connections and Kafka records open, in some cases, much longer than their +representative HTTP transactions. This is because records are considered 'active' to the replayer until they are +committed and records are only committed once _all_ previous records have also been committed. Details such as that +are defensive for when further diagnosis is necessary. diff --git a/deployment/cdk/opensearch-service-migration/README.md b/deployment/cdk/opensearch-service-migration/README.md index d65bfc437..1a1892bed 100644 --- a/deployment/cdk/opensearch-service-migration/README.md +++ b/deployment/cdk/opensearch-service-migration/README.md @@ -28,15 +28,9 @@ Java is used by the opensearch-migrations repo and Gradle, its associated build ``` More details can be found [here](../../../TrafficCapture/dockerSolution/README.md) -3- Fetch Migration Setup, in order to make use of Fetch Migration for historical data capture, a user should make any modifications necessary to the `dp_pipeline_template.yaml` file located in the same directory as this README before deploying. More information around the parameters used in the pipeline file can be found [here](https://opensearch.org/docs/latest/data-prepper/pipelines/pipelines/). +3- Configure the desired **[AWS credentials](https://docs.aws.amazon.com/cdk/v2/guide/getting_started.html#getting_started_prerequisites)**, as these will dictate the region and account used for deployment. -The existing pipeline template works for the `demo-deploy` stack without any further modifications. - -Further steps on starting Fetch Migration after deployment can be found [here](#kicking-off-fetch-migration) - -4- Configure the desired **[AWS credentials](https://docs.aws.amazon.com/cdk/v2/guide/getting_started.html#getting_started_prerequisites)**, as these will dictate the region and account used for deployment. - -5- There is a known issue where service linked roles fail to get applied when deploying certain AWS services for the first time in an account. This can be resolved by simply deploying again (for each failing role) or avoided entirely by creating the service linked role initially like seen below: +4- There is a known issue where service linked roles fail to get applied when deploying certain AWS services for the first time in an account. This can be resolved by simply deploying again (for each failing role) or avoided entirely by creating the service linked role initially like seen below: ```shell aws iam create-service-linked-role --aws-service-name opensearchservice.amazonaws.com; aws iam create-service-linked-role --aws-service-name ecs.amazonaws.com ``` @@ -60,10 +54,21 @@ cdk bootstrap --c contextId=demo-deploy Further CDK documentation [here](https://docs.aws.amazon.com/cdk/v2/guide/cli.html) ## Deploying the CDK + +### Which configuration options should I use? +Update the file named `cdk.context.json` in this directory to select migration options for metadata, historical backfill or traffic capture and replay, see details [here](https://github.com/opensearch-project/opensearch-migrations/wiki/Configuration-Options). + +### How is the CDK context used in this solution? This project uses CDK context parameters to configure deployments. These context values will dictate the composition of your stacks as well as which stacks get deployed. The full list of available configuration options for this project are listed [here](./options.md). Each option can be provided as an empty string `""` or simply not included, and in each of these 'empty' cases the option will use the project default value (if it exists) or CloudFormation's default value. +Depending on your use-case, you may choose to provide options from both the `cdk.context.json` and the CDK CLI, in which case it is important to know the precedence level for context values. The below order shows these levels with values being passed by the CDK CLI having the most importance +1. CDK CLI passed context values, e.g. --c stage=dev2 (highest precedence) +2. Created `cdk.context.json` in the same directory as this README +3. Existing `default-values.json` in the same directory as this README + +### Deploying the demo solution A set of demo context values (using the `demo-deploy` label) has been set in the `cdk.context.json` located in this directory, which can be customized or used as is for a quickstart demo solution. This demo solution can be deployed with the following command: @@ -75,7 +80,7 @@ Additionally, another context block in the `cdk.context.json` could be created w ```shell cdk deploy "*" --c contextId=uat-deploy --require-approval never --concurrency 3 ``` -**Note**: Separate deployments within the same account and region should use unique `stage` context values to avoid resource naming conflicts when deploying (**Except** in the multiple replay scenario stated [here](#how-to-run-multiple-traffic-replayer-scenarios)) +**Note**: Separate deployments within the same account and region should use unique `stage` context values to avoid resource naming conflicts when deploying (**Except** in the multiple replay scenario stated [here](#how-to-run-multiple-traffic-replayer-scenarios)) Stacks can also be redeployed individually, with any required stacks also being deployed initially, e.g. the following command would deploy the migration-console stack ```shell @@ -87,15 +92,14 @@ To get a list of all the available stack ids that can be deployed/redeployed for cdk ls --c contextId=demo-deploy ``` +## How to use the deployed Migration tools? +See the [wiki](https://github.com/opensearch-project/opensearch-migrations/wiki) for steps on how to use this tooling to perform different migrations. -Depending on your use-case, you may choose to provide options from both the `cdk.context.json` and the CDK CLI, in which case it is important to know the precedence level for context values. The below order shows these levels with values being passed by the CDK CLI having the most importance -1. CDK CLI passed context values (highest precedence) -2. Created `cdk.context.json` in the same directory as this README -3. Existing `default-values.json` in the same directory as this README +## Accessing the Migration Console -## Executing Commands on a Deployed Service +The Migration Console is a deployed ECS service container in this solution that should be accessed for managing/executing different phases of a migration -Once a service has been deployed, a command shell can be opened for that service's container. If the SSM Session Manager plugin is not installed, it should be installed when prompted from the below exec command. +To open a shell on the Migration Console container execute the below command. If the SSM Session Manager plugin is not installed, it should be installed when prompted from the below exec command. ```shell # ./accessContainer.sh ./accessContainer.sh migration-console dev us-east-1 @@ -117,177 +121,6 @@ To be able to execute this command the user will need to have their AWS credenti } ``` -## Starting the Traffic Replayer -When the Migration solution is deployed, the Traffic Replayer does not immediately begin replaying. This is designed to allow users time to do any historical backfill (e.g. Fetch Migration service) that is needed as well as setup the Capture Proxy on their source coordinating nodes. When the user is ready they can then run the following command from the Migration Console service and begin replaying the traffic that has been captured by the Capture Proxy - -```shell -aws ecs update-service --cluster migration--ecs-cluster --service migration--traffic-replayer-default --desired-count 1 -``` - -With this same command, a user could stop replaying capture traffic by removing the Traffic Replayer instance if they set `--desired-count 0` - -## Testing the deployed solution - -Once the solution is deployed, the easiest way to test the solution is to access the `migration-console` service container and run an opensearch-benchmark workload through to simulate incoming traffic, as the following steps illustrate - -```shell -# Exec into container -./accessContainer.sh migration-console dev us-east-1 - -# Run opensearch-benchmark workload (i.e. geonames, nyc_taxis, http_logs) -./runTestBenchmarks.sh -``` - -After the benchmark has been run, the indices and documents of the source and target clusters can be checked from the same migration-console container to confirm -```shell -# Check doc counts and indices for both source and target cluster -./catIndices.sh -``` - -## Importing Target Clusters -By default, if a `targetClusterEndpoint` option isn't provided, this CDK will create an OpenSearch Service Domain (using provided options) to be the target cluster of this solution. While setting up this Domain, the CDK will also configure a relevant security group and allows options to configure an access policy on the Domain (`accessPolicies` and `openAccessPolicyEnabled` options) such that the Domain is fully setup for use at deployment. - -In the case of an imported target cluster, there are normally some modifications that need to be made on the existing target cluster to allow proper functioning of this solution after deployment which the below subsections elaborate on. - -#### OpenSearch Service -For a Domain, there are typically two items that need to be configured to allow proper functioning of this solution -1. The Domain should have a security group that allows communication from the applicable Migration services (Traffic Replayer, Migration Console, Fetch Migration). This CDK will automatically create an `osClusterAccessSG` security group, which has already been applied to the Migration services, that a user should then add to their existing Domain to allow this access. -2. The access policy on the Domain should be an open access policy that allows all access or an access policy that at least allows the IAM task roles for the applicable Migration services (Traffic Replayer, Migration Console, Fetch Migration) - -#### OpenSearch Serverless -A Collection, will need to configure a Network and Data Access policy to allow proper functioning of this solution -1. The Collection should have a network policy that has a `VPC` access type by creating a VPC endpoint on the VPC used for this solution. This VPC endpoint should be configured for the private subnets of the VPC and attach the `osClusterAccessSG` security group. -2. The data access policy needed should grant permission to perform all [index operations](https://docs.aws.amazon.com/opensearch-service/latest/developerguide/serverless-data-access.html#serverless-data-supported-permissions) (`aoss:*`) for all indexes in the given collection, and use the task roles of the applicable Migration services (Traffic Replayer, Migration Console, Fetch Migration) as the principals for this data access policy. - -See [Configuring SigV4 Replayer Requests](#configuring-sigv4-replayer-requests) for details on enabling SigV4 requests from the Traffic Replayer to the target cluster - -## Configuring SigV4 Replayer Requests -With the [required setup](#importing-target-clusters) on the target cluster having been completed, a user can then use the `trafficReplayerExtraArgs` option to specify the Traffic Replayer service argument for enabling SigV4 authentication, which the below sections show. **Note**: As only one authorization header can be specified, the `trafficReplayerEnableClusterFGACAuth` option should not be used if enabling SigV4 authentication for the Traffic Replayer. See [here](#how-is-an-authorization-header-set-for-requests-from-the-replayer-to-the-target-cluster) for more details on how the Traffic Replayer sets its authorization header. -#### OpenSearch Service -```shell -# e.g. --sigv4-auth-header-service-region es,us-east-1 -"trafficReplayerExtraArgs": "--sigv4-auth-header-service-region es," -``` - -#### OpenSearch Serverless -```shell -# e.g. --sigv4-auth-header-service-region aoss,us-east-1 -"trafficReplayerExtraArgs": "--sigv4-auth-header-service-region aoss," -``` - -## Kicking off Fetch Migration - -* First, access the Migration Console container - -```shell -# ./accessContainer.sh migration-console STAGE REGION -./accessContainer.sh migration-console dev us-east-1 -``` - -* Execute the ECS run task command generated by `showFetchMigrationCommand.sh` script. - * The status of the ECS Task can be monitored from the AWS Console. Once the task is in the `Running` state, logs and progress can be viewed via CloudWatch. -```shell -# This will execute the script and print the required ECS run task command -./showFetchMigrationCommand.sh - -# Paste command output by the script into the terminal to kick off Fetch Migration -``` - -The pipeline configuration file can be viewed (and updated) via AWS Secrets Manager. -Please note that it will be base64 encoded. - -## Kicking off OpenSearch Ingestion Service - -**Note**: Using OpenSearch Ingestion Service is currently an experimental feature that must be enabled with the `migrationConsoleEnableOSI` option. Currently only Managed OpenSearch service as a source to Managed OpenSearch service as a target migrations are supported - -After enabling and deploying the CDK, log into the Migration Console -```shell -# ./accessContainer.sh migration-console STAGE REGION -./accessContainer.sh migration-console dev us-east-1 -``` -Make any modifications to the `osiPipelineTemplate.yaml` on the Migration Console, if needed. Note: Placeholder values exist in the file to automatically populate source/target endpoints and corresponding auth options by the python tool that uses this yaml file. - -The OpenSearch Ingestion pipeline can then be created by giving an existing source cluster endpoint and running the below command -```shell -./osiMigration.py create-pipeline-from-solution --source-endpoint= -``` - -When OpenSearch Ingestion pipelines are created they begin running immediately and can be stopped with the following command -```shell -./osiMigration.py stop-pipeline -``` -Or restarted with the following command -```shell -./osiMigration.py start-pipeline -``` - -## Kicking off Reindex from Snapshot (RFS) - -When the RFS service gets deployed, it does not start running immediately. Instead, the user controls when they want to kick off a historical data migration. - -The following command can be run from the Migration Console to initiate the RFS historical data migration -```shell -aws ecs update-service --cluster migration--ecs-cluster --service migration--reindex-from-snapshot --desired-count 1 -``` - -Currently, the RFS application will enter an idle state with the ECS container still running upon completion. This can be cleaned up by using the same command with `--desired-count 0` - - -## Monitoring Progress via Instrumentation - -The replayer and capture proxy (if started with the `--otelCollectorEndpoint` argument) emit metrics through an -otel-collector endpoint, which is deployed within Migrations Assistant tasks as a sidecar container. The -otel-collectors will publish metrics and traces to Amazon CloudWatch and AWS X-Ray. - -Some of these metrics will show simple progress, such as bytes or records transmitted. Other records can show higher -level information, such the number of responses with status codes that match vs those that don't. To observe those, -search for `statusCodesMatch` in the CloudWatch Console. That's emitted as an attribute along with the method and -the source/target status code (rounded down to the last hundred; i.e. a status code of 201 has a 200 attribute). - -Other metrics will show latencies, the number of requests, unique connections at a time and more. Low-level and -high-level metrics are being improved and added. For the latest information, see the -[README.md](../../../coreUtilities/README.md). - -Along with metrics, traces are emitted by the replayer and the proxy (when proxy is run with metrics enabled, e.g. by -launching with --otelCollectorEndpoint set to the otel-collector sidecar). Traces will include very granular data for -each connection, including how long the TCP connections are open, how long the source and target clusters took to send -a response, as well as other internal details that can explain the progress of each request. - -Notice that traces for the replayer will show connections and Kafka records open, in some cases, much longer than their -representative HTTP transactions. This is because records are considered 'active' to the replayer until they are -committed and records are only committed once _all_ previous records have also been committed. Details such as that -are defensive for when further diagnosis is necessary. - -## Configuring Capture Proxy IAM and Security Groups -Although this CDK does not set up the Capture Proxy on source cluster nodes (except in the case of the demo solution), the Capture Proxy instances do need to communicate with resources deployed by this CDK (e.g. Kafka) which this section covers - -#### Capture Proxy on OpenSearch/Elasticsearch nodes -Before [setting up Capture Proxy instances](../../../TrafficCapture/trafficCaptureProxyServer/README.md#how-to-attach-a-capture-proxy-on-a-coordinator-node) on the source cluster, the IAM policies and Security Groups for the nodes should allow access to the Migration tooling: -1. The coordinator nodes should add the `trafficStreamSourceSG` security group to allow access to Kafka -2. The IAM role used by the coordinator nodes should have permissions to publish captured traffic to Kafka. A template policy to use, can be seen below - * This can be added through the AWS Console (IAM Role -> Add permissions -> Create inline policy -> JSON view) -```json -{ - "Version": "2012-10-17", - "Statement": [ - { - "Action": "kafka-cluster:Connect", - "Resource": "arn:aws:kafka:::cluster/migration-msk-cluster-/*", - "Effect": "Allow" - }, - { - "Action": [ - "kafka-cluster:CreateTopic", - "kafka-cluster:DescribeTopic", - "kafka-cluster:WriteData" - ], - "Resource": "arn:aws:kafka:::topic/migration-msk-cluster-/*", - "Effect": "Allow" - } - ] -} -``` - ## Tearing down CDK To remove all the CDK stack(s) which get created during a deployment we can execute a command similar to below ```shell @@ -301,7 +134,9 @@ cdk destroy migration-console --c contextId=demo-deploy ``` **Note**: The `demo-deploy`contextId has the retention policy for the OpenSearch Domain set to `DESTROY`, which will remove this resource and all its data when the stack is deleted. In order to retain the Domain on stack deletion the `domainRemovalPolicy` would need to be set to `RETAIN`. -## How to run multiple Traffic Replayer scenarios +## Appendix + +### How to run multiple Traffic Replayer scenarios The project supports running distinct Replayers in parallel, with each Replayer sending traffic to a different target cluster. This functionality allows users to test replaying captured traffic to multiple different target clusters in parallel. Users are able to provide the desired configuration options to spin up a new OpenSearch Domain and Traffic Replayer while using the existing Migration infrastructure that has already been deployed. To give an example of this process, a user could decide to configure an additional Replayer and Domain for the demo setup in the `cdk.context.json` by configuring a new context block like below. **Note**: `addOnMigrationDeployId` is a required field to allow proper naming of these additional resources. @@ -329,19 +164,16 @@ Finally, the additional infrastructure can be removed with: cdk destroy "*" --c contextId=demo-addon1 ``` -## Appendix ### How is an Authorization header set for requests from the Replayer to the target cluster? The Replayer documentation [here](../../../TrafficCapture/trafficReplayer/README.md#authorization-header-for-replayed-requests) explains the reasoning the Replayer uses to determine what auth header it should use when replaying requests to the target cluster. -As it relates to this CDK, the two main avenues for setting an explicit auth header for the Replayer are through the `trafficReplayerEnableClusterFGACAuth` and `trafficReplayerExtraArgs` options -1. The `trafficReplayerEnableClusterFGACAuth` option will utilize the `--auth-header-user-and-secret` parameter of the Replayer service to create a basic auth header with a username and AWS Secrets Manager secret value. This option requires that a Fine Grained Access Control (FGAC) user be configured (see `fineGrainedManagerUserName` and `fineGrainedManagerUserSecretManagerKeyARN` CDK context options [here](./options.md)) or is running in demo mode (see `enableDemoAdmin` CDK context option). -2. The `trafficReplayerExtraArgs` option allows a user to directly specify the Replayer parameter they want to use for setting the auth header. For example to enable SigV4 as the auth header for an OpenSearch service in us-east-1, a user could set this option to `--sigv4-auth-header-service-region es,us-east-1` +As it relates to this CDK, the `targetCluster` configuration option (specifically the `auth` element) that a user provides will dictate which auth the Migration tools will use for communicating with the target cluster ### Common Deployment Errors -**Problem**: +**Problem**: ``` ERROR: failed to solve: public.ecr.aws/sam/build-nodejs18.x: pulling from host public.ecr.aws failed with status code [manifests latest]: 403 Forbidden ``` diff --git a/deployment/cdk/opensearch-service-migration/default-values.json b/deployment/cdk/opensearch-service-migration/default-values.json index 3d15ff04a..9991f0354 100644 --- a/deployment/cdk/opensearch-service-migration/default-values.json +++ b/deployment/cdk/opensearch-service-migration/default-values.json @@ -1,7 +1,6 @@ { "engineVersion": "OS_2.9", "targetClusterVersion": "OS_2.9", - "domainName": "os-service-domain", "tlsSecurityPolicy": "TLS_1_2", "enforceHTTPS": true, "nodeToNodeEncryptionEnabled": true, diff --git a/deployment/cdk/opensearch-service-migration/lib/stack-composer.ts b/deployment/cdk/opensearch-service-migration/lib/stack-composer.ts index bb827a59e..a96b3b739 100644 --- a/deployment/cdk/opensearch-service-migration/lib/stack-composer.ts +++ b/deployment/cdk/opensearch-service-migration/lib/stack-composer.ts @@ -214,7 +214,7 @@ export class StackComposer { const sourceClusterDisabledField = this.getContextForType('sourceClusterDisabled', 'boolean', defaultValues, contextJSON) const sourceClusterEndpointField = this.getContextForType('sourceClusterEndpoint', 'string', defaultValues, contextJSON) let sourceClusterDefinition = this.getContextForType('sourceCluster', 'object', defaultValues, contextJSON) - + if (!sourceClusterDefinition && (sourceClusterEndpointField || sourceClusterDisabledField)) { console.warn("`sourceClusterDisabled` and `sourceClusterEndpoint` are being deprecated in favor of a `sourceCluster` object.") console.warn("Please update your CDK context block to use the `sourceCluster` object.") @@ -261,11 +261,11 @@ export class StackComposer { "and in this case, `targetCluster` was provided to define an existing target cluster." ) } - + const targetClusterAuth = targetCluster?.auth const targetVersion = this.getEngineVersion(targetCluster?.version || engineVersion) - const requiredFields: { [key: string]: any; } = {"stage":stage, "domainName":domainName} + const requiredFields: { [key: string]: any; } = {"stage":stage} for (let key in requiredFields) { if (!requiredFields[key]) { throw new Error(`Required CDK context field ${key} is not present`) @@ -274,6 +274,10 @@ export class StackComposer { if (addOnMigrationDeployId && vpcId) { console.warn("Addon deployments will use the original deployment 'vpcId' regardless of passed 'vpcId' values") } + if (stage.length > 15) { + throw new Error(`Maximum allowed stage name length is 15 characters but received ${stage}`) + } + const clusterDomainName = domainName ? domainName : `os-cluster-${stage}` let preexistingOrContainerTargetEndpoint if (targetCluster && osContainerServiceEnabled) { throw new Error("The following options are mutually exclusive as only one target cluster can be specified for a given deployment: [targetCluster, osContainerServiceEnabled]") @@ -348,7 +352,7 @@ export class StackComposer { if (!preexistingOrContainerTargetEndpoint) { openSearchStack = new OpenSearchDomainStack(scope, `openSearchDomainStack-${deployId}`, { version: targetVersion, - domainName: domainName, + domainName: clusterDomainName, dataNodeInstanceType: dataNodeType, dataNodes: dataNodeCount, dedicatedManagerNodeType: dedicatedManagerNodeType, From 4803b59db02049224913c103b8e5dadea38551d1 Mon Sep 17 00:00:00 2001 From: Chris Helma <25470211+chelma@users.noreply.github.com> Date: Thu, 19 Sep 2024 12:24:50 -0500 Subject: [PATCH 10/12] More SonarQube fixes for the RFS version* code (#978) * More SonarQube fixes Signed-off-by: Chris Helma * Restored CDK package-lock file Signed-off-by: Chris Helma --------- Signed-off-by: Chris Helma --- .../GlobalMetadataCreator_OS_2_11.java | 59 +++++++++---------- .../GlobalMetadataData_OS_2_11.java | 10 ++-- .../version_os_2_11/IndexCreator_OS_2_11.java | 9 ++- .../RemoteIndexMetadata.java | 3 +- .../version_universal/RemoteReaderClient.java | 12 ++-- 5 files changed, 48 insertions(+), 45 deletions(-) diff --git a/RFS/src/main/java/com/rfs/version_os_2_11/GlobalMetadataCreator_OS_2_11.java b/RFS/src/main/java/com/rfs/version_os_2_11/GlobalMetadataCreator_OS_2_11.java index 4ffb4e0f6..b40328b2f 100644 --- a/RFS/src/main/java/com/rfs/version_os_2_11/GlobalMetadataCreator_OS_2_11.java +++ b/RFS/src/main/java/com/rfs/version_os_2_11/GlobalMetadataCreator_OS_2_11.java @@ -44,7 +44,7 @@ public List createLegacyTemplates(GlobalMetadataData_OS_2_11 metadata, M return createTemplates( metadata.getTemplates(), legacyTemplateAllowlist, - TemplateTypes.LegacyIndexTemplate, + TemplateTypes.LEGACY_INDEX_TEMPLATE, mode, context ); @@ -54,7 +54,7 @@ public List createComponentTemplates(GlobalMetadataData_OS_2_11 metadata return createTemplates( metadata.getComponentTemplates(), componentTemplateAllowlist, - TemplateTypes.ComponentTemplates, + TemplateTypes.COMPONENT_TEMPLATE, mode, context ); @@ -64,7 +64,7 @@ public List createIndexTemplates(GlobalMetadataData_OS_2_11 metadata, Mi return createTemplates( metadata.getIndexTemplates(), indexTemplateAllowlist, - TemplateTypes.IndexTemplate, + TemplateTypes.INDEX_TEMPLATE, mode, context ); @@ -72,19 +72,19 @@ public List createIndexTemplates(GlobalMetadataData_OS_2_11 metadata, Mi @AllArgsConstructor private enum TemplateTypes { - IndexTemplate( - (client, name, body, context) -> client.createIndexTemplate(name, body, context.createMigrateTemplateContext()), - (client, name) -> client.hasIndexTemplate(name) + INDEX_TEMPLATE( + (targetClient, name, body, context) -> targetClient.createIndexTemplate(name, body, context.createMigrateTemplateContext()), + (targetClient, name) -> targetClient.hasIndexTemplate(name) ), - LegacyIndexTemplate( - (client, name, body, context) -> client.createLegacyTemplate(name, body, context.createMigrateLegacyTemplateContext()), - (client, name) -> client.hasLegacyTemplate(name) + LEGACY_INDEX_TEMPLATE( + (targetClient, name, body, context) -> targetClient.createLegacyTemplate(name, body, context.createMigrateLegacyTemplateContext()), + (targetClient, name) -> targetClient.hasLegacyTemplate(name) ), - ComponentTemplates( - (client, name, body, context) -> client.createComponentTemplate(name, body, context.createComponentTemplateContext()), - (client, name) -> client.hasComponentTemplate(name) + COMPONENT_TEMPLATE( + (targetClient, name, body, context) -> targetClient.createComponentTemplate(name, body, context.createComponentTemplateContext()), + (targetClient, name) -> targetClient.hasComponentTemplate(name) ); final TemplateCreator creator; final TemplateExistsCheck alreadyExistsCheck; @@ -118,7 +118,7 @@ private List createTemplates( return List.of(); } - if (templateAllowlist != null && templateAllowlist.size() == 0) { + if (templateAllowlist != null && templateAllowlist.isEmpty()) { log.info("No {} in specified allowlist", templateType); return List.of(); } else if (templateAllowlist != null) { @@ -144,24 +144,21 @@ private List createTemplates( templatesToCreate.forEach((templateName, templateBody) -> { log.info("Creating {}: {}", templateType, templateName); - switch (mode) { - case SIMULATE: - var alreadyExists = templateType.alreadyExistsCheck.templateAlreadyExists(client, templateName); - if (!alreadyExists) { - templateList.add(templateName); - } else { - log.warn("Template {} already exists on the target, it will not be created during a migration", templateName); - } - break; - - case PERFORM: - var createdTemplate = templateType.creator.createTemplate(client, templateName, templateBody, context); - if (createdTemplate.isPresent()) { - templateList.add(templateName); - } else { - log.warn("Template {} already exists on the target, unable to create", templateName); - } - break; + + if (mode == MigrationMode.SIMULATE) { + var alreadyExists = templateType.alreadyExistsCheck.templateAlreadyExists(client, templateName); + if (!alreadyExists) { + templateList.add(templateName); + } else { + log.warn("Template {} already exists on the target, it will not be created during a migration", templateName); + } + } else if (mode == MigrationMode.PERFORM) { + var createdTemplate = templateType.creator.createTemplate(client, templateName, templateBody, context); + if (createdTemplate.isPresent()) { + templateList.add(templateName); + } else { + log.warn("Template {} already exists on the target, unable to create", templateName); + } } }); diff --git a/RFS/src/main/java/com/rfs/version_os_2_11/GlobalMetadataData_OS_2_11.java b/RFS/src/main/java/com/rfs/version_os_2_11/GlobalMetadataData_OS_2_11.java index 4b5d885a3..12ffbfb18 100644 --- a/RFS/src/main/java/com/rfs/version_os_2_11/GlobalMetadataData_OS_2_11.java +++ b/RFS/src/main/java/com/rfs/version_os_2_11/GlobalMetadataData_OS_2_11.java @@ -21,16 +21,18 @@ public ObjectNode getTemplates() { } public ObjectNode getIndexTemplates() { - if (root.get("index_template") != null) { - return (ObjectNode) root.get("index_template").get("index_template"); + String indexTemplateKey = "index_template"; + if (root.get(indexTemplateKey) != null) { + return (ObjectNode) root.get(indexTemplateKey).get(indexTemplateKey); } else { return null; } } public ObjectNode getComponentTemplates() { - if (root.get("component_template") != null) { - return (ObjectNode) root.get("component_template").get("component_template"); + String componentTemplateKey = "component_template"; + if (root.get(componentTemplateKey) != null) { + return (ObjectNode) root.get(componentTemplateKey).get(componentTemplateKey); } else { return null; } diff --git a/RFS/src/main/java/com/rfs/version_os_2_11/IndexCreator_OS_2_11.java b/RFS/src/main/java/com/rfs/version_os_2_11/IndexCreator_OS_2_11.java index 97ad3613b..d19092a40 100644 --- a/RFS/src/main/java/com/rfs/version_os_2_11/IndexCreator_OS_2_11.java +++ b/RFS/src/main/java/com/rfs/version_os_2_11/IndexCreator_OS_2_11.java @@ -42,11 +42,10 @@ public boolean create( // Create the index; it's fine if it already exists try { - switch (mode) { - case SIMULATE: - return !client.hasIndex(index.getName()); - case PERFORM: - return client.createIndex(index.getName(), body, context).isPresent(); + if (mode == MigrationMode.SIMULATE) { + return !client.hasIndex(index.getName()); + } else if (mode == MigrationMode.PERFORM) { + return client.createIndex(index.getName(), body, context).isPresent(); } } catch (InvalidResponse invalidResponse) { var illegalArguments = invalidResponse.getIllegalArguments(); diff --git a/RFS/src/main/java/com/rfs/version_universal/RemoteIndexMetadata.java b/RFS/src/main/java/com/rfs/version_universal/RemoteIndexMetadata.java index 2d818edeb..ee4ba4a15 100644 --- a/RFS/src/main/java/com/rfs/version_universal/RemoteIndexMetadata.java +++ b/RFS/src/main/java/com/rfs/version_universal/RemoteIndexMetadata.java @@ -24,7 +24,8 @@ public JsonNode getAliases() { @Override public String getId() { - return indexName; + // The ID is the name in this case + return getName(); } @Override diff --git a/RFS/src/main/java/com/rfs/version_universal/RemoteReaderClient.java b/RFS/src/main/java/com/rfs/version_universal/RemoteReaderClient.java index e781f7389..5e0c5474a 100644 --- a/RFS/src/main/java/com/rfs/version_universal/RemoteReaderClient.java +++ b/RFS/src/main/java/com/rfs/version_universal/RemoteReaderClient.java @@ -109,8 +109,7 @@ Mono getJsonForIndexApis(HttpResponse resp) { var tree = (ObjectNode) objectMapper.readTree(resp.body); return Mono.just(tree); } catch (Exception e) { - log.error("Unable to get json response: ", e); - return Mono.error(new OperationFailed("Unable to get json response: " + e.getMessage(), resp)); + return logAndReturnJsonError(e, resp); } } @@ -140,8 +139,13 @@ Mono getJsonForTemplateApis(HttpResponse resp) { } return Mono.just(tree); } catch (Exception e) { - log.error("Unable to get json response: ", e); - return Mono.error(new OperationFailed("Unable to get json response: " + e.getMessage(), resp)); + return logAndReturnJsonError(e, resp); } } + + Mono logAndReturnJsonError(Exception e, HttpResponse resp) { + String errorPrefix = "Unable to get json response: "; + log.atError().setCause(e).setMessage(errorPrefix).log(); + return Mono.error(new OperationFailed(errorPrefix + e.getMessage(), resp)); + } } From 7a72c1064a2460ed4355dd33d73400753eee1ca4 Mon Sep 17 00:00:00 2001 From: Andre Kurait Date: Thu, 19 Sep 2024 12:55:33 -0500 Subject: [PATCH 11/12] Resolve some sonar lint issues in captureOffloader (#979) Signed-off-by: Andre Kurait --- ...reamChannelConnectionCaptureSerializer.java | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) 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 35c83baf8..5e836c66e 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 @@ -124,10 +124,10 @@ private CodedOutputStreamHolder getOrCreateCodedOutputStreamHolder() throws IOEx } else { currentCodedOutputStreamHolderOrNull = streamManager.createStream(); var currentCodedOutputStream = currentCodedOutputStreamHolderOrNull.getOutputStream(); - // e.g. 1: "9a25a4fffe620014-00034cfa-00000001-d208faac76346d02-864e38e2" + // e.g.
 1: "9a25a4fffe620014-00034cfa-00000001-d208faac76346d02-864e38e2" 
currentCodedOutputStream.writeString(TrafficStream.CONNECTIONID_FIELD_NUMBER, connectionIdString); if (nodeIdString != null) { - // e.g. 5: "5ae27fca-0ac4-11ee-be56-0242ac120002" + // e.g.
 5: "5ae27fca-0ac4-11ee-be56-0242ac120002" 
currentCodedOutputStream.writeString(TrafficStream.NODEID_FIELD_NUMBER, nodeIdString); } if (eomsSoFar > 0) { @@ -213,11 +213,11 @@ private void beginSubstreamObservation( numFlushesSoFar + 1 ) ); - // e.g. 2 { + // e.g.
 2 { 
writeTrafficStreamTag(TrafficStream.SUBSTREAM_FIELD_NUMBER); // Write observation content length getOrCreateCodedOutputStream().writeUInt32NoTag(observationContentSize); - // e.g. 1 { 1: 1234 2: 1234 } + // e.g.
 1 { 1: 1234 2: 1234 } 
writeTimestampForNowToCurrentStream(timestamp); } @@ -371,7 +371,7 @@ private void addStringMessage(int captureFieldNumber, int dataFieldNumber, Insta lengthSize = CodedOutputStream.computeInt32SizeNoTag(dataSize); } beginSubstreamObservation(timestamp, captureFieldNumber, dataSize + lengthSize); - // e.g. 4 { + // e.g.
 4 { 
writeObservationTag(captureFieldNumber); if (dataSize > 0) { getOrCreateCodedOutputStream().writeInt32NoTag(dataSize); @@ -461,7 +461,7 @@ private void addSubstreamMessage( captureClosureLength = CodedOutputStream.computeInt32SizeNoTag(dataSize + segmentCountSize); } beginSubstreamObservation(timestamp, captureFieldNumber, captureClosureLength + dataSize + segmentCountSize); - // e.g. 4 { + // e.g.
 4 {  
writeObservationTag(captureFieldNumber); if (dataSize > 0) { // Write size of data after capture tag @@ -578,7 +578,7 @@ private void writeEndOfHttpMessage(Instant timestamp) throws IOException { ); int eomDataSize = eomPairSize + CodedOutputStream.computeInt32SizeNoTag(eomPairSize); beginSubstreamObservation(timestamp, TrafficObservation.ENDOFMESSAGEINDICATOR_FIELD_NUMBER, eomDataSize); - // e.g. 15 { + // e.g.
 15 { 
writeObservationTag(TrafficObservation.ENDOFMESSAGEINDICATOR_FIELD_NUMBER); getOrCreateCodedOutputStream().writeUInt32NoTag(eomPairSize); getOrCreateCodedOutputStream().writeInt32( @@ -650,6 +650,8 @@ public boolean isOpen() { } @Override - public void close() {} + public void close() { + // No resources to close + } } } From d2dce08ad2b9412195bbeb9ed23ace76373bbcb4 Mon Sep 17 00:00:00 2001 From: Tanner Lewis Date: Thu, 19 Sep 2024 15:07:03 -0400 Subject: [PATCH 12/12] Add Jenkins Webhook and RFS E2E job on PR (#938) This change is hopefully the final chapter in the Jenkins setup saga and a huge leap in the integration testing story. It adds support for triggering a Jenkins job through a configured Github Action and monitoring its completion to pass/fail the action. This design is one of many that can be taken, but a benefit of this approach is that it keeps the Github action in control of triggering Jenkins jobs and status updates, as well as doesn't require Jenkins to update the PR (though this may be a useful addition for logs or other data in the future). As a part of this pattern, a general python webhook was created to trigger Jenkins jobs and monitor for a terminal state, this could also potentially become its own GHA plugin separate from this repo. Additionally this adds our first GHA which triggers a Jenkins job for our RFS E2E integration test. This GHA is planned to be monitored as any bugs/issues are worked out and should not currently be restrictive for PRs to be merged. Jenkins pipeline files have also been updated to attain a lock on deployment stages to prevent different pipeline executions from using the same stage. The pipeline files have also been largely moved to the shared library, to allow any changes to the context options or other settings for the pipeline to be tested in PR actions. --------- Signed-off-by: Tanner Lewis Co-authored-by: Peter Nied --- .github/workflows/rfs_pr_e2e_test.yml | 40 ++++++++++ .../rfsBackfillE2EPipeline.groovy | 78 ------------------- .../rfsDefaultE2ETestCover.groovy | 9 +++ .../trafficReplayDefaultE2ETestCover.groovy | 9 +++ vars/defaultIntegPipeline.groovy | 39 ++++++++-- vars/rfsDefaultE2ETest.groovy | 57 ++++++++++++++ .../trafficReplayDefaultE2ETest.groovy | 43 +++++----- 7 files changed, 167 insertions(+), 108 deletions(-) create mode 100644 .github/workflows/rfs_pr_e2e_test.yml delete mode 100644 jenkins/migrationIntegPipelines/rfsBackfillE2EPipeline.groovy create mode 100644 jenkins/migrationIntegPipelines/rfsDefaultE2ETestCover.groovy create mode 100644 jenkins/migrationIntegPipelines/trafficReplayDefaultE2ETestCover.groovy create mode 100644 vars/rfsDefaultE2ETest.groovy rename jenkins/migrationIntegPipelines/ec2SourceE2EPipeline.groovy => vars/trafficReplayDefaultE2ETest.groovy (65%) diff --git a/.github/workflows/rfs_pr_e2e_test.yml b/.github/workflows/rfs_pr_e2e_test.yml new file mode 100644 index 000000000..abbb61b3d --- /dev/null +++ b/.github/workflows/rfs_pr_e2e_test.yml @@ -0,0 +1,40 @@ +name: Jenkins + +on: + push: + branches-ignore: + - 'backport/**' + - 'dependabot/**' + pull_request_target: + types: [opened, synchronize, reopened] + +env: + python-version: '3.11' + +permissions: + contents: read # to fetch code (actions/checkout) + +jobs: + rfs-e2e-aws-test: + runs-on: ubuntu-latest + steps: + - name: Determine Github repository and branch + id: determine-repo-vars + run: | + if [[ "${GITHUB_EVENT_NAME}" == "pull_request_target" ]]; then + branch_name="${{ github.event.pull_request.head.ref }}" + pr_repo_url="https://github.com/${{ github.event.pull_request.head.repo.full_name }}.git" + else + branch_name="${{ github.ref_name }}" + pr_repo_url="https://github.com/${{ github.repository }}.git" + fi + echo "Running jenkins test on repo: $pr_repo_url and branch: $branch_name" + echo "branch_name=$branch_name" >> $GITHUB_OUTPUT + echo "pr_repo_url=$pr_repo_url" >> $GITHUB_OUTPUT + - name: Jenkins Job Trigger and Monitor + uses: lewijacn/jenkins-trigger@1.0.4 + with: + jenkins_url: "https://migrations.ci.opensearch.org" + job_name: "rfs-default-e2e-test" + api_token: "${{ secrets.JENKINS_MIGRATIONS_GENERIC_WEBHOOK_TOKEN }}" + job_params: "GIT_REPO_URL=${{ steps.determine-repo-vars.outputs.pr_repo_url }},GIT_BRANCH=${{ steps.determine-repo-vars.outputs.branch_name }}" diff --git a/jenkins/migrationIntegPipelines/rfsBackfillE2EPipeline.groovy b/jenkins/migrationIntegPipelines/rfsBackfillE2EPipeline.groovy deleted file mode 100644 index a4e9038ca..000000000 --- a/jenkins/migrationIntegPipelines/rfsBackfillE2EPipeline.groovy +++ /dev/null @@ -1,78 +0,0 @@ -// Note: -// 1. There is a still a manual step needed on the EC2 source load balancer to replace its security group rule which allows all traffic (0.0.0.0/0) to -// allow traffic for the relevant service security group. This needs a better story around accepting user security groups in our Migration CDK. - -def sourceContextId = 'source-single-node-ec2' -def migrationContextId = 'migration-rfs' -// These default values should only be used on the initial Jenkins run in order to load parameter options into the UI, -// all future runs should use the specified parameters -def gitBranch = params.GIT_BRANCH ?: 'main' -def gitUrl = params.GIT_REPO_URL ?: 'https://github.com/opensearch-project/opensearch-migrations.git' -def source_cdk_context = """ - { - "source-single-node-ec2": { - "suffix": "ec2-source-", - "networkStackSuffix": "ec2-source-", - "distVersion": "7.10.2", - "distributionUrl": "https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-oss-7.10.2-linux-x86_64.tar.gz", - "captureProxyEnabled": false, - "securityDisabled": true, - "minDistribution": false, - "cpuArch": "x64", - "isInternal": true, - "singleNodeCluster": true, - "networkAvailabilityZones": 2, - "dataNodeCount": 1, - "managerNodeCount": 0, - "serverAccessType": "ipv4", - "restrictServerAccessTo": "0.0.0.0/0" - } - } -""" -def migration_cdk_context = """ - { - "migration-rfs": { - "stage": "", - "vpcId": "", - "engineVersion": "OS_2.11", - "domainName": "os-cluster-", - "dataNodeCount": 2, - "openAccessPolicyEnabled": true, - "domainRemovalPolicy": "DESTROY", - "artifactBucketRemovalPolicy": "DESTROY", - "trafficReplayerServiceEnabled": false, - "reindexFromSnapshotServiceEnabled": true, - "sourceClusterEndpoint": "" - } - } -""" - -library identifier: "migrations-lib@${gitBranch}", retriever: modernSCM( - [$class: 'GitSCMSource', - remote: "${gitUrl}"]) - -defaultIntegPipeline( - sourceContext: source_cdk_context, - migrationContext: migration_cdk_context, - sourceContextId: sourceContextId, - migrationContextId: migrationContextId, - defaultStageId: 'rfs-integ', - skipCaptureProxyOnNodeSetup: true, - integTestStep: { - def time = new Date().getTime() - def uniqueId = "integ_min_${time}_${currentBuild.number}" - def test_dir = "/root/lib/integ_test/integ_test" - def test_result_file = "${test_dir}/reports/${uniqueId}/report.xml" - def command = "pipenv run pytest --log-file=${test_dir}/reports/${uniqueId}/pytest.log " + - "--junitxml=${test_result_file} ${test_dir}/backfill_tests.py " + - "--unique_id ${uniqueId} " + - "-s" - withCredentials([string(credentialsId: 'migrations-test-account-id', variable: 'MIGRATIONS_TEST_ACCOUNT_ID')]) { - withAWS(role: 'JenkinsDeploymentRole', roleAccount: "${MIGRATIONS_TEST_ACCOUNT_ID}", duration: 3600, roleSessionName: 'jenkins-session') { - sh "sudo --preserve-env ./awsRunIntegTests.sh --command '${command}' " + - "--test-result-file ${test_result_file} " + - "--stage ${params.STAGE}" - } - } - } -) diff --git a/jenkins/migrationIntegPipelines/rfsDefaultE2ETestCover.groovy b/jenkins/migrationIntegPipelines/rfsDefaultE2ETestCover.groovy new file mode 100644 index 000000000..58ed79a0e --- /dev/null +++ b/jenkins/migrationIntegPipelines/rfsDefaultE2ETestCover.groovy @@ -0,0 +1,9 @@ +def gitBranch = params.GIT_BRANCH ?: 'main' +def gitUrl = params.GIT_REPO_URL ?: 'https://github.com/opensearch-project/opensearch-migrations.git' + +library identifier: "migrations-lib@${gitBranch}", retriever: modernSCM( + [$class: 'GitSCMSource', + remote: "${gitUrl}"]) + +// Shared library function (location from root: vars/rfsDefaultE2ETest.groovy) +rfsDefaultE2ETest() diff --git a/jenkins/migrationIntegPipelines/trafficReplayDefaultE2ETestCover.groovy b/jenkins/migrationIntegPipelines/trafficReplayDefaultE2ETestCover.groovy new file mode 100644 index 000000000..4a8ddc7e0 --- /dev/null +++ b/jenkins/migrationIntegPipelines/trafficReplayDefaultE2ETestCover.groovy @@ -0,0 +1,9 @@ +def gitBranch = params.GIT_BRANCH ?: 'main' +def gitUrl = params.GIT_REPO_URL ?: 'https://github.com/opensearch-project/opensearch-migrations.git' + +library identifier: "migrations-lib@${gitBranch}", retriever: modernSCM( + [$class: 'GitSCMSource', + remote: "${gitUrl}"]) + +// Shared library function (location from root: vars/trafficReplayDefaultE2ETest.groovy) +trafficReplayDefaultE2ETest() diff --git a/vars/defaultIntegPipeline.groovy b/vars/defaultIntegPipeline.groovy index cc67ac6f0..548370d58 100644 --- a/vars/defaultIntegPipeline.groovy +++ b/vars/defaultIntegPipeline.groovy @@ -2,6 +2,7 @@ def call(Map config = [:]) { def sourceContext = config.sourceContext def migrationContext = config.migrationContext def defaultStageId = config.defaultStageId + def jobName = config.jobName if(sourceContext == null || sourceContext.isEmpty()){ throw new RuntimeException("The sourceContext argument must be provided"); } @@ -11,11 +12,16 @@ def call(Map config = [:]) { if(defaultStageId == null || defaultStageId.isEmpty()){ throw new RuntimeException("The defaultStageId argument must be provided"); } + if(jobName == null || jobName.isEmpty()){ + throw new RuntimeException("The jobName argument must be provided"); + } def source_context_id = config.sourceContextId ?: 'source-single-node-ec2' def migration_context_id = config.migrationContextId ?: 'migration-default' def source_context_file_name = 'sourceJenkinsContext.json' def migration_context_file_name = 'migrationJenkinsContext.json' def skipCaptureProxyOnNodeSetup = config.skipCaptureProxyOnNodeSetup ?: false + def testDir = "/root/lib/integ_test/integ_test" + def integTestCommand = config.integTestCommand ?: "${testDir}/replayer_tests.py" pipeline { agent { label config.workerAgent ?: 'Jenkins-Default-Agent-X64-C5xlarge-Single-Host' } @@ -25,6 +31,27 @@ def call(Map config = [:]) { string(name: 'STAGE', defaultValue: "${defaultStageId}", description: 'Stage name for deployment environment') } + options { + // Acquire lock on a given deployment stage + lock(label: params.STAGE, quantity: 1, variable: 'stage') + timeout(time: 3, unit: 'HOURS') + buildDiscarder(logRotator(daysToKeepStr: '30')) + } + + triggers { + GenericTrigger( + genericVariables: [ + [key: 'GIT_REPO_URL', value: '$.GIT_REPO_URL'], + [key: 'GIT_BRANCH', value: '$.GIT_BRANCH'], + [key: 'job_name', value: '$.job_name'] + ], + tokenCredentialId: 'jenkins-migrations-generic-webhook-token', + causeString: 'Triggered by PR on opensearch-migrations repository', + regexpFilterExpression: "^$jobName\$", + regexpFilterText: "\$job_name", + ) + } + stages { stage('Checkout') { steps { @@ -92,13 +119,14 @@ def call(Map config = [:]) { if (config.deployStep) { config.deployStep() } else { + echo "Acquired deployment stage: ${stage}" sh 'sudo usermod -aG docker $USER' sh 'sudo newgrp docker' def baseCommand = "sudo --preserve-env ./awsE2ESolutionSetup.sh --source-context-file './$source_context_file_name' " + "--migration-context-file './$migration_context_file_name' " + "--source-context-id $source_context_id " + "--migration-context-id $migration_context_id " + - "--stage ${params.STAGE} " + + "--stage ${stage} " + "--migrations-git-url ${params.GIT_REPO_URL} " + "--migrations-git-branch ${params.GIT_BRANCH}" if (skipCaptureProxyOnNodeSetup) { @@ -127,17 +155,16 @@ def call(Map config = [:]) { } else { def time = new Date().getTime() def uniqueId = "integ_min_${time}_${currentBuild.number}" - def test_dir = "/root/lib/integ_test/integ_test" - def test_result_file = "${test_dir}/reports/${uniqueId}/report.xml" - def command = "pipenv run pytest --log-file=${test_dir}/reports/${uniqueId}/pytest.log " + - "--junitxml=${test_result_file} ${test_dir}/replayer_tests.py " + + def test_result_file = "${testDir}/reports/${uniqueId}/report.xml" + def command = "pipenv run pytest --log-file=${testDir}/reports/${uniqueId}/pytest.log " + + "--junitxml=${test_result_file} ${integTestCommand} " + "--unique_id ${uniqueId} " + "-s" withCredentials([string(credentialsId: 'migrations-test-account-id', variable: 'MIGRATIONS_TEST_ACCOUNT_ID')]) { withAWS(role: 'JenkinsDeploymentRole', roleAccount: "${MIGRATIONS_TEST_ACCOUNT_ID}", duration: 3600, roleSessionName: 'jenkins-session') { sh "sudo --preserve-env ./awsRunIntegTests.sh --command '${command}' " + "--test-result-file ${test_result_file} " + - "--stage ${params.STAGE}" + "--stage ${stage}" } } } diff --git a/vars/rfsDefaultE2ETest.groovy b/vars/rfsDefaultE2ETest.groovy new file mode 100644 index 000000000..aac9db03a --- /dev/null +++ b/vars/rfsDefaultE2ETest.groovy @@ -0,0 +1,57 @@ +// Note: +// 1. There is a still a manual step needed on the EC2 source load balancer to replace its security group rule which allows all traffic (0.0.0.0/0) to +// allow traffic for the relevant service security group. This needs a better story around accepting user security groups in our Migration CDK. + +def call(Map config = [:]) { + def sourceContextId = 'source-single-node-ec2' + def migrationContextId = 'migration-rfs' + def source_cdk_context = """ + { + "source-single-node-ec2": { + "suffix": "ec2-source-", + "networkStackSuffix": "ec2-source-", + "distVersion": "7.10.2", + "distributionUrl": "https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-oss-7.10.2-linux-x86_64.tar.gz", + "captureProxyEnabled": false, + "securityDisabled": true, + "minDistribution": false, + "cpuArch": "x64", + "isInternal": true, + "singleNodeCluster": true, + "networkAvailabilityZones": 2, + "dataNodeCount": 1, + "managerNodeCount": 0, + "serverAccessType": "ipv4", + "restrictServerAccessTo": "0.0.0.0/0" + } + } + """ + def migration_cdk_context = """ + { + "migration-rfs": { + "stage": "", + "vpcId": "", + "engineVersion": "OS_2.11", + "domainName": "os-cluster-", + "dataNodeCount": 2, + "openAccessPolicyEnabled": true, + "domainRemovalPolicy": "DESTROY", + "artifactBucketRemovalPolicy": "DESTROY", + "trafficReplayerServiceEnabled": false, + "reindexFromSnapshotServiceEnabled": true, + "sourceClusterEndpoint": "" + } + } + """ + + defaultIntegPipeline( + sourceContext: source_cdk_context, + migrationContext: migration_cdk_context, + sourceContextId: sourceContextId, + migrationContextId: migrationContextId, + defaultStageId: 'rfs-integ', + skipCaptureProxyOnNodeSetup: true, + jobName: 'rfs-default-e2e-test', + integTestCommand: '/root/lib/integ_test/integ_test/backfill_tests.py' + ) +} diff --git a/jenkins/migrationIntegPipelines/ec2SourceE2EPipeline.groovy b/vars/trafficReplayDefaultE2ETest.groovy similarity index 65% rename from jenkins/migrationIntegPipelines/ec2SourceE2EPipeline.groovy rename to vars/trafficReplayDefaultE2ETest.groovy index 5cf981ba0..26e44b2c7 100644 --- a/jenkins/migrationIntegPipelines/ec2SourceE2EPipeline.groovy +++ b/vars/trafficReplayDefaultE2ETest.groovy @@ -2,13 +2,10 @@ // 1. There is a still a manual step needed on the EC2 source load balancer to replace its security group rule which allows all traffic (0.0.0.0/0) to // allow traffic for the relevant service security group. This needs a better story around accepting user security groups in our Migration CDK. -def sourceContextId = 'source-single-node-ec2' -def migrationContextId = 'migration-default' -// These default values should only be used on the initial Jenkins run in order to load parameter options into the UI, -// all future runs should use the specified parameters -def gitBranch = params.GIT_BRANCH ?: 'main' -def gitUrl = params.GIT_REPO_URL ?: 'https://github.com/opensearch-project/opensearch-migrations.git' -def source_cdk_context = """ +def call(Map config = [:]) { + def sourceContextId = 'source-single-node-ec2' + def migrationContextId = 'migration-default' + def source_cdk_context = """ { "source-single-node-ec2": { "suffix": "ec2-source-", @@ -29,8 +26,8 @@ def source_cdk_context = """ "restrictServerAccessTo": "0.0.0.0/0" } } -""" -def migration_cdk_context = """ + """ + def migration_cdk_context = """ { "migration-default": { "stage": "", @@ -50,19 +47,17 @@ def migration_cdk_context = """ "migrationAPIEnabled": true } } -""" + """ -library identifier: "migrations-lib@${gitBranch}", retriever: modernSCM( - [$class: 'GitSCMSource', - remote: "${gitUrl}"]) - -defaultIntegPipeline( - sourceContext: source_cdk_context, - migrationContext: migration_cdk_context, - sourceContextId: sourceContextId, - migrationContextId: migrationContextId, - defaultStageId: 'aws-integ', - //deployStep: { - // echo 'Custom Test Step' - //} -) + defaultIntegPipeline( + sourceContext: source_cdk_context, + migrationContext: migration_cdk_context, + sourceContextId: sourceContextId, + migrationContextId: migrationContextId, + defaultStageId: 'aws-integ', + jobName: 'traffic-replay-default-e2e-test', + //deployStep: { + // echo 'Custom Test Step' + //} + ) +}