From fe88c1e3b60206b3d354cf84ae8459d464d188c2 Mon Sep 17 00:00:00 2001 From: Andre Kurait Date: Tue, 16 Apr 2024 02:21:10 -0500 Subject: [PATCH 1/3] Add ExitOnOutOfMemoryError Signed-off-by: Andre Kurait --- .../groovy/org/opensearch/migrations/common/CommonUtils.groovy | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/TrafficCapture/buildSrc/src/main/groovy/org/opensearch/migrations/common/CommonUtils.groovy b/TrafficCapture/buildSrc/src/main/groovy/org/opensearch/migrations/common/CommonUtils.groovy index 898fc4ca09..02efc86370 100644 --- a/TrafficCapture/buildSrc/src/main/groovy/org/opensearch/migrations/common/CommonUtils.groovy +++ b/TrafficCapture/buildSrc/src/main/groovy/org/opensearch/migrations/common/CommonUtils.groovy @@ -60,11 +60,12 @@ class CommonUtils { } copyFile("jars", "/jars") + def jvmParams = "-XX:MaxRAMPercentage=80.0 -XX:+ExitOnOutOfMemoryError" // can't set the environment variable from the runtimeClasspath because the Dockerfile is // constructed in the configuration phase and the classpath won't be realized until the // execution phase. Therefore, we need to have docker run the command to resolve the classpath // and it's simplest to pack that up into a helper script. - runCommand("printf \"#!/bin/sh\\njava -XX:MaxRAMPercentage=80.0 -cp `echo /jars/*.jar | tr \\ :` \\\"\\\$@\\\" \" > /runJavaWithClasspath.sh"); + runCommand("printf \"#!/bin/sh\\njava ${jvmParams} -cp `echo /jars/*.jar | tr \\ :` \\\"\\\$@\\\" \" > /runJavaWithClasspath.sh"); runCommand("chmod +x /runJavaWithClasspath.sh") // container stay-alive defaultCommand('tail', '-f', '/dev/null') From e53781280367c2194d9d7705b88616c5bed36503 Mon Sep 17 00:00:00 2001 From: Greg Schohn Date: Mon, 15 Apr 2024 11:59:14 -0400 Subject: [PATCH 2/3] Add a ReadTimeout handler on the target responses so that the replayer won't be jammed indefinitely when a server stops responding for any reason. I've also refactored some convenience constructors for top-level replayer sort of stuff that are only used tests into the replayer's testFixtures package. That cleans up some code that requires higher attention and lets us be a bit more cavalier about defaults since it's now more clear that they're only being used for tests. Signed-off-by: Greg Schohn --- .../netty/BacksideConnectionPool.java | 6 +- .../replay/ClientConnectionPool.java | 15 ++- .../migrations/replay/TrafficReplayer.java | 63 ++++++----- .../replay/TrafficReplayerTopLevel.java | 100 ++++-------------- .../NettyPacketToHttpConsumer.java | 15 ++- .../replay/netty/BacksideSnifferHandler.java | 5 - .../replay/RequestSenderOrchestratorTest.java | 5 +- .../replay/TrafficReplayerTest.java | 15 +-- .../NettyPacketToHttpConsumerTest.java | 71 +++++++++---- .../FullReplayerWithTracingChecksTest.java | 7 +- .../e2etests/FullTrafficReplayerTest.java | 10 +- ...ficStreamBecomesTwoTargetChannelsTest.java | 12 ++- .../e2etests/TrafficReplayerRunner.java | 7 +- .../RootReplayerConstructorExtensions.java | 69 ++++++++++++ 14 files changed, 238 insertions(+), 162 deletions(-) create mode 100644 TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/RootReplayerConstructorExtensions.java diff --git a/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/BacksideConnectionPool.java b/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/BacksideConnectionPool.java index b1bf526cb3..7e3b3f0b95 100644 --- a/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/BacksideConnectionPool.java +++ b/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/BacksideConnectionPool.java @@ -65,10 +65,8 @@ public ChannelFuture getOutboundConnectionFuture(EventLoop eventLoop) { private void logProgressAtInterval(Level logLevel, EventLoop eventLoop, ExpiringSubstitutableItemPool channelPoolMap, Duration frequency) { - eventLoop.schedule(() -> { - log.atLevel(logLevel).log(channelPoolMap.getStats().toString()); - logProgressAtInterval(logLevel, eventLoop, channelPoolMap, frequency); - }, frequency.toMillis(), TimeUnit.MILLISECONDS); + eventLoop.scheduleAtFixedRate(() -> log.atLevel(logLevel).log(channelPoolMap.getStats().toString()), + frequency.toMillis(), frequency.toMillis(), TimeUnit.MILLISECONDS); } private ChannelFuture buildConnectionFuture(EventLoop eventLoop) { diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/ClientConnectionPool.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/ClientConnectionPool.java index bcd49257a5..11089b8e4b 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/ClientConnectionPool.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/ClientConnectionPool.java @@ -12,6 +12,7 @@ import io.netty.util.concurrent.DefaultThreadFactory; import io.netty.util.concurrent.Future; import lombok.AllArgsConstructor; +import lombok.Builder; import lombok.EqualsAndHashCode; import lombok.NonNull; import lombok.SneakyThrows; @@ -23,15 +24,18 @@ import org.opensearch.migrations.replay.util.StringTrackableCompletableFuture; import java.net.URI; +import java.time.Duration; import java.util.concurrent.CompletableFuture; import java.util.stream.Collectors; @Slf4j public class ClientConnectionPool { + private final URI serverUri; private final SslContext sslContext; public final NioEventLoopGroup eventLoopGroup; private final LoadingCache connectionId2ChannelCache; + private final Duration timeout; @EqualsAndHashCode @AllArgsConstructor @@ -44,9 +48,14 @@ private Key getKey(String connectionId, int sessionNumber) { return new Key(connectionId, sessionNumber); } - public ClientConnectionPool(URI serverUri, SslContext sslContext, String targetConnectionPoolName, int numThreads) { + public ClientConnectionPool(@NonNull URI serverUri, + SslContext sslContext, + @NonNull String targetConnectionPoolName, + int numThreads, + @NonNull Duration timeout) { this.serverUri = serverUri; this.sslContext = sslContext; + this.timeout = timeout; this.eventLoopGroup = new NioEventLoopGroup(numThreads, new DefaultThreadFactory(targetConnectionPoolName)); @@ -55,7 +64,7 @@ public ClientConnectionPool(URI serverUri, SslContext sslContext, String targetC })); } - public ConnectionReplaySession buildConnectionReplaySession(final IReplayContexts.IChannelKeyContext channelKeyCtx) { + public ConnectionReplaySession buildConnectionReplaySession(IReplayContexts.IChannelKeyContext channelKeyCtx) { if (eventLoopGroup.isShuttingDown()) { throw new IllegalStateException("Event loop group is shutting down. Not creating a new session."); } @@ -74,7 +83,7 @@ public ConnectionReplaySession buildConnectionReplaySession(final IReplayContext return new AdaptiveRateLimiter() .get(() -> { var channelFuture = NettyPacketToHttpConsumer.createClientConnection(eventLoop, - sslContext, serverUri, connectionContext); + sslContext, serverUri, connectionContext, timeout); return getCompletedChannelFutureAsCompletableFuture(connectionContext, channelFuture); }); } 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 7a628f2db2..d2f20d72e8 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 @@ -6,6 +6,7 @@ import io.netty.util.concurrent.DefaultThreadFactory; import lombok.extern.slf4j.Slf4j; import org.opensearch.migrations.replay.tracing.RootReplayerContext; +import org.opensearch.migrations.replay.traffic.source.TrafficStreamLimiter; import org.opensearch.migrations.replay.util.ActiveContextMonitor; import org.opensearch.migrations.replay.util.OrderedWorkerTracker; import org.opensearch.migrations.tracing.ActiveContextTracker; @@ -128,6 +129,28 @@ public static class Parameters { String useSigV4ServiceAndRegion; + @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.") + String transformerConfigFile; + @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, @@ -162,6 +185,14 @@ public static class Parameters { 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.") + int targetServerResponseTimeoutSeconds = 30; + @Parameter(required = false, names = {"--kafka-traffic-brokers"}, arity=1, @@ -194,26 +225,6 @@ public static class Parameters { 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; - @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.") - String transformerConfigFile; - @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; } private static Parameters parseArgs(String[] args) { @@ -300,16 +311,18 @@ public static void main(String[] args) throws Exception { var orderedRequestTracker = new OrderedWorkerTracker(); var tr = new TrafficReplayerTopLevel(topContext, uri, authTransformer, new TransformationLoader().getTransformerFactoryLoader(uri.getHost(), params.userAgent, transformerConfig), - params.allowInsecureConnections, params.numClientThreads, params.maxConcurrentRequests, - orderedRequestTracker); + TrafficReplayerTopLevel.makeClientConnectionPool( + uri, params.allowInsecureConnections, params.numClientThreads, + Duration.ofSeconds(params.targetServerResponseTimeoutSeconds)), + new TrafficStreamLimiter(params.maxConcurrentRequests), orderedRequestTracker); activeContextMonitor = new ActiveContextMonitor( globalContextTracker, perContextTracker, orderedRequestTracker, 64, cf->cf.formatAsString(TrafficReplayerTopLevel::formatWorkItem), activeContextLogger); ActiveContextMonitor finalActiveContextMonitor = activeContextMonitor; - scheduledExecutorService.scheduleAtFixedRate(() -> { - activeContextLogger.atInfo().setMessage(() -> "Total requests outstanding: " + tr.requestWorkTracker.size()).log(); + scheduledExecutorService.scheduleAtFixedRate(()->{ + activeContextLogger.atInfo().setMessage(()->"Total requests outstanding: " + tr.requestWorkTracker.size()).log(); finalActiveContextMonitor.run(); - }, + }, ACTIVE_WORK_MONITOR_CADENCE_MS, ACTIVE_WORK_MONITOR_CADENCE_MS, TimeUnit.MILLISECONDS); setupShutdownHookForReplayer(tr); diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/TrafficReplayerTopLevel.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/TrafficReplayerTopLevel.java index 0616343b26..fff581d6d4 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/TrafficReplayerTopLevel.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/TrafficReplayerTopLevel.java @@ -82,79 +82,6 @@ public int size() { private final AtomicReference shutdownReasonRef; private final AtomicReference> shutdownFutureRef; - public TrafficReplayerTopLevel(IRootReplayerContext context, - URI serverUri, - IAuthTransformerFactory authTransformerFactory, - boolean allowInsecureConnections) - throws SSLException { - this(context, serverUri, authTransformerFactory, - new TransformationLoader().getTransformerFactoryLoader(serverUri.getHost()), allowInsecureConnections); - } - - public TrafficReplayerTopLevel(IRootReplayerContext topContext, - URI uri, - IAuthTransformerFactory authTransformerFactory, - IJsonTransformer jsonTransformer, - boolean allowInsecureConnections) - throws SSLException { - this(topContext, uri, authTransformerFactory, jsonTransformer, allowInsecureConnections, 0, - 1024); - } - - public TrafficReplayerTopLevel(IRootReplayerContext topContext, - URI uri, - IAuthTransformerFactory authTransformer, - IJsonTransformer jsonTransformer, - boolean allowInsecureConnections, - int numClientThreads, - int maxConcurrentRequests) throws SSLException { - this(topContext, uri, authTransformer, jsonTransformer, allowInsecureConnections, numClientThreads, - maxConcurrentRequests, new ConcurrentHashMapWorkTracker<>()); - } - - public TrafficReplayerTopLevel(IRootReplayerContext topContext, - URI uri, - IAuthTransformerFactory authTransformer, - IJsonTransformer jsonTransformer, - boolean allowInsecureConnections, - int numClientThreads, - int maxConcurrentRequests, - IStreamableWorkTracker workTracker) throws SSLException { - this(topContext, uri, authTransformer, jsonTransformer, allowInsecureConnections, - numClientThreads, maxConcurrentRequests, - getTargetConnectionPoolName(targetConnectionPoolUniqueCounter.getAndIncrement()), - workTracker); - } - - public TrafficReplayerTopLevel(IRootReplayerContext topContext, - URI uri, - IAuthTransformerFactory authTransformer, - IJsonTransformer jsonTransformer, - boolean allowInsecureConnections, - int numClientThreads, - int maxConcurrentRequests, - String connectionPoolName) throws SSLException { - this(topContext, uri, authTransformer, jsonTransformer, allowInsecureConnections, - numClientThreads, maxConcurrentRequests, - connectionPoolName, - new ConcurrentHashMapWorkTracker<>()); - } - - public TrafficReplayerTopLevel(IRootReplayerContext context, - URI serverUri, - IAuthTransformerFactory authTransformerFactory, - IJsonTransformer jsonTransformer, - boolean allowInsecureConnections, - int numSendingThreads, - int maxConcurrentOutstandingRequests, - String connectionPoolName, - IStreamableWorkTracker workTracker) throws SSLException { - this(context, serverUri, authTransformerFactory, jsonTransformer, - new ClientConnectionPool(serverUri, - loadSslContext(serverUri, allowInsecureConnections), connectionPoolName, numSendingThreads), - new TrafficStreamLimiter(maxConcurrentOutstandingRequests), workTracker); - } - public TrafficReplayerTopLevel(IRootReplayerContext context, URI serverUri, IAuthTransformerFactory authTransformerFactory, @@ -169,7 +96,28 @@ public TrafficReplayerTopLevel(IRootReplayerContext context, shutdownFutureRef = new AtomicReference<>(); } - private static SslContext loadSslContext(URI serverUri, boolean allowInsecureConnections) throws SSLException { + public static ClientConnectionPool makeClientConnectionPool(URI serverUri, boolean allowInsecureConnections, + int numSendingThreads, Duration timeout) + throws SSLException { + return makeClientConnectionPool(serverUri, allowInsecureConnections, numSendingThreads, null, timeout); + } + + public static ClientConnectionPool makeClientConnectionPool(URI serverUri, + boolean allowInsecureConnections, + int numSendingThreads, + String connectionPoolName, + Duration timeout) throws SSLException { + return new ClientConnectionPool(serverUri, loadSslContext(serverUri, allowInsecureConnections), + connectionPoolName != null ? connectionPoolName : + getTargetConnectionPoolName(targetConnectionPoolUniqueCounter.getAndIncrement()), + numSendingThreads, timeout); + } + + public static String getTargetConnectionPoolName(int i) { + return TARGET_CONNECTION_POOL_NAME + (i == 0 ? "" : Integer.toString(i)); + } + + public static SslContext loadSslContext(URI serverUri, boolean allowInsecureConnections) throws SSLException { if (serverUri.getScheme().equalsIgnoreCase("https")) { var sslContextBuilder = SslContextBuilder.forClient(); if (allowInsecureConnections) { @@ -181,10 +129,6 @@ private static SslContext loadSslContext(URI serverUri, boolean allowInsecureCon } } - private static String getTargetConnectionPoolName(int i) { - return TARGET_CONNECTION_POOL_NAME + (i == 0 ? "" : Integer.toString(i)); - } - public void setupRunAndWaitForReplayToFinish(Duration observedPacketConnectionTimeout, BlockingTrafficSource trafficSource, TimeShifter timeShifter, 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 a2aba11b86..e47654215d 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/NettyPacketToHttpConsumer.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/NettyPacketToHttpConsumer.java @@ -18,6 +18,7 @@ import io.netty.handler.logging.LoggingHandler; import io.netty.handler.ssl.SslContext; import io.netty.handler.ssl.SslHandler; +import io.netty.handler.timeout.ReadTimeoutHandler; import lombok.NonNull; import lombok.extern.slf4j.Slf4j; import org.opensearch.migrations.replay.AggregatedRawResponse; @@ -34,10 +35,12 @@ import java.net.URI; import java.nio.charset.StandardCharsets; +import java.time.Duration; import java.time.Instant; import java.util.NoSuchElementException; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; @Slf4j @@ -54,6 +57,7 @@ public class NettyPacketToHttpConsumer implements IPacketFinalizingConsumer currentRequestContextUnion; - private static class ConnectionClosedListenerHandler extends ChannelDuplexHandler { + private static class ConnectionClosedListenerHandler extends ReadTimeoutHandler { private final IReplayContexts.ISocketContext socketContext; - ConnectionClosedListenerHandler(IReplayContexts.IChannelKeyContext channelKeyContext) { + ConnectionClosedListenerHandler(IReplayContexts.IChannelKeyContext channelKeyContext, + Duration timeout) { + super(timeout.toMillis(), TimeUnit.MILLISECONDS); socketContext = channelKeyContext.createSocketContext(); } @Override @@ -139,7 +145,8 @@ public IReplayContexts.ITargetRequestContext getParentContext() { public static ChannelFuture createClientConnection(EventLoopGroup eventLoopGroup, SslContext sslContext, URI serverUri, - IReplayContexts.IChannelKeyContext channelKeyContext) { + IReplayContexts.IChannelKeyContext channelKeyContext, + Duration timeout) { String host = serverUri.getHost(); int port = serverUri.getPort(); log.atTrace().setMessage(()->"Active - setting up backend connection to " + host + ":" + port).log(); @@ -150,7 +157,7 @@ public static ChannelFuture createClientConnection(EventLoopGroup eventLoopGroup @Override protected void initChannel(@NonNull Channel ch) throws Exception { ch.pipeline().addFirst(CONNECTION_CLOSE_HANDLER_NAME, - new ConnectionClosedListenerHandler(channelKeyContext)); + new ConnectionClosedListenerHandler(channelKeyContext, timeout)); } }) .channel(NioSocketChannel.class) diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/netty/BacksideSnifferHandler.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/netty/BacksideSnifferHandler.java index 16598417eb..3985a10b30 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/netty/BacksideSnifferHandler.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/netty/BacksideSnifferHandler.java @@ -30,9 +30,4 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception bb.resetReaderIndex(); ctx.fireChannelRead(msg); } - - @Override - public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { - log.atWarn().setCause(cause).setMessage("Caught exception").log(); - } } \ No newline at end of file diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/RequestSenderOrchestratorTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/RequestSenderOrchestratorTest.java index 7f1572cc96..661d76ba31 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 @@ -37,8 +37,9 @@ public void testThatSchedulingWorks() throws Exception { try (var httpServer = SimpleHttpServer.makeServer(false, r -> TestHttpServerContext.makeResponse(r, Duration.ofMillis(100)))) { var testServerUri = httpServer.localhostEndpoint(); - var clientConnectionPool = new ClientConnectionPool(testServerUri, null, - "targetConnectionPool for testThatSchedulingWorks", 1); + var clientConnectionPool = TrafficReplayerTopLevel.makeClientConnectionPool(testServerUri, false, + 1, "targetConnectionPool for testThatSchedulingWorks", + Duration.ofSeconds(30)); var senderOrchestrator = new RequestSenderOrchestrator(clientConnectionPool); var baseTime = Instant.now(); Instant lastEndTime = baseTime; diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/TrafficReplayerTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/TrafficReplayerTest.java index ef8ba36d9e..85a97559df 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/TrafficReplayerTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/TrafficReplayerTest.java @@ -9,6 +9,7 @@ import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; import org.opensearch.migrations.replay.tracing.IReplayContexts; import org.opensearch.migrations.replay.traffic.source.InputStreamOfTraffic; +import org.opensearch.migrations.replay.traffic.source.TrafficStreamLimiter; import org.opensearch.migrations.testutils.WrapWithNettyLeakDetection; import org.opensearch.migrations.tracing.InstrumentationTest; import org.opensearch.migrations.trafficcapture.protos.CloseObservation; @@ -151,8 +152,9 @@ static byte[] synthesizeTrafficStreamsIntoByteArray(Instant timestamp, int numSt @Test public void testReader() throws Exception { - try (var tr = new TrafficReplayerTopLevel(rootContext, - new URI("http://localhost:9200"), null, false)) { + var uri = new URI("http://localhost:9200"); + try (var tr = new RootReplayerConstructorExtensions(rootContext, uri, null, null, + RootReplayerConstructorExtensions.makeClientConnectionPool(uri))) { List> byteArrays = new ArrayList<>(); CapturedTrafficToHttpTransactionAccumulator trafficAccumulator = new CapturedTrafficToHttpTransactionAccumulator(Duration.ofSeconds(30), null, @@ -207,10 +209,11 @@ public void onTrafficStreamIgnored(@NonNull IReplayContexts.ITrafficStreamsLifec @Test public void testCapturedReadsAfterCloseAreHandledAsNew() throws Exception { - try (var tr = new TrafficReplayerTopLevel(rootContext, - new URI("http://localhost:9200"), null, - new TransformationLoader().getTransformerFactoryLoader("localhost"), false - )) { + var uri = new URI("http://localhost:9200"); + try (var tr = new RootReplayerConstructorExtensions(rootContext, + uri, null, + new TransformationLoader().getTransformerFactoryLoader("localhost"), + RootReplayerConstructorExtensions.makeClientConnectionPool(uri))) { List> byteArrays = new ArrayList<>(); var remainingAccumulations = new AtomicInteger(); CapturedTrafficToHttpTransactionAccumulator trafficAccumulator = 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 13e7647324..abcd7fee12 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 @@ -1,28 +1,27 @@ package org.opensearch.migrations.replay.datahandlers; -import io.netty.buffer.Unpooled; import io.netty.channel.nio.NioEventLoopGroup; import io.netty.handler.codec.http.HttpHeaderNames; import io.netty.handler.codec.http.HttpHeaderValues; import io.netty.handler.ssl.SslContextBuilder; import io.netty.handler.ssl.util.InsecureTrustManagerFactory; +import io.netty.handler.timeout.ReadTimeoutException; import io.netty.util.concurrent.DefaultThreadFactory; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.ValueSource; import org.opensearch.migrations.replay.AggregatedRawResponse; import org.opensearch.migrations.replay.ClientConnectionPool; -import org.opensearch.migrations.replay.HttpByteBufFormatter; import org.opensearch.migrations.replay.PacketToTransformingHttpHandlerFactory; import org.opensearch.migrations.replay.ReplayEngine; import org.opensearch.migrations.replay.ReplayUtils; import org.opensearch.migrations.replay.RequestSenderOrchestrator; import org.opensearch.migrations.replay.TimeShifter; -import org.opensearch.migrations.replay.TrafficReplayer; import org.opensearch.migrations.replay.TrafficReplayerTopLevel; import org.opensearch.migrations.replay.TransformationLoader; import org.opensearch.migrations.replay.datatypes.ConnectionReplaySession; @@ -35,23 +34,18 @@ import org.opensearch.migrations.tracing.InstrumentationTest; import org.opensearch.migrations.tracing.TestContext; -import javax.net.ssl.SSLException; import java.io.BufferedReader; import java.io.IOException; import java.io.InputStreamReader; import java.net.URI; import java.nio.charset.StandardCharsets; -import java.security.KeyManagementException; -import java.security.KeyStoreException; -import java.security.NoSuchAlgorithmException; import java.time.Duration; import java.time.Instant; import java.util.Arrays; import java.util.Map; import java.util.Optional; -import java.util.SortedMap; import java.util.TreeMap; -import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -59,6 +53,8 @@ @WrapWithNettyLeakDetection public class NettyPacketToHttpConsumerTest extends InstrumentationTest { + public static final Duration REGULAR_RESPONSE_TIMEOUT = Duration.ofSeconds(30); + public static final String SERVER_RESPONSE_BODY = "I should be decrypted tester!\n"; public static final int LARGE_RESPONSE_CONTENT_LENGTH = 2 * 1024 * 1024; public static final int LARGE_RESPONSE_LENGTH = LARGE_RESPONSE_CONTENT_LENGTH + 107; @@ -155,7 +151,7 @@ public void testHttpResponseIsSuccessfullyCaptured(boolean useTls, boolean large () -> ClientConnectionPool.getCompletedChannelFutureAsCompletableFuture( httpContext.getChannelKeyContext(), NettyPacketToHttpConsumer.createClientConnection(eventLoop, sslContext, - testServer.localhostEndpoint(), channelContext))); + testServer.localhostEndpoint(), channelContext, REGULAR_RESPONSE_TIMEOUT))); var nphc = new NettyPacketToHttpConsumer(replaySession, httpContext); nphc.consumeBytes((EXPECTED_REQUEST_STRING).getBytes(StandardCharsets.UTF_8)); var aggregatedResponse = nphc.finalizeRequest().get(); @@ -179,36 +175,69 @@ public void testHttpResponseIsSuccessfullyCaptured(boolean useTls, boolean large "false, true", "true, false", "true, true"}) + @Tag("longTest") @WrapWithNettyLeakDetection(repetitions = 1) public void testThatPeerResetTriggersFinalizeFuture(boolean useTls, boolean withServerReadTimeout) throws Exception { + final var RESPONSE_TIMEOUT_FOR_HUNG_TEST = Duration.ofMillis(500); + testPeerResets(useTls, withServerReadTimeout, RESPONSE_TIMEOUT_FOR_HUNG_TEST, + RESPONSE_TIMEOUT_FOR_HUNG_TEST.plus(Duration.ofMillis(250))); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + @Tag("longTest") + @WrapWithNettyLeakDetection(repetitions = 1) + public void testThatWithBigResponseReadTimeoutResponseWouldHang(boolean useTls) throws Exception { + testPeerResets(useTls, false, Duration.ofSeconds(30), Duration.ofMillis(1250)); + } + + private void testPeerResets(boolean useTls, boolean withServerReadTimeout, + Duration readTimeout, Duration resultWaitTimeout) throws Exception { + ClientConnectionPool clientConnectionPool = null; try (var testServer = SimpleNettyHttpServer.makeServer(useTls, withServerReadTimeout ? Duration.ofMillis(100) : null, NettyPacketToHttpConsumerTest::makeResponseContext)) { log.atError().setMessage("Got port " + testServer.port).log(); - var sslContext = !testServer.localhostEndpoint().getScheme().equalsIgnoreCase("https") ? null : + var sslContext = !useTls ? null : SslContextBuilder.forClient().trustManager(InsecureTrustManagerFactory.INSTANCE).build(); var timeShifter = new TimeShifter(); timeShifter.setFirstTimestamp(Instant.now()); - var clientConnectionPool = new ClientConnectionPool(testServer.localhostEndpoint(), sslContext, - "targetPool for testThatConnectionsAreKeptAliveAndShared", 1); - + clientConnectionPool = new ClientConnectionPool(testServer.localhostEndpoint(), sslContext, + "targetPool for testThatConnectionsAreKeptAliveAndShared", 1, + readTimeout); var reqCtx = rootContext.getTestConnectionRequestContext(1); var nphc = new NettyPacketToHttpConsumer(clientConnectionPool .buildConnectionReplaySession(reqCtx.getChannelKeyContext()), reqCtx); - nphc.consumeBytes("\r\nbadrequest\r\n".getBytes(StandardCharsets.UTF_8)); - var result = nphc.finalizeRequest().get(Duration.ofSeconds(4)); + // purposefully send ONLY the beginning of a request + nphc.consumeBytes("GET ".getBytes(StandardCharsets.UTF_8)); + if (resultWaitTimeout.minus(readTimeout).isNegative()) { + Assertions.assertThrows(TimeoutException.class, ()->nphc.finalizeRequest().get(resultWaitTimeout)); + return; + } + var result = nphc.finalizeRequest().get(resultWaitTimeout); try (var is = ReplayUtils.byteArraysToInputStream(Arrays.stream(result.getCopyOfPackets())); var isr = new InputStreamReader(is); var br = new BufferedReader(isr)) { Assertions.assertEquals("", Optional.ofNullable(br.readLine()).orElse("")); Assertions.assertEquals(0, result.getResponseSizeInBytes()); } - var stopFuture = clientConnectionPool.shutdownNow(); - log.info("waiting for factory to shutdown: " + stopFuture); - stopFuture.get(); - log.info("done shutting down"); + if (withServerReadTimeout) { + log.trace("An empty response is all that we'll get. " + + "There won't be any packets coming back, so nothing will be accumulated and eventually " + + "the connection closes - so the above checks are sufficient"); + } else { + Assertions.assertInstanceOf(ReadTimeoutException.class, result.getError()); + } + } finally { + if (clientConnectionPool != null) { + var stopFuture = clientConnectionPool.shutdownNow(); + log.info("waiting for factory to shutdown: " + stopFuture); + stopFuture.get(); + log.info("done shutting down"); + } + } } @@ -230,7 +259,7 @@ public void testThatConnectionsAreKeptAliveAndShared(boolean useTls, boolean lar var timeShifter = new TimeShifter(); timeShifter.setFirstTimestamp(Instant.now()); var clientConnectionPool = new ClientConnectionPool(testServer.localhostEndpoint(), sslContext, - "targetPool for testThatConnectionsAreKeptAliveAndShared", 1); + "targetPool for testThatConnectionsAreKeptAliveAndShared", 1, REGULAR_RESPONSE_TIMEOUT); var sendingFactory = new ReplayEngine(new RequestSenderOrchestrator(clientConnectionPool), new TestFlowController(), timeShifter); for (int j = 0; j < 2; ++j) { diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/FullReplayerWithTracingChecksTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/FullReplayerWithTracingChecksTest.java index 26864435bf..ff3c2b6787 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/FullReplayerWithTracingChecksTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/FullReplayerWithTracingChecksTest.java @@ -8,6 +8,7 @@ import org.junit.jupiter.api.parallel.ResourceLock; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; +import org.opensearch.migrations.replay.RootReplayerConstructorExtensions; import org.opensearch.migrations.replay.TestHttpServerContext; import org.opensearch.migrations.replay.TimeShifter; import org.opensearch.migrations.replay.TrafficReplayerTopLevel; @@ -88,10 +89,10 @@ public void testStreamWithRequestsWithCloseIsCommittedOnce(int numRequests) thro var tuplesReceived = new HashSet(); var serverUri = httpServer.localhostEndpoint(); - try (var tr = new TrafficReplayerTopLevel(rootContext, serverUri, + try (var tr = new RootReplayerConstructorExtensions(rootContext, serverUri, new StaticAuthTransformerFactory("TEST"), - new TransformationLoader() - .getTransformerFactoryLoader(serverUri.getHost()), true, 10, 10 * 1024 + new TransformationLoader().getTransformerFactoryLoader(serverUri.getHost()), + RootReplayerConstructorExtensions.makeClientConnectionPool(serverUri, 10), 10 * 1024 ); var blockingTrafficSource = new BlockingTrafficSource(trafficSource, Duration.ofMinutes(2))) { tr.setupRunAndWaitForReplayToFinish(Duration.ofSeconds(70), blockingTrafficSource, diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/FullTrafficReplayerTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/FullTrafficReplayerTest.java index 47889b3bf0..398e73cc9e 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/FullTrafficReplayerTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/FullTrafficReplayerTest.java @@ -10,6 +10,7 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.CsvSource; import org.opensearch.migrations.replay.CapturedTrafficToHttpTransactionAccumulator; +import org.opensearch.migrations.replay.ClientConnectionPool; import org.opensearch.migrations.replay.ReplayEngine; import org.opensearch.migrations.replay.SourceTargetCaptureTuple; import org.opensearch.migrations.replay.TestHttpServerContext; @@ -26,6 +27,8 @@ import org.opensearch.migrations.replay.traffic.source.ISimpleTrafficCaptureSource; import org.opensearch.migrations.replay.traffic.source.ITrafficStreamWithKey; import org.opensearch.migrations.replay.traffic.source.TrafficStreamCursorKey; +import org.opensearch.migrations.replay.traffic.source.TrafficStreamLimiter; +import org.opensearch.migrations.replay.util.OrderedWorkerTracker; import org.opensearch.migrations.testutils.SimpleNettyHttpServer; import org.opensearch.migrations.testutils.WrapWithNettyLeakDetection; import org.opensearch.migrations.tracing.InstrumentationTest; @@ -80,9 +83,10 @@ public TrafficReplayerWithWaitOnClose(Duration maxWaitTime, int maxConcurrentOutstandingRequests, IJsonTransformer jsonTransformer, String targetConnectionPoolName) throws SSLException { - super(context, serverUri, authTransformerFactory, - jsonTransformer, allowInsecureConnections, numSendingThreads, maxConcurrentOutstandingRequests, - targetConnectionPoolName); + super(context, serverUri, authTransformerFactory, jsonTransformer, + TrafficReplayerTopLevel.makeClientConnectionPool(serverUri, allowInsecureConnections, numSendingThreads, + targetConnectionPoolName, Duration.ofSeconds(30)), + new TrafficStreamLimiter(maxConcurrentOutstandingRequests), new OrderedWorkerTracker<>()); this.maxWaitTime = maxWaitTime; } diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/SlowAndExpiredTrafficStreamBecomesTwoTargetChannelsTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/SlowAndExpiredTrafficStreamBecomesTwoTargetChannelsTest.java index da0f44d986..16866bd39a 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/SlowAndExpiredTrafficStreamBecomesTwoTargetChannelsTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/SlowAndExpiredTrafficStreamBecomesTwoTargetChannelsTest.java @@ -8,7 +8,7 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.opensearch.migrations.replay.TimeShifter; -import org.opensearch.migrations.replay.TrafficReplayerTopLevel; +import org.opensearch.migrations.replay.RootReplayerConstructorExtensions; import org.opensearch.migrations.replay.TransformationLoader; import org.opensearch.migrations.replay.traffic.source.ArrayCursorTrafficCaptureSource; import org.opensearch.migrations.replay.traffic.source.ArrayCursorTrafficSourceContext; @@ -97,12 +97,14 @@ public void test() throws Exception { var trafficSource = new BlockingTrafficSource(arraySource, Duration.ofSeconds(SPACING_SECONDS)); try (var httpServer = SimpleNettyHttpServer.makeServer(false, Duration.ofMillis(200), responseTracker); - var replayer = new TrafficReplayerTopLevel(rc, httpServer.localhostEndpoint(), + var replayer = new RootReplayerConstructorExtensions(rc, httpServer.localhostEndpoint(), new StaticAuthTransformerFactory("TEST"), - new TransformationLoader().getTransformerFactoryLoader("localhost"), true, 1, 1, - "targetConnectionPool for SlowAndExpiredTrafficStreamBecomesTwoTargetChannelsTest")) { + new TransformationLoader().getTransformerFactoryLoader("localhost"), + RootReplayerConstructorExtensions.makeClientConnectionPool(httpServer.localhostEndpoint(), true, + 0, "targetConnectionPool for SlowAndExpiredTrafficStreamBecomesTwoTargetChannelsTest", + Duration.ofSeconds(30)))) { new Thread(()->responseTracker.onCountDownFinished(Duration.ofSeconds(10), - ()->replayer.shutdown(null).join())); + ()->replayer.shutdown(null).join())); replayer.setupRunAndWaitForReplayWithShutdownChecks(Duration.ofMillis(1), trafficSource, new TimeShifter(TIME_SPEEDUP_FACTOR), t->{}); } diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/TrafficReplayerRunner.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/TrafficReplayerRunner.java index 08c94fe1fc..1e312f1e25 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/TrafficReplayerRunner.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/TrafficReplayerRunner.java @@ -4,6 +4,7 @@ import lombok.AllArgsConstructor; import lombok.extern.slf4j.Slf4j; import org.junit.jupiter.api.Assertions; +import org.opensearch.migrations.replay.RootReplayerConstructorExtensions; import org.opensearch.migrations.replay.SourceTargetCaptureTuple; import org.opensearch.migrations.replay.TimeShifter; import org.opensearch.migrations.replay.TrafficReplayer; @@ -65,10 +66,10 @@ public static void runReplayer(int numExpectedRequests, runReplayer(numExpectedRequests, (rootContext, targetConnectionPoolPrefix) -> { try { - return new TrafficReplayerTopLevel(rootContext, endpoint, + return new RootReplayerConstructorExtensions(rootContext, endpoint, new StaticAuthTransformerFactory("TEST"), - new TransformationLoader().getTransformerFactoryLoader(endpoint.getHost()), true, 10, 10*1024, - targetConnectionPoolPrefix); + new TransformationLoader().getTransformerFactoryLoader(endpoint.getHost()), + RootReplayerConstructorExtensions.makeClientConnectionPool(endpoint, targetConnectionPoolPrefix)); } catch (SSLException e) { throw new RuntimeException(e); } diff --git a/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/RootReplayerConstructorExtensions.java b/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/RootReplayerConstructorExtensions.java new file mode 100644 index 0000000000..4e56a3fcc2 --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/RootReplayerConstructorExtensions.java @@ -0,0 +1,69 @@ +package org.opensearch.migrations.replay; + +import org.opensearch.migrations.replay.tracing.IRootReplayerContext; +import org.opensearch.migrations.replay.traffic.source.TrafficStreamLimiter; +import org.opensearch.migrations.transform.IAuthTransformerFactory; +import org.opensearch.migrations.transform.IJsonTransformer; + +import javax.net.ssl.SSLException; +import java.net.URI; +import java.time.Duration; + +public class RootReplayerConstructorExtensions extends TrafficReplayerTopLevel { + + public static final Duration RESPONSE_TIMEOUT = Duration.ofSeconds(30); + + public RootReplayerConstructorExtensions(IRootReplayerContext topContext, + URI uri, + IAuthTransformerFactory authTransformerFactory, + IJsonTransformer jsonTransformer, + ClientConnectionPool clientConnectionPool) { + this(topContext, uri, authTransformerFactory, jsonTransformer, clientConnectionPool, 1024); + } + + public RootReplayerConstructorExtensions(IRootReplayerContext topContext, + URI uri, + IAuthTransformerFactory authTransformer, + IJsonTransformer jsonTransformer, + ClientConnectionPool clientConnectionPool, + int maxConcurrentRequests) { + this(topContext, uri, authTransformer, jsonTransformer, clientConnectionPool, + maxConcurrentRequests, new TrafficReplayerTopLevel.ConcurrentHashMapWorkTracker<>()); + } + + public RootReplayerConstructorExtensions(IRootReplayerContext context, + URI serverUri, + IAuthTransformerFactory authTransformerFactory, + IJsonTransformer jsonTransformer, + ClientConnectionPool clientConnectionPool, + int maxConcurrentOutstandingRequests, + TrafficReplayerTopLevel.IStreamableWorkTracker workTracker) { + super(context, serverUri, authTransformerFactory, jsonTransformer, clientConnectionPool, + new TrafficStreamLimiter(maxConcurrentOutstandingRequests), workTracker); + } + + public static ClientConnectionPool makeClientConnectionPool(URI serverUri) throws SSLException { + return makeClientConnectionPool(serverUri, null, RESPONSE_TIMEOUT); + } + + public static ClientConnectionPool makeClientConnectionPool(URI serverUri, String poolPrefix) throws SSLException { + return makeClientConnectionPool(serverUri, poolPrefix, RESPONSE_TIMEOUT); + } + + public static ClientConnectionPool makeClientConnectionPool(URI serverUri, + String poolPrefix, + Duration timeout) throws SSLException { + return makeClientConnectionPool(serverUri, true, 0, poolPrefix, timeout); + } + + public static ClientConnectionPool makeClientConnectionPool(URI serverUri, + int numSendingThreads) throws SSLException { + return makeClientConnectionPool(serverUri, numSendingThreads, RESPONSE_TIMEOUT); + } + + public static ClientConnectionPool makeClientConnectionPool(URI serverUri, + int numSendingThreads, + Duration timeout) throws SSLException { + return makeClientConnectionPool(serverUri, true, numSendingThreads, null, timeout); + } +} From 1d916bc922a0cced3631f6164eab6fe2f7b4cf5f Mon Sep 17 00:00:00 2001 From: Greg Schohn Date: Tue, 16 Apr 2024 09:47:29 -0400 Subject: [PATCH 3/3] Fix log level Signed-off-by: Greg Schohn --- .../trafficReplayer/src/main/resources/log4j2.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/TrafficCapture/trafficReplayer/src/main/resources/log4j2.properties b/TrafficCapture/trafficReplayer/src/main/resources/log4j2.properties index d4e48d2421..e495efd21e 100644 --- a/TrafficCapture/trafficReplayer/src/main/resources/log4j2.properties +++ b/TrafficCapture/trafficReplayer/src/main/resources/log4j2.properties @@ -66,7 +66,7 @@ appender.ALL_ACTIVE_WORK_MONITOR_LOGFILE.policies.time.modulate = true appender.ALL_ACTIVE_WORK_MONITOR_LOGFILE.strategy.type = DefaultRolloverStrategy appender.ALL_ACTIVE_WORK_MONITOR_LOGFILE.strategy.max = 4 -rootLogger.level = trace +rootLogger.level = info rootLogger.appenderRef.STDERR.ref = STDERR rootLogger.appenderRef.ReplayerLogFile.ref = ReplayerLogFile