From 93e6c5ab3fefc9791f21ef54a93c989685abc8ec Mon Sep 17 00:00:00 2001 From: Greg Schohn Date: Thu, 11 Apr 2024 12:32:24 -0400 Subject: [PATCH 01/11] Assorted bugfixes A memory leak for errantly formed http messages going through HttpByteBufFormatter; log4j2 config bug that stopped old replayer logs from compressing. I've also further improved the logging on shutdown. Signed-off-by: Greg Schohn --- .../replay/HttpByteBufFormatter.java | 6 +++- .../migrations/replay/TrafficReplayer.java | 19 ++++++++--- .../traffic/source/TrafficStreamLimiter.java | 33 +++++++++++++------ .../src/main/resources/log4j2.properties | 2 +- 4 files changed, 43 insertions(+), 17 deletions(-) diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/HttpByteBufFormatter.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/HttpByteBufFormatter.java index a7d8065c2..a258d033b 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/HttpByteBufFormatter.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/HttpByteBufFormatter.java @@ -171,7 +171,11 @@ public static HttpMessage parseHttpMessageFromBufs(HttpMessageType msgType, Stre } }); - return channel.readInbound(); + try { + return channel.readInbound(); + } finally { + channel.finishAndReleaseAll(); + } } public static FullHttpRequest parseHttpRequestFromBufs(Stream byteBufStream, boolean releaseByteBufs) { 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 3b6b94031..7a628f2db 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 @@ -288,6 +288,7 @@ public static void main(String[] args) throws Exception { RootOtelContext.initializeOpenTelemetryWithCollectorOrAsNoop(params.otelCollectorEndpoint, "replay"), contextTrackers); + ActiveContextMonitor activeContextMonitor = null; try (var blockingTrafficSource = TrafficCaptureSourceFactory.createTrafficCaptureSource(topContext, params, Duration.ofSeconds(params.lookaheadTimeSeconds)); var authTransformer = buildAuthTransformerFactory(params)) @@ -301,13 +302,14 @@ public static void main(String[] args) throws Exception { new TransformationLoader().getTransformerFactoryLoader(uri.getHost(), params.userAgent, transformerConfig), params.allowInsecureConnections, params.numClientThreads, params.maxConcurrentRequests, orderedRequestTracker); - var activeContextMonitor = new ActiveContextMonitor( + activeContextMonitor = new ActiveContextMonitor( globalContextTracker, perContextTracker, orderedRequestTracker, 64, cf->cf.formatAsString(TrafficReplayerTopLevel::formatWorkItem), activeContextLogger); - scheduledExecutorService.scheduleAtFixedRate(()->{ - activeContextLogger.atInfo().setMessage(()->"Total requests outstanding: " + tr.requestWorkTracker.size()).log(); - activeContextMonitor.run(); - }, + ActiveContextMonitor finalActiveContextMonitor = activeContextMonitor; + 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); @@ -318,6 +320,13 @@ public static void main(String[] args) throws Exception { log.info("Done processing TrafficStreams"); } finally { scheduledExecutorService.shutdown(); + if (activeContextMonitor != null) { + var acmLevel = globalContextTracker.getActiveScopesByAge().findAny().isPresent() ? + Level.ERROR : Level.INFO; + activeContextLogger.atLevel(acmLevel).setMessage(()->"Outstanding work after shutdown...").log(); + activeContextMonitor.run(); + activeContextLogger.atLevel(acmLevel).setMessage(()->"[end of run]]").log(); + } } } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/TrafficStreamLimiter.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/TrafficStreamLimiter.java index 008eca459..e64ef99a3 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/TrafficStreamLimiter.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/TrafficStreamLimiter.java @@ -42,16 +42,29 @@ public boolean isStopped() { @SneakyThrows private void consumeFromQueue() { - while (!stopped.get()) { - var workItem = workQueue.take(); - log.atDebug().setMessage(()->"liveTrafficStreamCostGate.permits: {} acquiring: {}") - .addArgument(liveTrafficStreamCostGate.availablePermits()) - .addArgument(workItem.cost) - .log(); - liveTrafficStreamCostGate.acquire(workItem.cost); - log.atDebug().setMessage(()->"Acquired liveTrafficStreamCostGate (available=" + - liveTrafficStreamCostGate.availablePermits()+") to process " + workItem.context).log(); - workItem.task.accept(workItem); + WorkItem workItem = null; + try { + while (!stopped.get()) { + workItem = workQueue.take(); + log.atDebug().setMessage(() -> "liveTrafficStreamCostGate.permits: {} acquiring: {}") + .addArgument(liveTrafficStreamCostGate.availablePermits()) + .addArgument(workItem.cost) + .log(); + liveTrafficStreamCostGate.acquire(workItem.cost); + WorkItem finalWorkItem = workItem; + log.atDebug().setMessage(() -> "Acquired liveTrafficStreamCostGate (available=" + + liveTrafficStreamCostGate.availablePermits() + ") to process " + finalWorkItem.context).log(); + workItem.task.accept(workItem); + workItem = null; + } + } catch (InterruptedException e) { + if (!stopped.get()) { + WorkItem finalWorkItem = workItem; + log.atError().setMessage(()->"consumeFromQueue() was interrupted with " + + (finalWorkItem != null ? "an active task and " : "") + + workQueue.size() + " enqueued items").log(); + } + throw e; } } diff --git a/TrafficCapture/trafficReplayer/src/main/resources/log4j2.properties b/TrafficCapture/trafficReplayer/src/main/resources/log4j2.properties index ffccfb189..d4e48d242 100644 --- a/TrafficCapture/trafficReplayer/src/main/resources/log4j2.properties +++ b/TrafficCapture/trafficReplayer/src/main/resources/log4j2.properties @@ -13,7 +13,7 @@ appender.console.layout.pattern = [%-5level] %d{yyyy-MM-dd HH:mm:ss,SSS}{UTC} [% appender.ReplayerLogFile.type = RollingFile appender.ReplayerLogFile.name = ReplayerLogFile appender.ReplayerLogFile.fileName = logs/replayer.log -appender.ReplayerLogFile.filePattern = logs/%d{yyyy-MM}{UTC}/replayer-%d{yyyy-MM-dd-HH-mm}{UTC}-%i.log +appender.ReplayerLogFile.filePattern = logs/%d{yyyy-MM}{UTC}/replayer-%d{yyyy-MM-dd-HH-mm}{UTC}-%i.log.gz appender.ReplayerLogFile.layout.type = PatternLayout appender.ReplayerLogFile.layout.pattern = [%-5level] %d{yyyy-MM-dd HH:mm:ss,SSS}{UTC} [%t] %c{1} - %msg%equals{ ctx=%mdc}{ ctx={}}{}%n appender.ReplayerLogFile.policies.type = Policies From 506d9cf9a6e610c2cdab23b50e65256bb5e87666 Mon Sep 17 00:00:00 2001 From: Greg Schohn Date: Mon, 15 Apr 2024 11:27:15 -0400 Subject: [PATCH 02/11] Clean up a lot of future dependency graphs. 1) Netty futures are now bound to CompletableFutures via a utility function that has the netty future simply mark a CompletableFuture as finished (with the null value) or propagate the exception into the CompletableFuture. That allows the rest of the codebase to leverage CompletableFuture and DiagnosticTrackableCompletableFuture objects and methods rather than a cobbled approach. 2) Helper classes that managed coordination and scheduling (TimeToResponseFulfillmentFutureMap and OnlineRadixSorter) have been redesigned to be more functional so that future values can be derived from earlier values rather than rely upon side effects to connect the dots. Some tests still don't pass and intense testing will be required before these changes are better than what was previously there. Signed-off-by: Greg Schohn --- .../replay/ClientConnectionPool.java | 6 +- .../replay/RequestSenderOrchestrator.java | 335 ++++++++++-------- .../replay/TrafficReplayerCore.java | 75 ++-- .../NettyPacketToHttpConsumer.java | 2 +- .../replay/datatypes/ChannelTask.java | 4 +- .../datatypes/ConnectionReplaySession.java | 16 +- .../replay/datatypes/FutureTransformer.java | 13 + .../TimeToResponseFulfillmentFutureMap.java | 29 +- .../DiagnosticTrackableCompletableFuture.java | 5 + .../replay/util/OnlineRadixSorter.java | 120 ++++--- .../StringTrackableCompletableFuture.java | 13 + ...imeToResponseFulfillmentFutureMapTest.java | 18 +- .../replay/util/OnlineRadixSorterTest.java | 30 +- .../OnlineRadixSorterForIntegratedKeys.java | 8 +- 14 files changed, 411 insertions(+), 263 deletions(-) create mode 100644 TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/FutureTransformer.java rename TrafficCapture/trafficReplayer/src/{main => testFixtures}/java/org/opensearch/migrations/replay/util/OnlineRadixSorterForIntegratedKeys.java (68%) 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 bcd49257a..45e3055cd 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 @@ -83,8 +83,7 @@ public ConnectionReplaySession buildConnectionReplaySession(final IReplayContext getCompletedChannelFutureAsCompletableFuture(IReplayContexts.IChannelKeyContext connectionContext, ChannelFuture channelFuture) { var clientConnectionChannelCreatedFuture = - new StringTrackableCompletableFuture(new CompletableFuture<>(), - () -> "waiting for createClientConnection to finish"); + new StringTrackableCompletableFuture("waiting for createClientConnection to finish"); channelFuture.addListener(f -> { log.atInfo().setMessage(()-> "New network connection result for " + connectionContext + "=" + f.isSuccess()).log(); @@ -136,8 +135,7 @@ public void closeConnection(IReplayContexts.IChannelKeyContext ctx, int sessionN private DiagnosticTrackableCompletableFuture closeClientConnectionChannel(ConnectionReplaySession channelAndFutureWork) { var channelClosedFuture = - new StringTrackableCompletableFuture(new CompletableFuture<>(), - ()->"Waiting for closeFuture() on channel"); + new StringTrackableCompletableFuture("Waiting for closeFuture() on channel"); channelAndFutureWork.getFutureThatReturnsChannelFuture(false) .thenAccept(channelFuture-> { diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/RequestSenderOrchestrator.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/RequestSenderOrchestrator.java index 381f4dd83..19fe363d3 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/RequestSenderOrchestrator.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/RequestSenderOrchestrator.java @@ -13,7 +13,6 @@ import org.opensearch.migrations.replay.tracing.IReplayContexts; import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; import org.opensearch.migrations.replay.util.StringTrackableCompletableFuture; -import org.slf4j.event.Level; import java.time.Duration; import java.time.Instant; @@ -22,13 +21,86 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Consumer; +import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Stream; @Slf4j public class RequestSenderOrchestrator { + public static CompletableFuture + bindNettyFutureToCompletableFuture(Future nettyFuture, CompletableFuture cf) { + nettyFuture.addListener(f -> { + if (!f.isSuccess()) { + cf.completeExceptionally(f.cause()); + } else { + cf.complete(null); + } + }); + return cf; + } + + public static CompletableFuture + bindNettyFutureToCompletableFuture(Future nettyFuture) { + return bindNettyFutureToCompletableFuture(nettyFuture, new CompletableFuture<>()); + } + + public static DiagnosticTrackableCompletableFuture + bindNettyFutureToTrackableFuture(Future nettyFuture, String label) { + return new StringTrackableCompletableFuture<>(bindNettyFutureToCompletableFuture(nettyFuture), label); + } + + public static DiagnosticTrackableCompletableFuture + bindNettyFutureToTrackableFuture(Future nettyFuture, Supplier labelProvider) { + return new StringTrackableCompletableFuture<>(bindNettyFutureToCompletableFuture(nettyFuture), labelProvider); + } + + public static DiagnosticTrackableCompletableFuture + bindNettyFutureToTrackableFuture(Function> nettyFutureGenerator, String label) { + return bindNettyFutureToTrackableFuture(nettyFutureGenerator.apply(()->{}), label); + } + + public static DiagnosticTrackableCompletableFuture + bindNettySubmitToTrackableFuture(EventLoop eventLoop) { + return bindNettyFutureToTrackableFuture(eventLoop::submit, "waiting for event loop submission"); + } + + public static DiagnosticTrackableCompletableFuture + bindNettyScheduleToCompletableFuture(EventLoop eventLoop, Duration delay) { + var delayMs = Math.max(0, delay.toMillis()); + return bindNettyFutureToTrackableFuture(eventLoop.schedule(()->{}, delayMs, TimeUnit.MILLISECONDS), + "scheduling to run next send in " + delay); + } + + private DiagnosticTrackableCompletableFuture + bindNettyScheduleToCompletableFuture(EventLoop eventLoop, Instant timestamp) { + return bindNettyScheduleToCompletableFuture(eventLoop, getDelayFromNowMs(timestamp)); + } + + public static CompletableFuture + bindNettyScheduleToCompletableFuture(EventLoop eventLoop, Duration delay, CompletableFuture cf) { + var delayMs = Math.max(0, delay.toMillis()); + return bindNettyFutureToCompletableFuture(eventLoop.schedule(()->{}, delayMs, TimeUnit.MILLISECONDS), cf); + } + + private CompletableFuture + bindNettyScheduleToCompletableFuture(EventLoop eventLoop, Instant timestamp, CompletableFuture cf) { + return bindNettyScheduleToCompletableFuture(eventLoop, getDelayFromNowMs(timestamp), cf); + } + + + public void + bindNettyFutureToTrackableFuture(EventLoop eventLoop, + Instant timestamp, + DiagnosticTrackableCompletableFuture existingFuture) { + var delayMs = Math.max(0, getDelayFromNowMs(timestamp).toMillis()); + var scheduleFuture = eventLoop.schedule(()->{}, delayMs, TimeUnit.MILLISECONDS); + new StringTrackableCompletableFuture<>( + bindNettyFutureToCompletableFuture(scheduleFuture, existingFuture.future), + "scheduling to run next send in " + timestamp); + } + + public final ClientConnectionPool clientConnectionPool; public RequestSenderOrchestrator(ClientConnectionPool clientConnectionPool) { @@ -40,201 +112,186 @@ public RequestSenderOrchestrator(ClientConnectionPool clientConnectionPool) { Supplier> task) { var connectionSession = clientConnectionPool.getCachedSession(ctx.getChannelKeyContext(), ctx.getReplayerRequestKey().sourceRequestIndexSessionIdentifier); - var finalTunneledResponse = - new StringTrackableCompletableFuture(new CompletableFuture<>(), - ()->"waiting for final signal to confirm processing work has finished"); log.atDebug().setMessage(()->"Scheduling work for "+ctx.getConnectionId()+" at time "+timestamp).log(); var scheduledContext = ctx.createScheduledContext(timestamp); - // this method doesn't use the scheduling that scheduleRequest and scheduleClose use because + // This method doesn't use the scheduling that scheduleRequest and scheduleClose use because // doing work associated with a connection is considered to be preprocessing work independent // of the underlying network connection itself, so it's fair to be able to do this without - // first needing to wait for a connection to succeed. In fact, making them more independent - // means that the work item being enqueued is less likely to cause a connection timeout. - connectionSession.eventLoop.schedule(()-> { - scheduledContext.close(); - return task.get().map(f -> f.whenComplete((v, t) -> { - if (t != null) { - finalTunneledResponse.future.completeExceptionally(t); - } else { - finalTunneledResponse.future.complete(v); - } - }), - () -> ""); - }, - getDelayFromNowMs(timestamp), TimeUnit.MILLISECONDS); - return finalTunneledResponse; + // first needing to wait for a connection to succeed. + // + // This means that this method might run transformation work "out-of-order" from the natural + // ordering of the requests (defined by their original captured order). However, the final + // order will be preserved once they're sent since sending requires the channelInteractionIndex, + // which is the caller's responsibility to track and pass. This method doesn't need it to + // schedule work to happen on the channel's thread at some point in the future. + // + // Making them more independent means that the work item being enqueued is lighter-weight and + // less likely to cause a connection timeout. + return bindNettyScheduleToCompletableFuture(connectionSession.eventLoop, timestamp) + .getDeferredFutureThroughHandle((nullValue,scheduleFailure)-> { + scheduledContext.close(); + if (scheduleFailure == null) { + return task.get(); + } else { + return StringTrackableCompletableFuture.failedFuture(scheduleFailure, ()->"netty scheduling failure"); + } + }, ()->"The scheduled callback is running"); } public DiagnosticTrackableCompletableFuture scheduleRequest(UniqueReplayerRequestKey requestKey, IReplayContexts.IReplayerHttpTransactionContext ctx, Instant start, Duration interval, Stream packets) { - var finalTunneledResponse = - new StringTrackableCompletableFuture(new CompletableFuture<>(), - ()->"waiting for final aggregated response"); - // When a socket connection is attempted could be more precise. + var sessionNumber = requestKey.sourceRequestIndexSessionIdentifier; + var channelInteractionNum = requestKey.getReplayerRequestIndex(); + // TODO: Separate socket connection from the first bytes sent. // Ideally, we would match the relative timestamps of when connections were being initiated // as well as the period between connection and the first bytes sent. However, this code is a - // bit too cavalier. It should be tightened at some point. - return asynchronouslyInvokeRunnable(ctx.getLogicalEnclosingScope(), - requestKey.sourceRequestIndexSessionIdentifier, - requestKey.getReplayerRequestIndex(), false, finalTunneledResponse, - channelFutureAndRequestSchedule -> scheduleSendRequestOnConnectionReplaySession(ctx, - channelFutureAndRequestSchedule, finalTunneledResponse, start, interval, packets)); - } - - public StringTrackableCompletableFuture scheduleClose(IReplayContexts.IChannelKeyContext ctx, - int sessionNumber, - int channelInteractionNum, - Instant timestamp) { + // bit too cavalier. It should be tightened at some point by adding a first packet that is empty. + // Thankfully, given the trickiness of this class, that would be something that should be tracked + // upstream and should be handled transparently by this class. + return submitUnorderedWorkToEventLoop(ctx.getLogicalEnclosingScope(), sessionNumber, channelInteractionNum, + connectionReplaySession -> scheduleSendRequestOnConnectionReplaySession(ctx, + connectionReplaySession, start, interval, packets)); + } + + public DiagnosticTrackableCompletableFuture scheduleClose(IReplayContexts.IChannelKeyContext ctx, + int sessionNumber, + int channelInteractionNum, + Instant timestamp) { var channelKey = ctx.getChannelKey(); var channelInteraction = new IndexedChannelInteraction(channelKey, channelInteractionNum); - var finalTunneledResponse = - new StringTrackableCompletableFuture(new CompletableFuture<>(), - ()->"waiting for final signal to confirm close has finished"); log.atDebug().setMessage(() -> "Scheduling CLOSE for " + channelInteraction + " at time " + timestamp).log(); - asynchronouslyInvokeRunnable(ctx, sessionNumber, channelInteractionNum, true, - finalTunneledResponse, - channelFutureAndRequestSchedule-> - scheduleOnConnectionReplaySession(ctx, channelInteractionNum, - channelFutureAndRequestSchedule, finalTunneledResponse, timestamp, - new ChannelTask(ChannelTaskType.CLOSE, () -> { - log.trace("Closing client connection " + channelInteraction); - clientConnectionPool.closeConnection(ctx, sessionNumber); - finalTunneledResponse.future.complete(null); - }))); - return finalTunneledResponse; + return submitUnorderedWorkToEventLoop(ctx, sessionNumber, channelInteractionNum, + connectionReplaySession -> scheduleCloseOnConnectionReplaySession(ctx, + connectionReplaySession, timestamp, sessionNumber, channelInteractionNum, channelInteraction)); } /** - * This method sets up the onSessionCallback to run in the order defined by channeInteractionNumber. The + * This method will run the callback on the connection's dedicated thread such that all of the executions + * of the callbacks sent for the connection are in the order defined by channelInteractionNumber, whose + * values must be of the entire set of ints [0,N] for N work items (so, 0,1,2. no gaps, no dups). The * onSessionCallback task passed will be called only after all callbacks for previous channelInteractionNumbers - * have been called. This method isn't concerned with scheduling items, that would be left up to the callback. + * have been called. This method isn't concerned with scheduling items to run at a specific time, that is + * left up to the callback. */ private DiagnosticTrackableCompletableFuture - asynchronouslyInvokeRunnable(IReplayContexts.IChannelKeyContext ctx, int sessionNumber, - int channelInteractionNumber, - boolean ignoreIfChannelNotActive, - DiagnosticTrackableCompletableFuture finalTunneledResponse, - Consumer onSessionCallback) { + submitUnorderedWorkToEventLoop(IReplayContexts.IChannelKeyContext ctx, + int sessionNumber, + int channelInteractionNumber, + Function> + onSessionCallback) { final var replaySession = clientConnectionPool.getCachedSession(ctx, sessionNumber); - replaySession.eventLoop.submit(()->{ + return bindNettySubmitToTrackableFuture(replaySession.eventLoop) + .getDeferredFutureThroughHandle((v,t) -> { log.atTrace().setMessage(() -> "adding work item at slot " + channelInteractionNumber + " for " + replaySession.getChannelKeyContext() + " with " + replaySession.scheduleSequencer).log(); - replaySession.scheduleSequencer.add(channelInteractionNumber, - () -> onSessionCallback.accept(replaySession), - Runnable::run); - log.atLevel(replaySession.scheduleSequencer.hasPending() ? Level.DEBUG : Level.TRACE) - .setMessage(() -> "Sequencer for " + replaySession.getChannelKeyContext() + " = " + - replaySession.scheduleSequencer).log(); - }); - return finalTunneledResponse; + return replaySession.scheduleSequencer.addFutureForWork(channelInteractionNumber, + f->f.thenCompose(voidValue -> + onSessionCallback.apply(replaySession), ()->"Work callback on replay session")); + }, () -> "Waiting for sequencer to run for slot " + channelInteractionNumber); } - private void scheduleSendRequestOnConnectionReplaySession(IReplayContexts.IReplayerHttpTransactionContext ctx, - ConnectionReplaySession connectionReplaySession, - StringTrackableCompletableFuture - responseFuture, - Instant start, - Duration interval, - Stream packets) { + private DiagnosticTrackableCompletableFuture + scheduleSendRequestOnConnectionReplaySession(IReplayContexts.IReplayerHttpTransactionContext ctx, + ConnectionReplaySession connectionReplaySession, + Instant startTime, + Duration interval, + Stream packets) { var eventLoop = connectionReplaySession.eventLoop; - var scheduledContext = ctx.createScheduledContext(start); - scheduleOnConnectionReplaySession(ctx.getLogicalEnclosingScope(), - ctx.getReplayerRequestKey().getSourceRequestIndex(), connectionReplaySession, responseFuture, start, - new ChannelTask(ChannelTaskType.TRANSMIT, ()->{ - scheduledContext.close(); - sendNextPartAndContinue(new NettyPacketToHttpConsumer(connectionReplaySession, ctx), - eventLoop, packets.iterator(), start, interval, new AtomicInteger(), responseFuture); - })); - } - - private void scheduleOnConnectionReplaySession(IReplayContexts.IChannelKeyContext ctx, - int channelInteractionIdx, - ConnectionReplaySession channelFutureAndRequestSchedule, - StringTrackableCompletableFuture futureToBeCompletedByTask, - Instant atTime, - ChannelTask task) { - var channelInteraction = new IndexedChannelInteraction(ctx.getChannelKey(), channelInteractionIdx); + var scheduledContext = ctx.createScheduledContext(startTime); + int channelInterationNum = ctx.getReplayerRequestKey().getSourceRequestIndex(); + var diagnosticCtx = + new IndexedChannelInteraction(ctx.getLogicalEnclosingScope().getChannelKey(), channelInterationNum); + return scheduleOnConnectionReplaySession(diagnosticCtx, connectionReplaySession, startTime, + new ChannelTask<>(ChannelTaskType.TRANSMIT, trigger -> + trigger.thenCompose(voidVal -> { + scheduledContext.close(); + return sendSendingRestOfPackets(new NettyPacketToHttpConsumer(connectionReplaySession, ctx), + eventLoop, packets.iterator(), startTime, interval, new AtomicInteger()); + }, ()->"sending next packets"))); + } + + private DiagnosticTrackableCompletableFuture + scheduleCloseOnConnectionReplaySession(IReplayContexts.IChannelKeyContext ctx, + ConnectionReplaySession connectionReplaySession, + Instant timestamp, + int connectionReplaySessionNum, + int channelInteractionNum, + IndexedChannelInteraction channelInteraction) { + var diagnosticCtx = new IndexedChannelInteraction(ctx.getChannelKey(), channelInteractionNum); + return scheduleOnConnectionReplaySession(diagnosticCtx, connectionReplaySession, timestamp, + new ChannelTask<>(ChannelTaskType.CLOSE, + dcf -> dcf.whenComplete((v,t) -> { + log.trace("Calling closeConnection at slot " + channelInteraction); + clientConnectionPool.closeConnection(ctx, connectionReplaySessionNum); + }, () -> "Close connection") + )); + } + + private DiagnosticTrackableCompletableFuture + scheduleOnConnectionReplaySession(IndexedChannelInteraction channelInteraction, + ConnectionReplaySession channelFutureAndRequestSchedule, + Instant atTime, + ChannelTask task) { log.atInfo().setMessage(()->channelInteraction + " scheduling " + task.kind + " at " + atTime).log(); var schedule = channelFutureAndRequestSchedule.schedule; var eventLoop = channelFutureAndRequestSchedule.eventLoop; - if (schedule.isEmpty()) { - var scheduledFuture = - eventLoop.schedule(task.runnable, getDelayFromNowMs(atTime), TimeUnit.MILLISECONDS); - scheduledFuture.addListener(f->{ - if (!f.isSuccess()) { - log.atError().setCause(f.cause()).setMessage(()->"Error running the scheduled task: " + ctx + - " interaction: " + channelInteraction).log(); - } else { - log.atInfo().setMessage(()->"scheduled task has finished for " + ctx + " interaction: " + - channelInteraction).log(); - } - }); - } else { - assert !atTime.isBefore(schedule.peekFirstItem().getKey()) : - "Per-connection TrafficStream ordering should force a time ordering on incoming requests"; - } - - schedule.appendTask(atTime, task); + var wasEmpty = schedule.isEmpty(); + assert wasEmpty || !atTime.isBefore(schedule.peekFirstItem().getKey()) : + "Per-connection TrafficStream ordering should force a time ordering on incoming requests"; + var workPointTrigger = schedule.appendTaskTrigger(atTime, task.kind).scheduleFuture; + var workFuture = task.getRunnable().apply(workPointTrigger); log.atTrace().setMessage(()->channelInteraction + " added a scheduled event at " + atTime + "... " + schedule).log(); + if (wasEmpty) { + bindNettyScheduleToCompletableFuture(eventLoop, atTime, workPointTrigger.future); + } - futureToBeCompletedByTask.map(f->f.whenComplete((v,t)-> { + workFuture.map(f->f.whenComplete((v,t)-> { var itemStartTimeOfPopped = schedule.removeFirstItem(); assert atTime.equals(itemStartTimeOfPopped): "Expected to have popped the item to match the start time for the responseFuture that finished"; log.atDebug().setMessage(()->channelInteraction.toString() + " responseFuture completed - checking " + schedule + " for the next item to schedule").log(); Optional.ofNullable(schedule.peekFirstItem()).ifPresent(kvp-> { - var runnable = kvp.getValue().runnable; - var sf = eventLoop.schedule(runnable, getDelayFromNowMs(kvp.getKey()), TimeUnit.MILLISECONDS); - sf.addListener(sfp->{ - if (!sfp.isSuccess()) { - log.atWarn().setCause(sfp.cause()).setMessage(()->"Scheduled future did not successfully run " + - channelInteraction).log(); - } - }); + bindNettyFutureToTrackableFuture(eventLoop, kvp.getKey(), kvp.getValue().scheduleFuture); }); }), ()->""); + + return workFuture; } private Instant now() { return Instant.now(); } - private long getDelayFromNowMs(Instant to) { - return Math.max(0, Duration.between(now(), to).toMillis()); + private Duration getDelayFromNowMs(Instant to) { + return Duration.ofMillis(Math.max(0, Duration.between(now(), to).toMillis())); } - // TODO - rewrite this - the recursion (at least as it is) is terribly confusing - private void sendNextPartAndContinue(NettyPacketToHttpConsumer packetReceiver, - EventLoop eventLoop, Iterator iterator, - Instant start, Duration interval, AtomicInteger counter, - StringTrackableCompletableFuture responseFuture) { + private DiagnosticTrackableCompletableFuture + sendSendingRestOfPackets(NettyPacketToHttpConsumer packetReceiver, + EventLoop eventLoop, + Iterator iterator, + Instant startAt, + Duration interval, + AtomicInteger counter) { final var oldCounter = counter.getAndIncrement(); log.atTrace().setMessage(()->"sendNextPartAndContinue: counter=" + oldCounter).log(); assert iterator.hasNext() : "Should not have called this with no items to send"; packetReceiver.consumeBytes(iterator.next()); if (iterator.hasNext()) { - Runnable packetSender = () -> sendNextPartAndContinue(packetReceiver, eventLoop, - iterator, start, interval, counter, responseFuture); - var delayMs = Duration.between(now(), - start.plus(interval.multipliedBy(counter.get()))).toMillis(); - eventLoop.schedule(packetSender, Math.min(0, delayMs), TimeUnit.MILLISECONDS); + var delay = Duration.between(now(), + startAt.plus(interval.multipliedBy(counter.get()))); + return bindNettyScheduleToCompletableFuture(eventLoop, delay) + .getDeferredFutureThroughHandle((v,t)-> sendSendingRestOfPackets(packetReceiver, eventLoop, + iterator, startAt, interval, counter), () -> "sending next packet"); } else { - packetReceiver.finalizeRequest().handle((v,t)-> { - if (t != null) { - responseFuture.future.completeExceptionally(t); - } else { - responseFuture.future.complete(v); - } - return null; - }, ()->"waiting for finalize to send Aggregated Response"); + return packetReceiver.finalizeRequest(); } } - } 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 f7578116e..82eaee200 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 @@ -105,45 +105,46 @@ class TrafficReplayerAccumulationCallbacks implements AccumulationCallbacks { @NonNull HttpMessageAndTimestamp request) { replayEngine.setFirstTimestamp(request.getFirstPacketTimestamp()); - var allWorkFinishedForTransaction = - new StringTrackableCompletableFuture(new CompletableFuture<>(), - ()->"waiting for " + ctx + " to be queued and run through TrafficStreamLimiter"); - var requestPushFuture = new StringTrackableCompletableFuture( - new CompletableFuture<>(), () -> "Waiting to get response from target"); var requestKey = ctx.getReplayerRequestKey(); - liveTrafficStreamLimiter.queueWork(1, ctx, wi -> { - transformAndSendRequest(replayEngine, request, ctx).future.whenComplete((v,t)->{ - liveTrafficStreamLimiter.doneProcessing(wi); - if (t != null) { - requestPushFuture.future.completeExceptionally(t); - } else { - requestPushFuture.future.complete(v); - } - }); - }); - if (!allWorkFinishedForTransaction.future.isDone()) { - log.trace("Adding " + requestKey + " to targetTransactionInProgressMap"); - requestWorkTracker.put(requestKey, allWorkFinishedForTransaction); - if (allWorkFinishedForTransaction.future.isDone()) { - requestWorkTracker.remove(requestKey); - } - } - return rrPair -> - requestPushFuture.map(f -> f.handle((v, t) -> { - log.atInfo().setMessage(() -> "Done receiving captured stream for " + ctx + - ":" + rrPair.requestData).log(); - log.atTrace().setMessage(() -> - "Summary response value for " + requestKey + " returned=" + v).log(); - return handleCompletedTransaction(ctx, rrPair, v, t); - }), () -> "logging summary") - .whenComplete((v,t)->{ - if (t != null) { - allWorkFinishedForTransaction.future.completeExceptionally(t); - } else { - allWorkFinishedForTransaction.future.complete(null); - } - }, ()->""); + var finishedAccumulatingResponseFuture = + new StringTrackableCompletableFuture( + ()->"waiting for response to be accumulated for " + ctx); + finishedAccumulatingResponseFuture.future.whenComplete((v,t)-> log.atInfo() + .setMessage(() -> "Done receiving captured stream for " + ctx + ":" + v.requestData).log()); + + var allWorkFinishedForTransactionFuture = sendRequestAfterGoingThroughWorkQueue(ctx, request, requestKey) + .getDeferredFutureThroughHandle((arr,httpRequestException) -> finishedAccumulatingResponseFuture + .thenCompose(rrPair-> + StringTrackableCompletableFuture.completedFuture( + handleCompletedTransaction(ctx, rrPair, arr, httpRequestException), + ()->"Synchronously committed results"), + () -> "logging summary"), + ()->"waiting for accumulation to combine with target response"); + + assert !allWorkFinishedForTransactionFuture.future.isDone(); + log.trace("Adding " + requestKey + " to targetTransactionInProgressMap"); + requestWorkTracker.put(requestKey, allWorkFinishedForTransactionFuture); + + return finishedAccumulatingResponseFuture.future::complete; + } + + private DiagnosticTrackableCompletableFuture + sendRequestAfterGoingThroughWorkQueue(IReplayContexts.IReplayerHttpTransactionContext ctx, + HttpMessageAndTimestamp request, + UniqueReplayerRequestKey requestKey) { + var workDequeuedByLimiterFuture = + new StringTrackableCompletableFuture( + ()->"waiting for " + ctx + " to be queued and run through TrafficStreamLimiter"); + var wi = liveTrafficStreamLimiter.queueWork(1, ctx, workDequeuedByLimiterFuture.future::complete); + var httpSentRequestFuture = workDequeuedByLimiterFuture + .thenCompose(ignored -> transformAndSendRequest(replayEngine, request, ctx), + ()->"Waiting to get response from target") + .whenComplete((v,t)-> liveTrafficStreamLimiter.doneProcessing(wi), + ()->"releasing work item for the traffic limiter"); + httpSentRequestFuture.future.whenComplete((v,t)->log.atTrace().setMessage(() -> + "Summary response value for " + requestKey + " returned=" + v).log()); + return httpSentRequestFuture; } Void handleCompletedTransaction(@NonNull IReplayContexts.IReplayerHttpTransactionContext context, 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 a98a7def5..fd228bc4b 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 @@ -83,7 +83,7 @@ public NettyPacketToHttpConsumer(ConnectionReplaySession replaySession, var parentContext = ctx.createTargetRequestContext(); this.setCurrentMessageContext(parentContext.createHttpSendingContext()); responseBuilder = AggregatedRawResponse.builder(Instant.now()); - this.activeChannelFuture = new StringTrackableCompletableFuture<>(new CompletableFuture<>(), + this.activeChannelFuture = new StringTrackableCompletableFuture<>( () -> "incoming connection is ready for " + replaySession); var initialFuture = this.activeChannelFuture; diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/ChannelTask.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/ChannelTask.java index f8a43945f..e45ead7a3 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/ChannelTask.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/ChannelTask.java @@ -5,7 +5,7 @@ @AllArgsConstructor @Getter -public class ChannelTask { +public class ChannelTask { public final ChannelTaskType kind; - public final Runnable runnable; + public final FutureTransformer runnable; } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/ConnectionReplaySession.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/ConnectionReplaySession.java index c591a86c4..a1d1f0df7 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/ConnectionReplaySession.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/ConnectionReplaySession.java @@ -2,6 +2,7 @@ import io.netty.channel.ChannelFuture; import io.netty.channel.EventLoop; +import lombok.AllArgsConstructor; import lombok.Getter; import lombok.NonNull; import lombok.SneakyThrows; @@ -12,8 +13,13 @@ import org.opensearch.migrations.replay.util.StringTrackableCompletableFuture; import java.io.IOException; +import java.util.Comparator; +import java.util.Map; +import java.util.Optional; +import java.util.PriorityQueue; import java.util.concurrent.CompletableFuture; import java.util.function.Supplier; +import java.util.function.UnaryOperator; /** * This class contains everything that is needed to replay packets to a specific channel. @@ -34,10 +40,10 @@ public class ConnectionReplaySession { * EventLoop so that we can route all calls for this object into that loop/thread. */ public final EventLoop eventLoop; + public final OnlineRadixSorter scheduleSequencer; @Getter private Supplier> channelFutureFutureFactory; private ChannelFuture cachedChannel; // only can be accessed from the eventLoop thread - public final OnlineRadixSorter scheduleSequencer; public final TimeToResponseFulfillmentFutureMap schedule; @Getter private final IReplayContexts.IChannelKeyContext channelKeyContext; @@ -49,7 +55,7 @@ public ConnectionReplaySession(EventLoop eventLoop, IReplayContexts.IChannelKeyC { this.eventLoop = eventLoop; this.channelKeyContext = channelKeyContext; - this.scheduleSequencer = new OnlineRadixSorter<>(0); + this.scheduleSequencer = new OnlineRadixSorter(0); this.schedule = new TimeToResponseFulfillmentFutureMap(); this.channelFutureFutureFactory = channelFutureFutureFactory; } @@ -57,7 +63,7 @@ public ConnectionReplaySession(EventLoop eventLoop, IReplayContexts.IChannelKeyC public DiagnosticTrackableCompletableFuture getFutureThatReturnsChannelFuture(boolean requireActiveChannel) { StringTrackableCompletableFuture eventLoopFuture = - new StringTrackableCompletableFuture<>(new CompletableFuture<>(), () -> "procuring a connection"); + new StringTrackableCompletableFuture<>("procuring a connection"); eventLoop.submit(() -> { if (!requireActiveChannel || (cachedChannel != null && cachedChannel.channel().isActive())) { eventLoopFuture.future.complete(cachedChannel); @@ -103,10 +109,10 @@ private static boolean exceptionIsRetryable(@NonNull Throwable t) { } public boolean hasWorkRemaining() { - return scheduleSequencer.hasPending() || schedule.hasPendingTransmissions(); + return !scheduleSequencer.isEmpty() || schedule.hasPendingTransmissions(); } public long calculateSizeSlowly() { - return schedule.calculateSizeSlowly() + scheduleSequencer.numPending(); + return schedule.calculateSizeSlowly() + scheduleSequencer.size(); } } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/FutureTransformer.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/FutureTransformer.java new file mode 100644 index 000000000..0f63a5015 --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/FutureTransformer.java @@ -0,0 +1,13 @@ +package org.opensearch.migrations.replay.datatypes; + +import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; + +import java.util.function.Function; + +/** + * This is a function rather than just a supplier so that the future returned can be + * chained to its logical parent dependency. + */ +public interface FutureTransformer extends + Function, DiagnosticTrackableCompletableFuture> { +} diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/TimeToResponseFulfillmentFutureMap.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/TimeToResponseFulfillmentFutureMap.java index 7e8f63387..d13f50128 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/TimeToResponseFulfillmentFutureMap.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/TimeToResponseFulfillmentFutureMap.java @@ -1,23 +1,38 @@ package org.opensearch.migrations.replay.datatypes; +import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; +import org.opensearch.migrations.replay.util.StringTrackableCompletableFuture; + import java.time.Instant; import java.util.AbstractMap; import java.util.ArrayDeque; +import java.util.Collection; import java.util.Map; import java.util.StringJoiner; import java.util.TreeMap; public class TimeToResponseFulfillmentFutureMap { - TreeMap> timeToRunnableMap = new TreeMap<>(); + public static class FutureWorkPoint { + public final DiagnosticTrackableCompletableFuture scheduleFuture; + private final ChannelTaskType channelTaskType; + public FutureWorkPoint(Instant forTime, ChannelTaskType taskType) { + scheduleFuture = new StringTrackableCompletableFuture<>("scheduled start for " + forTime); + channelTaskType = taskType; + } + } - public void appendTask(Instant start, ChannelTask task) { + TreeMap> timeToRunnableMap = new TreeMap<>(); + + public FutureWorkPoint appendTaskTrigger(Instant start, ChannelTaskType taskType) { assert timeToRunnableMap.keySet().stream().allMatch(t->!t.isAfter(start)); var existing = timeToRunnableMap.computeIfAbsent(start, k->new ArrayDeque<>()); - existing.offer(task); + var fpp = new FutureWorkPoint(start, taskType); + existing.offer(fpp); + return fpp; } - public Map.Entry peekFirstItem() { + public Map.Entry peekFirstItem() { var e = timeToRunnableMap.firstEntry(); return e == null ? null : new AbstractMap.SimpleEntry<>(e.getKey(), e.getValue().peek()); } @@ -45,9 +60,8 @@ public boolean hasPendingTransmissions() { return false; } else { return timeToRunnableMap.values().stream() - .flatMap(d->d.stream()) - .filter(ct->ct.kind==ChannelTaskType.TRANSMIT) - .findAny().isPresent(); + .flatMap(Collection::stream) + .anyMatch(fwp->fwp.channelTaskType==ChannelTaskType.TRANSMIT); } } @@ -72,5 +86,4 @@ private String formatBookends() { .toString(); } } - } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFuture.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFuture.java index ec4c46fa0..0907cae6e 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFuture.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFuture.java @@ -88,6 +88,11 @@ public DiagnosticTrackableCompletableFuture(@NonNull CompletableFuture future return this.map(dcf->dcf.thenAccept(fn), diagnosticSupplier); } + public DiagnosticTrackableCompletableFuture + thenApply(Function fn, @NonNull Supplier diagnosticSupplier) { + return this.map(dcf->dcf.thenApply(fn), diagnosticSupplier); + } + public DiagnosticTrackableCompletableFuture exceptionally(Function fn, @NonNull Supplier diagnosticSupplier) { return this.map(cf->cf.exceptionally(fn), diagnosticSupplier); diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java index 57dc2dd79..ff13d2c5e 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java @@ -1,68 +1,94 @@ package org.opensearch.migrations.replay.util; +import lombok.AllArgsConstructor; import lombok.extern.slf4j.Slf4j; +import org.opensearch.migrations.replay.datatypes.FutureTransformer; -import java.util.ArrayList; -import java.util.function.Consumer; +import java.util.Comparator; +import java.util.Optional; +import java.util.PriorityQueue; /** * This provides a simple implementation to sort incoming elements that are ordered by a sequence - * of unique and contiguous integers. This implementation uses an ArrayList for staging out of order - * elements and the memory utilization will be O(total number of items to be sequenced). + * of unique and contiguous integers. This implementation uses a PriorityQueue for staging out of order + * elements and the memory utilization will be O(total number of items to be sequenced) in the worst case, + * but O(1) when the items are arriving in order. * - * After the item has been added, all the next currently sequenced items are passed to the Consumer - * that was provided to add(). This allows the calling context to visit the items in the natural - * order as opposed to the order that items were added. This class maintains a cursor of the last - * item that was sent so that items are only visited once and so that the class knows which item is - * the next item in the sequence. + * After the item has been added, if other items were waiting for it, all the next currently sequenced + * items are signaled. This allows the calling context to visit the items in the natural + * order as opposed to the order that items were added. * - * As items are visited, the object will drop its reference to the item, but no efforts are made to - * free its own storage. The assumption is that this class will be used for small, short-lived - * data sets. or in cases where the worst-case performance (needing to hold space for all the items) - * would be common. - * - * TODO - replace this with a PriorityQueue - * - * @param + * When an item is next to run, that 'slot' signals via the completion of a CompletableFuture. The future + * signaled is the same one that was passed to the processor function in addFutureWork. That processor + * is responsible for setting up any work necessary when the future is signaled (compose, whenComplete, etc) + * and returning the resultant future. That resultant future's completion will block the OnlineRadixSorter + * instance from proceeding to signal any subsequent signals. */ @Slf4j -public class OnlineRadixSorter { - ArrayList items; +public class OnlineRadixSorter { + @AllArgsConstructor + private static class IndexedWork { + public final int index; + public final DiagnosticTrackableCompletableFuture signalingFuture; + public final DiagnosticTrackableCompletableFuture workCompletedFuture; + } + + private final PriorityQueue items; int currentOffset; public OnlineRadixSorter(int startingOffset) { - items = new ArrayList<>(); + items = new PriorityQueue<>(Comparator.comparingInt(iw->iw.index)); currentOffset = startingOffset; } - public void add(int index, T item, Consumer sortedItemVisitor) { - assert index >= currentOffset; - if (currentOffset == index) { - ++currentOffset; - log.atTrace().setMessage(()->"Running callback for "+index+": "+this).log(); - sortedItemVisitor.accept(item); - while (currentOffset < items.size()) { - var nextItem = items.get(currentOffset); - if (nextItem != null) { - items.set(currentOffset, null); - ++currentOffset; - sortedItemVisitor.accept(nextItem); - } else { - break; - } - } - } else { - while (index >= items.size()) { - items.add(null); - } - items.set(index, item); - } + /** + * Add a new future that will be responsible for triggering some work now or in the future once all + * prior indices of work have been completed. Once the work is ready to be run, a future is marked + * as complete. It is the responsibility of the caller to supply a processor function that takes the + * completed future and supplies further processing upon its completion, returning the new future. + * Both futures will be tracked by this class with the first future acting as a signal while the + * second future returned by processor acts as a gate that prevents the triggering of subsequent + * work from happening until it has completed. + * @param index + * @param processor + * @return + */ + public DiagnosticTrackableCompletableFuture + addFutureForWork(int index, FutureTransformer processor) { + var signalFuture = new StringTrackableCompletableFuture("signaling future"); + var continueFuture = processor.apply(signalFuture); + + // purposefully use getDeferredFutureThroughHandle to do type erasure on T to get it back to Void + // since the caller is creating a DCF for their needs. However, type T will only come up again + // as per the work that was set within the processor. There's no benefit to making the underlying + // datastore aware of that T, hence the erasure. + var workBundle = new IndexedWork(index, signalFuture, + continueFuture.thenApply(v->{ + log.atDebug().setMessage(()->"Increasing currentOffset to " + currentOffset + + " for " + System.identityHashCode(this)).log(); + ++currentOffset; + return null; + }, () -> "Bumping currentOffset and checking if the next items should be signaled")); + items.add(workBundle); + pullNextWorkItemOrDoNothing(); + return continueFuture; } - public boolean hasPending() { - return currentOffset < items.size(); + private void pullNextWorkItemOrDoNothing() { + Optional.ofNullable(items.isEmpty() ? null : items.peek()) + .filter(indexedWork -> indexedWork.index == currentOffset) + .ifPresent(indexedWork -> { + var firstSignal = indexedWork.signalingFuture.future.complete(null); + assert firstSignal : "expected only this function to signal completion of the signaling future " + + "and for it to only be called once"; + var oldHead = items.remove(); + assert oldHead == indexedWork; + pullNextWorkItemOrDoNothing(); + }); } + public boolean hasPending() { return !items.isEmpty(); } + @Override public String toString() { final StringBuilder sb = new StringBuilder("OnlineRadixSorter{"); @@ -74,6 +100,10 @@ public String toString() { } public long numPending() { - return items.size() - (long) currentOffset; + return items.size(); } + + public boolean isEmpty() { return items.isEmpty(); } + + public int size() { return items.size(); } } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/StringTrackableCompletableFuture.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/StringTrackableCompletableFuture.java index 5ba4697a1..1ac8d8c82 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/StringTrackableCompletableFuture.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/StringTrackableCompletableFuture.java @@ -9,10 +9,23 @@ public class StringTrackableCompletableFuture extends DiagnosticTrackableCompletableFuture { + + public StringTrackableCompletableFuture(String diagnosticLabel) { + this(new CompletableFuture<>(), () -> diagnosticLabel); + } + + public StringTrackableCompletableFuture(Supplier diagnosticSupplier) { + this(new CompletableFuture<>(), diagnosticSupplier); + } + public StringTrackableCompletableFuture(@NonNull CompletableFuture future, Supplier diagnosticSupplier) { super(future, diagnosticSupplier); } + public StringTrackableCompletableFuture(@NonNull CompletableFuture future, String diagnosticLabel) { + super(future, () -> diagnosticLabel); + } + public static StringTrackableCompletableFuture failedFuture(Throwable e, Supplier diagnosticSupplier) { return new StringTrackableCompletableFuture<>(CompletableFuture.failedFuture(e), diagnosticSupplier); diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datatypes/TimeToResponseFulfillmentFutureMapTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datatypes/TimeToResponseFulfillmentFutureMapTest.java index 2da823381..93c32ca2f 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datatypes/TimeToResponseFulfillmentFutureMapTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datatypes/TimeToResponseFulfillmentFutureMapTest.java @@ -10,21 +10,27 @@ @WrapWithNettyLeakDetection(disableLeakChecks = true) class TimeToResponseFulfillmentFutureMapTest { @Test - public void testAddsAndPopsAreOrdered() { + public void testAddsAndPopsAreOrdered() throws Exception { var timeMap = new TimeToResponseFulfillmentFutureMap(); StringBuilder log = new StringBuilder(); - timeMap.appendTask(Instant.EPOCH, new ChannelTask(ChannelTaskType.TRANSMIT, ()->log.append('A'))); - timeMap.appendTask(Instant.EPOCH, new ChannelTask(ChannelTaskType.TRANSMIT, ()->log.append('B'))); - timeMap.appendTask(Instant.EPOCH.plus(Duration.ofMillis(1)), new ChannelTask(ChannelTaskType.TRANSMIT, ()->log.append('C'))); - timeMap.appendTask(Instant.EPOCH.plus(Duration.ofMillis(1)), new ChannelTask(ChannelTaskType.TRANSMIT, ()->log.append('D'))); + timeMap.appendTaskTrigger(Instant.EPOCH, ChannelTaskType.TRANSMIT) + .scheduleFuture.thenAccept(v->log.append('A'), ()->""); + timeMap.appendTaskTrigger(Instant.EPOCH, ChannelTaskType.TRANSMIT) + .scheduleFuture.thenAccept(v->log.append('B'), ()->""); + timeMap.appendTaskTrigger(Instant.EPOCH.plus(Duration.ofMillis(1)), ChannelTaskType.TRANSMIT) + .scheduleFuture.thenAccept(v->log.append('C'), ()->""); + var lastWorkFuture = + timeMap.appendTaskTrigger(Instant.EPOCH.plus(Duration.ofMillis(1)), ChannelTaskType.TRANSMIT) + .scheduleFuture.thenAccept(v->log.append('D'), ()->""); while (true) { var t = timeMap.peekFirstItem(); if (t == null) { break; } - t.getValue().runnable.run(); + t.getValue().scheduleFuture.future.complete(null); timeMap.removeFirstItem(); } + lastWorkFuture.get(); Assertions.assertEquals("ABCD", log.toString()); } } \ No newline at end of file diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/OnlineRadixSorterTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/OnlineRadixSorterTest.java index c615ac19e..b8ce37c6e 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/OnlineRadixSorterTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/OnlineRadixSorterTest.java @@ -13,28 +13,32 @@ private static String stringify(Stream stream) { return stream.map(i->i.toString()).collect(Collectors.joining(",")); } - private static String add(OnlineRadixSorterForIntegratedKeys sorter, int v) { - var sortedItems = new ArrayList(); - sorter.add(v, i->sortedItems.add(i)); - return stringify(sortedItems.stream()); + private static String add(OnlineRadixSorterForIntegratedKeys sorter, ArrayList receivedItems, int v) { + sorter.add(v, ()-> receivedItems.add(v)); + return stringify(receivedItems.stream()); } @Test void testOnlineRadixSorter_inOrder() { var radixSorter = new OnlineRadixSorterForIntegratedKeys(1, i -> (int) i); - Assertions.assertEquals("1", add(radixSorter,1)); - Assertions.assertEquals("2", add(radixSorter, 2)); - Assertions.assertEquals("3", add(radixSorter, 3)); + Assertions.assertEquals("1", add(radixSorter, new ArrayList(), 1)); + Assertions.assertEquals("2", add(radixSorter, new ArrayList(), 2)); + Assertions.assertEquals("3", add(radixSorter, new ArrayList(), 3)); } @Test void testOnlineRadixSorter_outOfOrder() { var radixSorter = new OnlineRadixSorterForIntegratedKeys(1, i->(int) i); - Assertions.assertEquals("", add(radixSorter, 3)); - Assertions.assertEquals("", add(radixSorter, 4)); - Assertions.assertEquals("1", add(radixSorter, 1)); - Assertions.assertEquals("2,3,4", add(radixSorter, 2)); - Assertions.assertEquals("5", add(radixSorter, 5)); - Assertions.assertEquals("", add(radixSorter, 7)); + var receiverList = new ArrayList(); + Assertions.assertEquals("", add(radixSorter, receiverList, 3)); + Assertions.assertEquals("", add(radixSorter, receiverList, 4)); + Assertions.assertEquals("1", add(radixSorter, receiverList, 1)); + receiverList.clear(); + Assertions.assertEquals("2,3,4", add(radixSorter, receiverList, 2)); + receiverList.clear(); + Assertions.assertEquals("5", add(radixSorter, receiverList, 5)); + receiverList.clear(); + Assertions.assertEquals("", add(radixSorter, receiverList, 7)); + receiverList.clear(); } } \ No newline at end of file diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorterForIntegratedKeys.java b/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/util/OnlineRadixSorterForIntegratedKeys.java similarity index 68% rename from TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorterForIntegratedKeys.java rename to TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/util/OnlineRadixSorterForIntegratedKeys.java index 90a677cb0..2acf33308 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorterForIntegratedKeys.java +++ b/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/util/OnlineRadixSorterForIntegratedKeys.java @@ -10,7 +10,7 @@ * * @param */ -public class OnlineRadixSorterForIntegratedKeys extends OnlineRadixSorter { +public class OnlineRadixSorterForIntegratedKeys extends OnlineRadixSorter { ToIntFunction radixResolver; @@ -19,7 +19,9 @@ public OnlineRadixSorterForIntegratedKeys(int startingOffset, ToIntFunction r this.radixResolver = radixResolver; } - public void add(T item, Consumer sortedItemVisitor) { - super.add(radixResolver.applyAsInt(item), item, sortedItemVisitor); + public void add(T item, Runnable sortedItemVisitor) { + super.addFutureForWork(radixResolver.applyAsInt(item), signalFuture->signalFuture.map( + f->f.whenComplete((v,t)->sortedItemVisitor.run()), + ()->"OnlineRadixSorterForIntegratedKeys.add")); } } From b52e1ba074f1f3ab9be953072f7bac7116f4e81b Mon Sep 17 00:00:00 2001 From: Greg Schohn Date: Wed, 17 Apr 2024 07:37:45 -0400 Subject: [PATCH 03/11] Bugfix in OnlineRadixSorter. All 'tests' pass, but I'm working through issues in 'slowTest' Signed-off-by: Greg Schohn --- .../migrations/replay/util/OnlineRadixSorter.java | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java index ff13d2c5e..77e39619a 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java @@ -66,11 +66,15 @@ public OnlineRadixSorter(int startingOffset) { continueFuture.thenApply(v->{ log.atDebug().setMessage(()->"Increasing currentOffset to " + currentOffset + " for " + System.identityHashCode(this)).log(); - ++currentOffset; - return null; + items.remove(); + ++currentOffset; + pullNextWorkItemOrDoNothing(); + return null; }, () -> "Bumping currentOffset and checking if the next items should be signaled")); items.add(workBundle); - pullNextWorkItemOrDoNothing(); + if (index == this.currentOffset) { + pullNextWorkItemOrDoNothing(); + } return continueFuture; } @@ -81,9 +85,6 @@ private void pullNextWorkItemOrDoNothing() { var firstSignal = indexedWork.signalingFuture.future.complete(null); assert firstSignal : "expected only this function to signal completion of the signaling future " + "and for it to only be called once"; - var oldHead = items.remove(); - assert oldHead == indexedWork; - pullNextWorkItemOrDoNothing(); }); } From 030f91c423f0156452f2a9cec527e5a42a9cb5df Mon Sep 17 00:00:00 2001 From: Greg Schohn Date: Thu, 25 Apr 2024 07:01:58 -0400 Subject: [PATCH 04/11] Refactoring + change the underlying data structure for TimeToResponseFulfillmentFutureMap to use a dequeue instead of a queue. If all of the events for the time fulfillment map come in order thanks to the sequencer, there's no reason to use a more complicated/expensive data structure. When the items are in order, they lend themselves more to chaining too. Also... Refactor netty-CompletableFuture binding helpers into a new utility class. Open up the DiagnosticTrackableCompletableFuture class to make it easier to test that chains are properly formed and updated. Start to build out the RequestSenderOrchestratorTest a little bit more. Signed-off-by: Greg Schohn --- .../NettyToCompletableFutureBinders.java | 66 +++++++++ .../replay/RequestSenderOrchestrator.java | 126 ++++++---------- .../replay/TrafficReplayerTopLevel.java | 3 +- .../NettyPacketToHttpConsumer.java | 2 +- .../TimeToResponseFulfillmentFutureMap.java | 43 ++---- .../DiagnosticTrackableCompletableFuture.java | 45 ++++-- .../replay/util/OnlineRadixSorter.java | 2 +- .../replay/RequestSenderOrchestratorTest.java | 137 +++++++++++++++++- .../NettyPacketToHttpConsumerTest.java | 5 +- ...imeToResponseFulfillmentFutureMapTest.java | 2 +- .../e2etests/FullTrafficReplayerTest.java | 6 +- 11 files changed, 294 insertions(+), 143 deletions(-) create mode 100644 TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/NettyToCompletableFutureBinders.java diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/NettyToCompletableFutureBinders.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/NettyToCompletableFutureBinders.java new file mode 100644 index 000000000..155de099a --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/NettyToCompletableFutureBinders.java @@ -0,0 +1,66 @@ +package org.opensearch.migrations; + +import io.netty.channel.EventLoop; +import io.netty.util.concurrent.Future; +import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; +import org.opensearch.migrations.replay.util.StringTrackableCompletableFuture; + +import java.time.Duration; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; +import java.util.function.Supplier; + +public class NettyToCompletableFutureBinders { + public static CompletableFuture + bindNettyFutureToCompletableFuture(Future nettyFuture, CompletableFuture cf) { + nettyFuture.addListener(f -> { + if (!f.isSuccess()) { + cf.completeExceptionally(f.cause()); + } else { + cf.complete(null); + } + }); + return cf; + } + + public static CompletableFuture + bindNettyFutureToCompletableFuture(Future nettyFuture) { + return bindNettyFutureToCompletableFuture(nettyFuture, new CompletableFuture<>()); + } + + public static DiagnosticTrackableCompletableFuture + bindNettyFutureToTrackableFuture(Future nettyFuture, String label) { + return new StringTrackableCompletableFuture<>(bindNettyFutureToCompletableFuture(nettyFuture), label); + } + + public static DiagnosticTrackableCompletableFuture + bindNettyFutureToTrackableFuture(Future nettyFuture, Supplier labelProvider) { + return new StringTrackableCompletableFuture<>(bindNettyFutureToCompletableFuture(nettyFuture), labelProvider); + } + + public static DiagnosticTrackableCompletableFuture + bindNettyFutureToTrackableFuture(Function> nettyFutureGenerator, String label) { + return bindNettyFutureToTrackableFuture(nettyFutureGenerator.apply(() -> { + }), label); + } + + public static DiagnosticTrackableCompletableFuture + bindNettySubmitToTrackableFuture(EventLoop eventLoop) { + return bindNettyFutureToTrackableFuture(eventLoop::submit, "waiting for event loop submission"); + } + + public static DiagnosticTrackableCompletableFuture + bindNettyScheduleToCompletableFuture(EventLoop eventLoop, Duration delay) { + var delayMs = Math.max(0, delay.toMillis()); + return bindNettyFutureToTrackableFuture(eventLoop.schedule(() -> {}, delayMs, TimeUnit.MILLISECONDS), + "scheduling to run next send at " + delay + " in " + delayMs +" ms (clipped)"); + } + + public static CompletableFuture + bindNettyScheduleToCompletableFuture(EventLoop eventLoop, Duration delay, CompletableFuture cf) { + var delayMs = Math.max(0, delay.toMillis()); + return bindNettyFutureToCompletableFuture(eventLoop.schedule(() -> { + }, delayMs, TimeUnit.MILLISECONDS), cf); + } +} diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/RequestSenderOrchestrator.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/RequestSenderOrchestrator.java index 19fe363d3..094e36e1d 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/RequestSenderOrchestrator.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/RequestSenderOrchestrator.java @@ -2,8 +2,9 @@ import io.netty.buffer.ByteBuf; import io.netty.channel.EventLoop; -import io.netty.util.concurrent.Future; import lombok.extern.slf4j.Slf4j; +import org.opensearch.migrations.NettyToCompletableFutureBinders; +import org.opensearch.migrations.replay.datahandlers.IPacketFinalizingConsumer; import org.opensearch.migrations.replay.datahandlers.NettyPacketToHttpConsumer; import org.opensearch.migrations.replay.datatypes.ChannelTask; import org.opensearch.migrations.replay.datatypes.ChannelTaskType; @@ -19,8 +20,8 @@ import java.util.Iterator; import java.util.Optional; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiFunction; import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Stream; @@ -28,83 +29,39 @@ @Slf4j public class RequestSenderOrchestrator { - public static CompletableFuture - bindNettyFutureToCompletableFuture(Future nettyFuture, CompletableFuture cf) { - nettyFuture.addListener(f -> { - if (!f.isSuccess()) { - cf.completeExceptionally(f.cause()); - } else { - cf.complete(null); - } - }); - return cf; - } - - public static CompletableFuture - bindNettyFutureToCompletableFuture(Future nettyFuture) { - return bindNettyFutureToCompletableFuture(nettyFuture, new CompletableFuture<>()); - } - - public static DiagnosticTrackableCompletableFuture - bindNettyFutureToTrackableFuture(Future nettyFuture, String label) { - return new StringTrackableCompletableFuture<>(bindNettyFutureToCompletableFuture(nettyFuture), label); - } - - public static DiagnosticTrackableCompletableFuture - bindNettyFutureToTrackableFuture(Future nettyFuture, Supplier labelProvider) { - return new StringTrackableCompletableFuture<>(bindNettyFutureToCompletableFuture(nettyFuture), labelProvider); - } - - public static DiagnosticTrackableCompletableFuture - bindNettyFutureToTrackableFuture(Function> nettyFutureGenerator, String label) { - return bindNettyFutureToTrackableFuture(nettyFutureGenerator.apply(()->{}), label); - } - - public static DiagnosticTrackableCompletableFuture - bindNettySubmitToTrackableFuture(EventLoop eventLoop) { - return bindNettyFutureToTrackableFuture(eventLoop::submit, "waiting for event loop submission"); - } - - public static DiagnosticTrackableCompletableFuture - bindNettyScheduleToCompletableFuture(EventLoop eventLoop, Duration delay) { - var delayMs = Math.max(0, delay.toMillis()); - return bindNettyFutureToTrackableFuture(eventLoop.schedule(()->{}, delayMs, TimeUnit.MILLISECONDS), - "scheduling to run next send in " + delay); - } - - private DiagnosticTrackableCompletableFuture + public DiagnosticTrackableCompletableFuture bindNettyScheduleToCompletableFuture(EventLoop eventLoop, Instant timestamp) { - return bindNettyScheduleToCompletableFuture(eventLoop, getDelayFromNowMs(timestamp)); + return NettyToCompletableFutureBinders.bindNettyScheduleToCompletableFuture(eventLoop, getDelayFromNowMs(timestamp)); } - public static CompletableFuture - bindNettyScheduleToCompletableFuture(EventLoop eventLoop, Duration delay, CompletableFuture cf) { - var delayMs = Math.max(0, delay.toMillis()); - return bindNettyFutureToCompletableFuture(eventLoop.schedule(()->{}, delayMs, TimeUnit.MILLISECONDS), cf); + public StringTrackableCompletableFuture + bindNettyScheduleToCompletableFuture(EventLoop eventLoop, + Instant timestamp, + DiagnosticTrackableCompletableFuture existingFuture) { + var delayMs = getDelayFromNowMs(timestamp); + NettyToCompletableFutureBinders.bindNettyScheduleToCompletableFuture(eventLoop, delayMs, existingFuture.future); + return new StringTrackableCompletableFuture<>(existingFuture.future, + "scheduling to run next send at " + timestamp + " in " + delayMs + "ms"); } - private CompletableFuture + public CompletableFuture bindNettyScheduleToCompletableFuture(EventLoop eventLoop, Instant timestamp, CompletableFuture cf) { - return bindNettyScheduleToCompletableFuture(eventLoop, getDelayFromNowMs(timestamp), cf); - } - - - public void - bindNettyFutureToTrackableFuture(EventLoop eventLoop, - Instant timestamp, - DiagnosticTrackableCompletableFuture existingFuture) { - var delayMs = Math.max(0, getDelayFromNowMs(timestamp).toMillis()); - var scheduleFuture = eventLoop.schedule(()->{}, delayMs, TimeUnit.MILLISECONDS); - new StringTrackableCompletableFuture<>( - bindNettyFutureToCompletableFuture(scheduleFuture, existingFuture.future), - "scheduling to run next send in " + timestamp); + return NettyToCompletableFutureBinders.bindNettyScheduleToCompletableFuture(eventLoop, + getDelayFromNowMs(timestamp), cf); } public final ClientConnectionPool clientConnectionPool; - - public RequestSenderOrchestrator(ClientConnectionPool clientConnectionPool) { + public final BiFunction> packetConsumerFactory; + + public RequestSenderOrchestrator(ClientConnectionPool clientConnectionPool, + BiFunction> packetConsumerFactory) { this.clientConnectionPool = clientConnectionPool; + this.packetConsumerFactory = packetConsumerFactory; } public DiagnosticTrackableCompletableFuture @@ -135,7 +92,7 @@ public RequestSenderOrchestrator(ClientConnectionPool clientConnectionPool) { } else { return StringTrackableCompletableFuture.failedFuture(scheduleFailure, ()->"netty scheduling failure"); } - }, ()->"The scheduled callback is running"); + }, ()->"The scheduled callback is running work for " + ctx); } public DiagnosticTrackableCompletableFuture @@ -181,7 +138,7 @@ public DiagnosticTrackableCompletableFuture scheduleClose(IReplayCo Function> onSessionCallback) { final var replaySession = clientConnectionPool.getCachedSession(ctx, sessionNumber); - return bindNettySubmitToTrackableFuture(replaySession.eventLoop) + return NettyToCompletableFutureBinders.bindNettySubmitToTrackableFuture(replaySession.eventLoop) .getDeferredFutureThroughHandle((v,t) -> { log.atTrace().setMessage(() -> "adding work item at slot " + channelInteractionNumber + " for " + replaySession.getChannelKeyContext() + " with " + @@ -189,7 +146,7 @@ public DiagnosticTrackableCompletableFuture scheduleClose(IReplayCo return replaySession.scheduleSequencer.addFutureForWork(channelInteractionNumber, f->f.thenCompose(voidValue -> onSessionCallback.apply(replaySession), ()->"Work callback on replay session")); - }, () -> "Waiting for sequencer to run for slot " + channelInteractionNumber); + }, () -> "Waiting for sequencer to finish for slot " + channelInteractionNumber); } private DiagnosticTrackableCompletableFuture @@ -207,9 +164,9 @@ public DiagnosticTrackableCompletableFuture scheduleClose(IReplayCo new ChannelTask<>(ChannelTaskType.TRANSMIT, trigger -> trigger.thenCompose(voidVal -> { scheduledContext.close(); - return sendSendingRestOfPackets(new NettyPacketToHttpConsumer(connectionReplaySession, ctx), + return sendSendingRestOfPackets(packetConsumerFactory.apply(connectionReplaySession, ctx), eventLoop, packets.iterator(), startTime, interval, new AtomicInteger()); - }, ()->"sending next packets"))); + }, ()->"sending packets for request"))); } private DiagnosticTrackableCompletableFuture @@ -240,7 +197,7 @@ public DiagnosticTrackableCompletableFuture scheduleClose(IReplayCo var eventLoop = channelFutureAndRequestSchedule.eventLoop; var wasEmpty = schedule.isEmpty(); - assert wasEmpty || !atTime.isBefore(schedule.peekFirstItem().getKey()) : + assert wasEmpty || !atTime.isBefore(schedule.peekFirstItem().startTime) : "Per-connection TrafficStream ordering should force a time ordering on incoming requests"; var workPointTrigger = schedule.appendTaskTrigger(atTime, task.kind).scheduleFuture; var workFuture = task.getRunnable().apply(workPointTrigger); @@ -256,9 +213,8 @@ public DiagnosticTrackableCompletableFuture scheduleClose(IReplayCo "Expected to have popped the item to match the start time for the responseFuture that finished"; log.atDebug().setMessage(()->channelInteraction.toString() + " responseFuture completed - checking " + schedule + " for the next item to schedule").log(); - Optional.ofNullable(schedule.peekFirstItem()).ifPresent(kvp-> { - bindNettyFutureToTrackableFuture(eventLoop, kvp.getKey(), kvp.getValue().scheduleFuture); - }); + Optional.ofNullable(schedule.peekFirstItem()).ifPresent(kvp -> + bindNettyScheduleToCompletableFuture(eventLoop, kvp.startTime, kvp.scheduleFuture)); }), ()->""); return workFuture; @@ -273,7 +229,7 @@ private Duration getDelayFromNowMs(Instant to) { } private DiagnosticTrackableCompletableFuture - sendSendingRestOfPackets(NettyPacketToHttpConsumer packetReceiver, + sendSendingRestOfPackets(IPacketFinalizingConsumer packetReceiver, EventLoop eventLoop, Iterator iterator, Instant startAt, @@ -283,15 +239,17 @@ private Duration getDelayFromNowMs(Instant to) { log.atTrace().setMessage(()->"sendNextPartAndContinue: counter=" + oldCounter).log(); assert iterator.hasNext() : "Should not have called this with no items to send"; - packetReceiver.consumeBytes(iterator.next()); + var consumeFuture = packetReceiver.consumeBytes(iterator.next()); if (iterator.hasNext()) { - var delay = Duration.between(now(), - startAt.plus(interval.multipliedBy(counter.get()))); - return bindNettyScheduleToCompletableFuture(eventLoop, delay) + return consumeFuture.thenCompose(cf -> + NettyToCompletableFutureBinders.bindNettyScheduleToCompletableFuture(eventLoop, + Duration.between(now(), startAt.plus(interval.multipliedBy(counter.get())))) .getDeferredFutureThroughHandle((v,t)-> sendSendingRestOfPackets(packetReceiver, eventLoop, - iterator, startAt, interval, counter), () -> "sending next packet"); + iterator, startAt, interval, counter), () -> "sending next packet"), + () -> "recursing, once ready"); } else { - return packetReceiver.finalizeRequest(); + return consumeFuture.getDeferredFutureThroughHandle((v,t) -> packetReceiver.finalizeRequest(), + ()->"finalizing, once ready"); } } } 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 fff581d6d..5b9553660 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 @@ -6,6 +6,7 @@ import lombok.NonNull; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; +import org.opensearch.migrations.replay.datahandlers.NettyPacketToHttpConsumer; import org.opensearch.migrations.replay.datatypes.UniqueReplayerRequestKey; import org.opensearch.migrations.replay.tracing.IRootReplayerContext; import org.opensearch.migrations.replay.tracing.RootReplayerContext; @@ -135,7 +136,7 @@ public void setupRunAndWaitForReplayToFinish(Duration observedPacketConnectionTi Consumer resultTupleConsumer) throws InterruptedException, ExecutionException { - var senderOrchestrator = new RequestSenderOrchestrator(clientConnectionPool); + var senderOrchestrator = new RequestSenderOrchestrator(clientConnectionPool, NettyPacketToHttpConsumer::new); var replayEngine = new ReplayEngine(senderOrchestrator, trafficSource, timeShifter); CapturedTrafficToHttpTransactionAccumulator trafficToHttpTransactionAccumulator = 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 b45a628de..17aa3db3f 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 @@ -288,7 +288,7 @@ public DiagnosticTrackableCompletableFuture consumeBytes(ByteBuf pa System.identityHashCode(packetData) + "): " + httpContext() + ": " + packetData.toString(StandardCharsets.UTF_8)).log(); return writePacketAndUpdateFuture(packetData).whenComplete((v2,t2)->{ - log.atDebug().setMessage(()->"finished writing " + httpContext() + " t=" + t2).log(); + log.atTrace().setMessage(()->"finished writing " + httpContext() + " t=" + t2).log(); }, ()->""); } else { log.atWarn().setMessage(()-> httpContext().getReplayerRequestKey() + diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/TimeToResponseFulfillmentFutureMap.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/TimeToResponseFulfillmentFutureMap.java index 195893614..255952d86 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/TimeToResponseFulfillmentFutureMap.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/TimeToResponseFulfillmentFutureMap.java @@ -1,12 +1,9 @@ package org.opensearch.migrations.replay.datatypes; import java.time.Instant; -import java.util.AbstractMap; import java.util.ArrayDeque; -import java.util.Collection; -import java.util.Map; +import java.util.Deque; import java.util.StringJoiner; -import java.util.TreeMap; import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; import org.opensearch.migrations.replay.util.StringTrackableCompletableFuture; @@ -14,41 +11,31 @@ public class TimeToResponseFulfillmentFutureMap { public static class FutureWorkPoint { + public final Instant startTime; public final DiagnosticTrackableCompletableFuture scheduleFuture; private final ChannelTaskType channelTaskType; public FutureWorkPoint(Instant forTime, ChannelTaskType taskType) { + startTime = forTime; scheduleFuture = new StringTrackableCompletableFuture<>("scheduled start for " + forTime); channelTaskType = taskType; } } - TreeMap> timeToRunnableMap = new TreeMap<>(); + Deque timeToRunnableMap = new ArrayDeque<>(); public FutureWorkPoint appendTaskTrigger(Instant start, ChannelTaskType taskType) { - assert timeToRunnableMap.keySet().stream().allMatch(t->!t.isAfter(start)); - var existing = timeToRunnableMap.computeIfAbsent(start, k->new ArrayDeque<>()); + assert timeToRunnableMap.stream().map(fwp->fwp.startTime).allMatch(t->!t.isAfter(start)); var fpp = new FutureWorkPoint(start, taskType); - existing.offer(fpp); + timeToRunnableMap.offer(fpp); return fpp; } - public Map.Entry peekFirstItem() { - var e = timeToRunnableMap.firstEntry(); - return e == null ? null : new AbstractMap.SimpleEntry<>(e.getKey(), e.getValue().peek()); + public FutureWorkPoint peekFirstItem() { + return timeToRunnableMap.peekFirst(); } public Instant removeFirstItem() { - var e = timeToRunnableMap.firstEntry(); - if (e != null) { - var q = e.getValue(); - q.remove(); - if (q.isEmpty()) { - timeToRunnableMap.remove(e.getKey()); - } - return e.getKey(); - } else { - return null; - } + return timeToRunnableMap.isEmpty() ? null : timeToRunnableMap.pop().startTime; } public boolean isEmpty() { @@ -59,14 +46,12 @@ public boolean hasPendingTransmissions() { if (timeToRunnableMap.isEmpty()) { return false; } else { - return timeToRunnableMap.values().stream() - .flatMap(Collection::stream) - .anyMatch(fwp->fwp.channelTaskType==ChannelTaskType.TRANSMIT); + return timeToRunnableMap.stream().anyMatch(fwp->fwp.channelTaskType==ChannelTaskType.TRANSMIT); } } public long calculateSizeSlowly() { - return timeToRunnableMap.values().stream().map(ArrayDeque::size).mapToInt(x->x).sum(); + return timeToRunnableMap.size(); } @Override @@ -78,11 +63,11 @@ private String formatBookends() { if (timeToRunnableMap.isEmpty()) { return ""; } else if (timeToRunnableMap.size() == 1) { - return timeToRunnableMap.firstKey().toString(); + return timeToRunnableMap.peekFirst().startTime.toString(); } else { return new StringJoiner("...") - .add(timeToRunnableMap.firstKey().toString()) - .add(timeToRunnableMap.lastKey().toString()) + .add(timeToRunnableMap.peekFirst().startTime.toString()) + .add(timeToRunnableMap.peekLast().toString()) .toString(); } } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFuture.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFuture.java index 9bad7a0a3..b577b0589 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFuture.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFuture.java @@ -1,5 +1,6 @@ package org.opensearch.migrations.replay.util; +import lombok.Getter; import lombok.NonNull; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; @@ -20,6 +21,7 @@ import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.IntStream; +import java.util.stream.Stream; /** * This class wraps CompletableFutures into traceable and identifiable pieces so that when @@ -36,7 +38,10 @@ public class DiagnosticTrackableCompletableFuture { public final CompletableFuture future; protected AtomicReference> innerComposedPendingCompletableFutureReference; - protected final Supplier diagnosticSupplier; + @Getter + public final Supplier diagnosticSupplier; + // TODO: Clear this once it has been completed to prevemt chains with + // thousands/millions of items that are no longer needed protected final DiagnosticTrackableCompletableFuture dependencyDiagnosticFuture; private DiagnosticTrackableCompletableFuture() { @@ -76,6 +81,11 @@ public DiagnosticTrackableCompletableFuture(@NonNull CompletableFuture future this(future, diagnosticSupplier, null); } + public DiagnosticTrackableCompletableFuture getInnerComposedPendingCompletableFuture() { + return Optional.ofNullable(innerComposedPendingCompletableFutureReference) + .map(AtomicReference::get).orElse(null); + } + public DiagnosticTrackableCompletableFuture map(@NonNull Function, CompletableFuture> fn, @NonNull Supplier diagnosticSupplier) { @@ -147,6 +157,7 @@ public DiagnosticTrackableCompletableFuture(@NonNull CompletableFuture future var wrappedDiagnosticFuture = new DiagnosticTrackableCompletableFuture<>(newCf, diagnosticSupplier, this); wrappedDiagnosticFuture.innerComposedPendingCompletableFutureReference = innerComposedCompletableFutureReference; + // TODO: Add a count to how many futures have been completed and are falling away? wrappedDiagnosticFuture.future.whenComplete((v2,t2)->innerComposedCompletableFutureReference.set(null)); return wrappedDiagnosticFuture; } @@ -176,36 +187,38 @@ public String toString() { } public String formatAsString(@NonNull Function,String> resultFormatter) { + return walkParentsAsStream().map(kvp->formatFutureWithDiagnostics(kvp, resultFormatter)) + .collect(Collectors.joining("<-")); + } + + public Stream> walkParentsAsStream() { AtomicReference> chainHeadReference = new AtomicReference<>(this); - var strList = IntStream.generate(() -> chainHeadReference.get() != null ? 1 : 0) + return IntStream.generate(() -> chainHeadReference.get() != null ? 1 : 0) .takeWhile(x -> x == 1) - ., Supplier>>mapToObj(i -> { + .mapToObj(i -> { var dcf = chainHeadReference.get(); chainHeadReference.set(dcf.dependencyDiagnosticFuture); - return new AbstractMap.SimpleEntry<>(dcf, dcf.diagnosticSupplier); - }).map(kvp->formatFutureWithDiagnostics(kvp, resultFormatter)) - .collect(Collectors.toList()); - return strList.stream().collect(Collectors.joining("<-")); + return dcf; + }); } @SneakyThrows protected String formatFutureWithDiagnostics( - @NonNull AbstractMap.SimpleEntry, Supplier> kvp, + @NonNull DiagnosticTrackableCompletableFuture dcf, @NonNull Function,String> resultFormatter) { - var diagnosticInfo = kvp.getValue().get(); - var isDone = kvp.getKey().isDone(); - return "[" + System.identityHashCode(kvp.getKey()) + "] " + diagnosticInfo + - (isDone ? formatWithDefault(resultFormatter, kvp.getKey()) : - getPendingString(kvp, resultFormatter)); + var diagnosticInfo = dcf.diagnosticSupplier.get(); + var isDone = dcf.isDone(); + return "[" + System.identityHashCode(dcf) + "] " + diagnosticInfo + + (isDone ? formatWithDefault(resultFormatter, dcf) : + getPendingString(dcf, resultFormatter)); } private static String - getPendingString(AbstractMap.SimpleEntry, Supplier> kvp, + getPendingString(DiagnosticTrackableCompletableFuture dcf, Function, String> resultFormatter) { - return Optional.ofNullable(kvp.getKey().innerComposedPendingCompletableFutureReference) + return Optional.ofNullable(dcf.innerComposedPendingCompletableFutureReference) .map(r -> (DiagnosticTrackableCompletableFuture) r.get()) - .filter(Objects::nonNull) .map(df -> " --[[" + df.formatAsString(resultFormatter) + " ]] ") .orElse("[…]"); } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java index 77e39619a..61e954555 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java @@ -55,7 +55,7 @@ public OnlineRadixSorter(int startingOffset) { */ public DiagnosticTrackableCompletableFuture addFutureForWork(int index, FutureTransformer processor) { - var signalFuture = new StringTrackableCompletableFuture("signaling future"); + var signalFuture = new StringTrackableCompletableFuture("OnlineRadixSorter signal future #" + index); var continueFuture = processor.apply(signalFuture); // purposefully use getDeferredFutureThroughHandle to do type erasure on T to get it back to Void 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 661d76ba3..9820e99e5 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 @@ -4,43 +4,169 @@ import io.netty.buffer.ByteBufHolder; import io.netty.buffer.Unpooled; import io.netty.handler.codec.http.FullHttpResponse; +import lombok.Lombok; 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.api.parallel.Execution; import org.junit.jupiter.api.parallel.ExecutionMode; +import org.opensearch.migrations.replay.datahandlers.IPacketFinalizingConsumer; +import org.opensearch.migrations.replay.datahandlers.NettyPacketToHttpConsumer; import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; +import org.opensearch.migrations.replay.util.StringTrackableCompletableFuture; import org.opensearch.migrations.testutils.SimpleHttpServer; import org.opensearch.migrations.testutils.WrapWithNettyLeakDetection; import org.opensearch.migrations.tracing.InstrumentationTest; +import java.net.URI; import java.nio.charset.StandardCharsets; import java.time.Duration; import java.time.Instant; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Semaphore; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; +import java.util.stream.IntStream; @Slf4j @WrapWithNettyLeakDetection(repetitions = 1) class RequestSenderOrchestratorTest extends InstrumentationTest { - public static final int NUM_REQUESTS_TO_SCHEDULE = 20; - public static final int NUM_REPEATS = 2; + public static class BlockingPacketConsumer implements IPacketFinalizingConsumer { + + private final long id; + public final Semaphore consumeIsReady = new Semaphore(0, true); + public final Semaphore lastCheckIsReady = new Semaphore(0, true); + private final AtomicInteger calls = new AtomicInteger(); + + BlockingPacketConsumer(long id) { + this.id = id; + } + + @Override + public DiagnosticTrackableCompletableFuture consumeBytes(ByteBuf nextRequestPacket) { + var index = calls.getAndIncrement(); + return new StringTrackableCompletableFuture<>(CompletableFuture.supplyAsync(()->{ + try { + lastCheckIsReady.release(); + log.atDebug().setMessage(()->"trying to acquire semaphore for packet #"+index+" and id="+id).log(); + consumeIsReady.acquire(); + log.atDebug().setMessage(()->"Acquired semaphore for packet #"+index+" and id="+id).log(); + } catch (InterruptedException e) { + throw Lombok.sneakyThrow(e); + } + return (Void)null; + }), ()->"consumeBytes waiting on test-gate semaphore release"); + } + + @Override + public DiagnosticTrackableCompletableFuture finalizeRequest() { + var index = calls.getAndIncrement(); + return new StringTrackableCompletableFuture<>(CompletableFuture.supplyAsync(()->{ + try { + lastCheckIsReady.release(); + log.atDebug().setMessage(()->"trying to acquire semaphore for finalize and id="+id).log(); + consumeIsReady.acquire(); + } catch (InterruptedException e) { + throw Lombok.sneakyThrow(e); + } + return new AggregatedRawResponse(0, Duration.ZERO, null, null); + }), ()->"finalizeRequest waiting on test-gate semaphore release"); + } + } + + @Test + public void testFutureGraphBuildout() throws Exception { + final int NUM_REQUESTS_TO_SCHEDULE = 2; + final int NUM_REPEATS = 1; + final int NUM_PACKETS = 1; + + var clientConnectionPool = TrafficReplayerTopLevel.makeClientConnectionPool(new URI("http://localhost"), + false, 1, "testFutureGraphBuildout targetConnectionPool", + Duration.ofSeconds(30)); + var connectionToConsumerMap = new HashMap(); + var senderOrchestrator = new RequestSenderOrchestrator(clientConnectionPool, (s,c) -> + connectionToConsumerMap.get(c.getSourceRequestIndex())); + var baseTime = Instant.EPOCH; + Instant lastEndTime = baseTime; + var scheduledRequests = new ArrayList>(); + for (int i = 0; i < NUM_REQUESTS_TO_SCHEDULE; ++i) { + connectionToConsumerMap.put((long)i, new BlockingPacketConsumer(i)); + var requestContext = rootContext.getTestConnectionRequestContext(i); + // same as the test below... + // half the time schedule at the same time as the last one, the other half, 10ms later than the previous + var perPacketShift = Duration.ofMillis(10 * i / NUM_REPEATS); + var startTimeForThisRequest = baseTime.plus(perPacketShift); + var requestPackets = IntStream.range(0,NUM_PACKETS) + .mapToObj(b->Unpooled.wrappedBuffer(new byte[]{(byte)b})) // TODO refCnt issue + .collect(Collectors.toList()); + var arrCf = senderOrchestrator.scheduleRequest(requestContext.getReplayerRequestKey(), requestContext, + startTimeForThisRequest, Duration.ofMillis(1), requestPackets.stream()); + + log.info("Scheduled item to run at " + startTimeForThisRequest); + scheduledRequests.add(arrCf); + lastEndTime = startTimeForThisRequest.plus(perPacketShift.multipliedBy(requestPackets.size())); + } + var connectionCtx = rootContext.getTestConnectionRequestContext(NUM_REQUESTS_TO_SCHEDULE); + var closeFuture = senderOrchestrator.scheduleClose( + connectionCtx.getLogicalEnclosingScope(), NUM_REQUESTS_TO_SCHEDULE, 0, + lastEndTime.plus(Duration.ofMillis(100))); + + Assertions.assertEquals(NUM_REQUESTS_TO_SCHEDULE, scheduledRequests.size()); + for (int i = 0; i < scheduledRequests.size(); ++i) { + for (int j = 0; j "cf @ " + finalI + "," + finalJ + " =\n" + + scheduledRequests.stream().map(sr-> getParentsDiagnosticString(sr, "")) + .collect(Collectors.joining("\n"))) + .log(); + pktConsumer.consumeIsReady.release(); + } + } + for (var cf : scheduledRequests) { + var arr = cf.get(); + log.info("Finalized cf="+ getParentsDiagnosticString(cf, "")); + Assertions.assertNull(arr.error); + } + closeFuture.get(); + } + + private String getParentsDiagnosticString(DiagnosticTrackableCompletableFuture cf, String indent) { + return cf.walkParentsAsStream() + .map(dcf->Optional.ofNullable(dcf.getInnerComposedPendingCompletableFuture()) + .map(idf->indent + "<\n" + + getParentsDiagnosticString(idf, indent+" ") + "\n" + indent + ">\n") + .orElse("") + + indent + dcf.diagnosticSupplier.get() + + "[" + System.identityHashCode(dcf) + "]" + + ": " + dcf.isDone()) + .collect(Collectors.joining(";\n")); + } @Test @Tag("longTest") @Execution(ExecutionMode.SAME_THREAD) public void testThatSchedulingWorks() throws Exception { + final int NUM_REQUESTS_TO_SCHEDULE = 20; + final int NUM_REPEATS = 2; + try (var httpServer = SimpleHttpServer.makeServer(false, r -> TestHttpServerContext.makeResponse(r, Duration.ofMillis(100)))) { var testServerUri = httpServer.localhostEndpoint(); var clientConnectionPool = TrafficReplayerTopLevel.makeClientConnectionPool(testServerUri, false, 1, "targetConnectionPool for testThatSchedulingWorks", Duration.ofSeconds(30)); - var senderOrchestrator = new RequestSenderOrchestrator(clientConnectionPool); + var senderOrchestrator = + new RequestSenderOrchestrator(clientConnectionPool, NettyPacketToHttpConsumer::new); var baseTime = Instant.now(); Instant lastEndTime = baseTime; var scheduledItems = new ArrayList>(); @@ -75,7 +201,7 @@ public void testThatSchedulingWorks() throws Exception { var body = response.content(); Assertions.assertEquals(TestHttpServerContext.SERVER_RESPONSE_BODY_PREFIX + TestHttpServerContext.getUriForIthRequest(i / NUM_REPEATS), - new String(body.duplicate().toString(StandardCharsets.UTF_8))); + body.duplicate().toString(StandardCharsets.UTF_8)); } finally { Optional.ofNullable((httpMessage instanceof ByteBufHolder) ? (ByteBufHolder) httpMessage : null) .ifPresent(bbh -> bbh.content().release()); @@ -88,7 +214,8 @@ public void testThatSchedulingWorks() throws Exception { private List makeRequest(int i) { // uncomment/swap for a simpler test case to run return //List.of(Unpooled.wrappedBuffer(getRequestString(i).getBytes())); - TestHttpServerContext.getRequestString(i).chars().mapToObj(c->Unpooled.wrappedBuffer(new byte[]{(byte) c})) + TestHttpServerContext.getRequestString(i).chars() + .mapToObj(c->Unpooled.wrappedBuffer(new byte[]{(byte) c})) // TODO refCnt issue .collect(Collectors.toList()); } } \ No newline at end of file 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 abcd7fee1..0951900ce 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,8 +260,9 @@ public void testThatConnectionsAreKeptAliveAndShared(boolean useTls, boolean lar timeShifter.setFirstTimestamp(Instant.now()); var clientConnectionPool = new ClientConnectionPool(testServer.localhostEndpoint(), sslContext, "targetPool for testThatConnectionsAreKeptAliveAndShared", 1, REGULAR_RESPONSE_TIMEOUT); - var sendingFactory = new ReplayEngine(new RequestSenderOrchestrator(clientConnectionPool), - new TestFlowController(), timeShifter); + var sendingFactory = new ReplayEngine( + new RequestSenderOrchestrator(clientConnectionPool, NettyPacketToHttpConsumer::new), + new TestFlowController(), timeShifter); for (int j = 0; j < 2; ++j) { for (int i = 0; i < 2; ++i) { var ctx = rootContext.getTestConnectionRequestContext("TEST_" + i, j); diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datatypes/TimeToResponseFulfillmentFutureMapTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datatypes/TimeToResponseFulfillmentFutureMapTest.java index 93c32ca2f..f43c0fb54 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datatypes/TimeToResponseFulfillmentFutureMapTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datatypes/TimeToResponseFulfillmentFutureMapTest.java @@ -27,7 +27,7 @@ public void testAddsAndPopsAreOrdered() throws Exception { if (t == null) { break; } - t.getValue().scheduleFuture.future.complete(null); + t.scheduleFuture.future.complete(null); timeMap.removeFirstItem(); } lastWorkFuture.get(); 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 398e73cc9..a5a1720e8 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 @@ -299,9 +299,9 @@ public void makeSureThatCollateralDamageDoesntFreezeTests() throws Throwable { @ParameterizedTest @CsvSource(value = { - "3,false", - "-1,false", - "3,true", +// "3,false", +// "-1,false", +// "3,true", "-1,true", }) @Tag("longTest") From e8305a7b69e9fc439e4d8d0272bcfd06b4a325bf Mon Sep 17 00:00:00 2001 From: Greg Schohn Date: Thu, 25 Apr 2024 12:04:52 -0400 Subject: [PATCH 05/11] =?UTF-8?q?Change=20the=20OnlineRadixSorter=20yet=20?= =?UTF-8?q?again,=20this=20time=20to=20use=20a=20sorted=20map=20to=20allow?= =?UTF-8?q?=20for=20chaining.=20In=20addition=20to=20chaining=20sequential?= =?UTF-8?q?=20stages=20together=20(which=20now,=20again,=20need=20to=20cre?= =?UTF-8?q?ate=20all=20intermediates=20before=20the=20current=20work=20ite?= =?UTF-8?q?m),=20the=20class=20now=20emits=20which=20sequential=20stages?= =?UTF-8?q?=20are=20missing=20(outstanding)=20in=20the=20DCF's=20diagnosti?= =?UTF-8?q?c=20supplier.=20Sample=20output=20may=20look=20like=20this=20[5?= =?UTF-8?q?04336483]=20Caller-task=20completion=20for=20idx=3D3[=E2=80=A6]?= =?UTF-8?q?<-[335107734]=20OnlineRadixSorterForIntegratedKeys.addFutureFor?= =?UTF-8?q?Work[=E2=80=A6]<-[215078753]=20Kickoff=20for=20slot=20#3[?= =?UTF-8?q?=E2=80=A6]<-[1384454980]=20Work=20to=20finish=20for=20slot=20#2?= =?UTF-8?q?=20is=20awaiting=20[slotsOutstanding:2][=E2=80=A6]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Greg Schohn --- .../DiagnosticTrackableCompletableFuture.java | 5 +- .../replay/util/OnlineRadixSorter.java | 99 ++++++++++++------- .../replay/RequestSenderOrchestratorTest.java | 7 +- .../replay/util/OnlineRadixSorterTest.java | 35 ++++--- .../OnlineRadixSorterForIntegratedKeys.java | 10 +- 5 files changed, 99 insertions(+), 57 deletions(-) diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFuture.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFuture.java index b577b0589..a5bf73127 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFuture.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFuture.java @@ -163,9 +163,8 @@ public DiagnosticTrackableCompletableFuture getInnerComposedPendingCompleta } public DiagnosticTrackableCompletableFuture - handle(@NonNull BiFunction fn, - @NonNull Supplier diagnosticSupplier) { - CompletableFuture newCf = this.future.handle(fn::apply); + handle(@NonNull BiFunction fn, @NonNull Supplier diagnosticSupplier) { + CompletableFuture newCf = this.future.handle(fn); return new DiagnosticTrackableCompletableFuture<>(newCf, diagnosticSupplier, this); } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java index 61e954555..c97f53495 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java @@ -2,11 +2,20 @@ import lombok.AllArgsConstructor; import lombok.extern.slf4j.Slf4j; +import org.apache.lucene.document.IntRange; import org.opensearch.migrations.replay.datatypes.FutureTransformer; +import java.util.ArrayDeque; import java.util.Comparator; +import java.util.Deque; import java.util.Optional; -import java.util.PriorityQueue; +import java.util.SortedMap; +import java.util.SortedSet; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; +import java.util.stream.IntStream; /** * This provides a simple implementation to sort incoming elements that are ordered by a sequence @@ -28,16 +37,26 @@ public class OnlineRadixSorter { @AllArgsConstructor private static class IndexedWork { - public final int index; - public final DiagnosticTrackableCompletableFuture signalingFuture; - public final DiagnosticTrackableCompletableFuture workCompletedFuture; + public final DiagnosticTrackableCompletableFuture signalingToStartFuture; + public DiagnosticTrackableCompletableFuture workCompletedFuture; + public final DiagnosticTrackableCompletableFuture signalWorkCompletedFuture; + + public DiagnosticTrackableCompletableFuture + addWorkFuture(FutureTransformer processor, int index) { + var rval = processor.apply(signalingToStartFuture) + .whenComplete((v,t)-> + signalWorkCompletedFuture.future.complete(null), + ()->"Caller-task completion for idx=" + index); + workCompletedFuture = rval; + return rval; + } } - private final PriorityQueue items; + private final SortedMap items; int currentOffset; public OnlineRadixSorter(int startingOffset) { - items = new PriorityQueue<>(Comparator.comparingInt(iw->iw.index)); + items = new TreeMap<>(); currentOffset = startingOffset; } @@ -54,42 +73,48 @@ public OnlineRadixSorter(int startingOffset) { * @return */ public DiagnosticTrackableCompletableFuture - addFutureForWork(int index, FutureTransformer processor) { - var signalFuture = new StringTrackableCompletableFuture("OnlineRadixSorter signal future #" + index); - var continueFuture = processor.apply(signalFuture); - - // purposefully use getDeferredFutureThroughHandle to do type erasure on T to get it back to Void - // since the caller is creating a DCF for their needs. However, type T will only come up again - // as per the work that was set within the processor. There's no benefit to making the underlying - // datastore aware of that T, hence the erasure. - var workBundle = new IndexedWork(index, signalFuture, - continueFuture.thenApply(v->{ - log.atDebug().setMessage(()->"Increasing currentOffset to " + currentOffset + - " for " + System.identityHashCode(this)).log(); - items.remove(); + addFutureForWork(final int index, FutureTransformer processor) { + var oldWorkItem = items.get(index); + if (oldWorkItem == null) { + if (index < currentOffset) { + throw new IllegalArgumentException("index (" + index + ")" + + " must be > last processed item (" + currentOffset + ")"); + } + for (int nextKey = Math.max(currentOffset, items.isEmpty() ? 0 : items.lastKey()+1); + nextKey<=index; + ++nextKey) { + int finalNextKey = nextKey; + var signalFuture = items.isEmpty() ? + new StringTrackableCompletableFuture( + CompletableFuture.completedFuture(null), "unlinked signaling future") : + items.get(items.lastKey()).signalWorkCompletedFuture + .thenAccept(v-> {}, + ()->"Kickoff for slot #" + finalNextKey); + oldWorkItem = new IndexedWork(signalFuture, null, + new StringTrackableCompletableFuture(()->"Work to finish for slot #" + finalNextKey + + " is awaiting [" + getAwaitingTextUpTo(index) + "]")); + oldWorkItem.signalWorkCompletedFuture.whenComplete((v,t)->{ ++currentOffset; - pullNextWorkItemOrDoNothing(); - return null; - }, () -> "Bumping currentOffset and checking if the next items should be signaled")); - items.add(workBundle); - if (index == this.currentOffset) { - pullNextWorkItemOrDoNothing(); + items.remove(finalNextKey); + }, ()->"cleaning up spent work for idx #" + finalNextKey); + items.put(nextKey, oldWorkItem); + } } - return continueFuture; + return oldWorkItem.addWorkFuture(processor, index); } - private void pullNextWorkItemOrDoNothing() { - Optional.ofNullable(items.isEmpty() ? null : items.peek()) - .filter(indexedWork -> indexedWork.index == currentOffset) - .ifPresent(indexedWork -> { - var firstSignal = indexedWork.signalingFuture.future.complete(null); - assert firstSignal : "expected only this function to signal completion of the signaling future " + - "and for it to only be called once"; - }); + public String getAwaitingTextUpTo(int upTo) { + return "slotsOutstanding:" + + IntStream.range(0, upTo-currentOffset) + .map(i->upTo-i-1) + .mapToObj(i -> Optional.ofNullable(items.get(i)) + .flatMap(wi->Optional.ofNullable(wi.workCompletedFuture)) + .map(ignored->"") + .orElse(i+"")) + .filter(s->!s.isEmpty()) + .collect(Collectors.joining(",")); } - public boolean hasPending() { return !items.isEmpty(); } - @Override public String toString() { final StringBuilder sb = new StringBuilder("OnlineRadixSorter{"); @@ -100,6 +125,8 @@ public String toString() { return sb.toString(); } + public boolean hasPending() { return !items.isEmpty(); } + public long numPending() { return items.size(); } 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 9820e99e5..9f0c6886d 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 @@ -24,6 +24,7 @@ import java.time.Duration; import java.time.Instant; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Optional; @@ -118,6 +119,8 @@ public void testFutureGraphBuildout() throws Exception { lastEndTime.plus(Duration.ofMillis(100))); Assertions.assertEquals(NUM_REQUESTS_TO_SCHEDULE, scheduledRequests.size()); + var reversedScheduledRequests = new ArrayList<>(scheduledRequests); + Collections.reverse(reversedScheduledRequests); for (int i = 0; i < scheduledRequests.size(); ++i) { for (int j = 0; j "cf @ " + finalI + "," + finalJ + " =\n" + - scheduledRequests.stream().map(sr-> getParentsDiagnosticString(sr, "")) - .collect(Collectors.joining("\n"))) + reversedScheduledRequests.stream().map(sr-> getParentsDiagnosticString(sr, "")) + .collect(Collectors.joining("\n---\n"))) .log(); pktConsumer.consumeIsReady.release(); } diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/OnlineRadixSorterTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/OnlineRadixSorterTest.java index b8ce37c6e..1058b1d7f 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/OnlineRadixSorterTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/OnlineRadixSorterTest.java @@ -1,44 +1,55 @@ package org.opensearch.migrations.replay.util; +import lombok.extern.slf4j.Slf4j; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import java.util.ArrayList; +import java.util.HashMap; +import java.util.Map; import java.util.stream.Collectors; import java.util.stream.Stream; +@Slf4j class OnlineRadixSorterTest { private static String stringify(Stream stream) { - return stream.map(i->i.toString()).collect(Collectors.joining(",")); + return stream.map(Object::toString).collect(Collectors.joining(",")); } - private static String add(OnlineRadixSorterForIntegratedKeys sorter, ArrayList receivedItems, int v) { - sorter.add(v, ()-> receivedItems.add(v)); + private static String add(OnlineRadixSorterForIntegratedKeys sorter, + Map> m, + ArrayList receivedItems, int v) { + var dcf = sorter.add(v, () -> receivedItems.add(v)); + if (m != null) { m.put(v, dcf); } + log.atInfo().setMessage(()->"after adding work... "+dcf).log(); return stringify(receivedItems.stream()); } @Test void testOnlineRadixSorter_inOrder() { var radixSorter = new OnlineRadixSorterForIntegratedKeys(1, i -> (int) i); - Assertions.assertEquals("1", add(radixSorter, new ArrayList(), 1)); - Assertions.assertEquals("2", add(radixSorter, new ArrayList(), 2)); - Assertions.assertEquals("3", add(radixSorter, new ArrayList(), 3)); + Assertions.assertEquals("1", add(radixSorter, null, new ArrayList(), 1)); + Assertions.assertEquals("2", add(radixSorter, null, new ArrayList(), 2)); + Assertions.assertEquals("3", add(radixSorter, null, new ArrayList(), 3)); } @Test void testOnlineRadixSorter_outOfOrder() { var radixSorter = new OnlineRadixSorterForIntegratedKeys(1, i->(int) i); var receiverList = new ArrayList(); - Assertions.assertEquals("", add(radixSorter, receiverList, 3)); - Assertions.assertEquals("", add(radixSorter, receiverList, 4)); - Assertions.assertEquals("1", add(radixSorter, receiverList, 1)); + var dcfMap = new HashMap>(); + Assertions.assertEquals("", add(radixSorter, dcfMap, receiverList, 3)); + Assertions.assertEquals("", add(radixSorter, dcfMap, receiverList, 4)); + Assertions.assertEquals("1", add(radixSorter, dcfMap, receiverList, 1)); + log.atInfo().setMessage(()->"after adding work for '1'... dcf[3]=" + dcfMap.get(3)).log(); + log.atInfo().setMessage(()->"after adding work for '1'... dcf[4]=" + dcfMap.get(4)).log(); receiverList.clear(); - Assertions.assertEquals("2,3,4", add(radixSorter, receiverList, 2)); + Assertions.assertEquals("2,3,4", add(radixSorter, dcfMap, receiverList, 2)); receiverList.clear(); - Assertions.assertEquals("5", add(radixSorter, receiverList, 5)); + Assertions.assertEquals("5", add(radixSorter, dcfMap, receiverList, 5)); receiverList.clear(); - Assertions.assertEquals("", add(radixSorter, receiverList, 7)); + Assertions.assertEquals("", add(radixSorter, dcfMap, receiverList, 7)); receiverList.clear(); } } \ No newline at end of file diff --git a/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/util/OnlineRadixSorterForIntegratedKeys.java b/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/util/OnlineRadixSorterForIntegratedKeys.java index 2acf33308..4c04388fb 100644 --- a/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/util/OnlineRadixSorterForIntegratedKeys.java +++ b/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/util/OnlineRadixSorterForIntegratedKeys.java @@ -1,6 +1,7 @@ package org.opensearch.migrations.replay.util; -import java.util.function.Consumer; +import lombok.extern.slf4j.Slf4j; + import java.util.function.ToIntFunction; /** @@ -10,6 +11,7 @@ * * @param */ +@Slf4j public class OnlineRadixSorterForIntegratedKeys extends OnlineRadixSorter { ToIntFunction radixResolver; @@ -19,9 +21,9 @@ public OnlineRadixSorterForIntegratedKeys(int startingOffset, ToIntFunction r this.radixResolver = radixResolver; } - public void add(T item, Runnable sortedItemVisitor) { - super.addFutureForWork(radixResolver.applyAsInt(item), signalFuture->signalFuture.map( + public DiagnosticTrackableCompletableFuture add(T item, Runnable sortedItemVisitor) { + return super.addFutureForWork(radixResolver.applyAsInt(item), signalFuture->signalFuture.map( f->f.whenComplete((v,t)->sortedItemVisitor.run()), - ()->"OnlineRadixSorterForIntegratedKeys.add")); + ()->"OnlineRadixSorterForIntegratedKeys.addFutureForWork")); } } From 7ff0ccc948ef6822a9a65e453a2de6076fcda74a Mon Sep 17 00:00:00 2001 From: Greg Schohn Date: Thu, 25 Apr 2024 19:14:39 -0400 Subject: [PATCH 06/11] More control of DiagnosticTrackableCompletableFuture parent dependencies. Cull grandparents (if they exist) when the current future is done, checking when traversing the ancestry chain and in a whenComplete handler (which may be pretty late to the mix given how many other dependent functions might be in line before it). There's also now the ability to set the dependent parent AFTER construction. This helps to bind sequential stages from the OnlineRadixSorter together. Signed-off-by: Greg Schohn --- .../DiagnosticTrackableCompletableFuture.java | 49 ++++++++++++++++--- .../replay/util/OnlineRadixSorter.java | 16 ++---- .../replay/RequestSenderOrchestratorTest.java | 4 +- ...gnosticTrackableCompletableFutureTest.java | 46 +++++++++++++++++ .../StringTrackableCompletableFutureTest.java | 5 +- 5 files changed, 98 insertions(+), 22 deletions(-) create mode 100644 TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFutureTest.java diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFuture.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFuture.java index a5bf73127..234e788f7 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFuture.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFuture.java @@ -6,8 +6,6 @@ import lombok.extern.slf4j.Slf4j; import java.time.Duration; -import java.util.AbstractMap; -import java.util.Objects; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; @@ -40,9 +38,7 @@ public class DiagnosticTrackableCompletableFuture { protected AtomicReference> innerComposedPendingCompletableFutureReference; @Getter public final Supplier diagnosticSupplier; - // TODO: Clear this once it has been completed to prevemt chains with - // thousands/millions of items that are no longer needed - protected final DiagnosticTrackableCompletableFuture dependencyDiagnosticFuture; + protected final AtomicReference> parentDiagnosticFutureRef; private DiagnosticTrackableCompletableFuture() { throw new IllegalCallerException(); @@ -73,7 +69,8 @@ private DiagnosticTrackableCompletableFuture( DiagnosticTrackableCompletableFuture parentFuture) { this.future = future; this.diagnosticSupplier = diagnosticSupplier; - this.dependencyDiagnosticFuture = parentFuture; + this.parentDiagnosticFutureRef = new AtomicReference<>(); + setParentDiagnosticFuture(parentFuture); } public DiagnosticTrackableCompletableFuture(@NonNull CompletableFuture future, @@ -81,6 +78,44 @@ public DiagnosticTrackableCompletableFuture(@NonNull CompletableFuture future this(future, diagnosticSupplier, null); } + public DiagnosticTrackableCompletableFuture getParentDiagnosticFuture() { + var p = parentDiagnosticFutureRef.get(); + if (future.isDone() && p != null) { + p.setParentDiagnosticFuture(null); + } + return p; + } + + protected void setParentDiagnosticFuture(DiagnosticTrackableCompletableFuture parent) { + if (parent == null) { + parentDiagnosticFutureRef.set(null); + return; + } + var wasSet = parentDiagnosticFutureRef.compareAndSet(null, parent); + if (!wasSet) { + throw new IllegalStateException("dependencyDiagnosticFutureRef was already set to " + + parentDiagnosticFutureRef.get()); + } + // the parent is a pretty good breadcrumb for the current stack... but the grandparent of the most recently + // finished ancestor begins to have diminished value immediately, so cut the ancestry tree at this point + future.whenComplete((v, t) -> { + Optional.ofNullable(getParentDiagnosticFuture()) + .ifPresent(p->p.setParentDiagnosticFuture(null)); + }); + } + + /** + * @throws IllegalStateException if the dependentFuture has already been passed to this method + * before or if it has already been marked as completed or was initialized with a parent. + */ + public DiagnosticTrackableCompletableFuture + propagateCompletionToDependentFuture(DiagnosticTrackableCompletableFuture dependentFuture, + BiConsumer, CompletableFuture> consume, + @NonNull Supplier diagnosticSupplier) { + dependentFuture.setParentDiagnosticFuture(this); + return this.whenComplete((v,t) -> consume.accept(this.future, dependentFuture.future), diagnosticSupplier); + } + public DiagnosticTrackableCompletableFuture getInnerComposedPendingCompletableFuture() { return Optional.ofNullable(innerComposedPendingCompletableFutureReference) .map(AtomicReference::get).orElse(null); @@ -197,7 +232,7 @@ public Stream> walkParentsAsStream() { .takeWhile(x -> x == 1) .mapToObj(i -> { var dcf = chainHeadReference.get(); - chainHeadReference.set(dcf.dependencyDiagnosticFuture); + chainHeadReference.set(dcf.getParentDiagnosticFuture()); return dcf; }); } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java index c97f53495..70bbb3239 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java @@ -2,17 +2,11 @@ import lombok.AllArgsConstructor; import lombok.extern.slf4j.Slf4j; -import org.apache.lucene.document.IntRange; import org.opensearch.migrations.replay.datatypes.FutureTransformer; -import java.util.ArrayDeque; -import java.util.Comparator; -import java.util.Deque; import java.util.Optional; import java.util.SortedMap; -import java.util.SortedSet; import java.util.TreeMap; -import java.util.TreeSet; import java.util.concurrent.CompletableFuture; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -43,10 +37,9 @@ private static class IndexedWork { public DiagnosticTrackableCompletableFuture addWorkFuture(FutureTransformer processor, int index) { - var rval = processor.apply(signalingToStartFuture) - .whenComplete((v,t)-> - signalWorkCompletedFuture.future.complete(null), - ()->"Caller-task completion for idx=" + index); + var rval = processor.apply(signalingToStartFuture).propagateCompletionToDependentFuture(signalWorkCompletedFuture, + (processedCf, dependentCf) -> dependentCf.complete(null), + ()->"Caller-task completion for idx=" + index); workCompletedFuture = rval; return rval; } @@ -86,7 +79,8 @@ public OnlineRadixSorter(int startingOffset) { int finalNextKey = nextKey; var signalFuture = items.isEmpty() ? new StringTrackableCompletableFuture( - CompletableFuture.completedFuture(null), "unlinked signaling future") : + CompletableFuture.completedFuture(null), + "unlinked signaling future for slot #" + finalNextKey) : items.get(items.lastKey()).signalWorkCompletedFuture .thenAccept(v-> {}, ()->"Kickoff for slot #" + finalNextKey); 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 9f0c6886d..9bc557bc0 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 @@ -85,7 +85,7 @@ public DiagnosticTrackableCompletableFuture final public void testFutureGraphBuildout() throws Exception { final int NUM_REQUESTS_TO_SCHEDULE = 2; final int NUM_REPEATS = 1; - final int NUM_PACKETS = 1; + final int NUM_PACKETS = 3; var clientConnectionPool = TrafficReplayerTopLevel.makeClientConnectionPool(new URI("http://localhost"), false, 1, "testFutureGraphBuildout targetConnectionPool", @@ -150,7 +150,7 @@ private String getParentsDiagnosticString(DiagnosticTrackableCompletableFuture\n") .orElse("") + indent + dcf.diagnosticSupplier.get() + - "[" + System.identityHashCode(dcf) + "]" + + " [" + System.identityHashCode(dcf) + "]" + ": " + dcf.isDone()) .collect(Collectors.joining(";\n")); } diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFutureTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFutureTest.java new file mode 100644 index 000000000..a1a33bd89 --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFutureTest.java @@ -0,0 +1,46 @@ +package org.opensearch.migrations.replay.util; + +import lombok.Lombok; +import lombok.extern.slf4j.Slf4j; +import org.junit.jupiter.api.Test; + +import java.util.concurrent.Semaphore; +import java.util.concurrent.ThreadPoolExecutor; + +@Slf4j +class DiagnosticTrackableCompletableFutureTest { + + @Test + public void test() throws Exception { + final int ITERATIONS = 5; + DiagnosticTrackableCompletableFuture base = + new StringTrackableCompletableFuture<>("initial future"); + var dcf = base; + var dcfSemaphore = new Semaphore(0); + var observerSemaphore = new Semaphore(0); + for (int i = 0; i< ITERATIONS; ++i) { + int finalI = i; + var lastDcf = dcf; + dcf = dcf.thenApply(v->{ + try { + observerSemaphore.release(); + dcfSemaphore.acquire(); + log.atInfo().setMessage(()->"dcf[" + finalI+"]"+lastDcf).log(); + } catch (InterruptedException e) { + throw Lombok.sneakyThrow(e); + } + return v + "," + finalI; + }, ()->"run for "+ finalI); + } + base.future.completeAsync(()->""); + DiagnosticTrackableCompletableFuture finalDcf = dcf; + for (int i=0; i"top dcf after " + finalI + " releases="+ finalDcf).log(); + } + log.atInfo().setMessage(()->"final dcf after any ancestor culls=" + finalDcf).log(); + } +} \ No newline at end of file diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/StringTrackableCompletableFutureTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/StringTrackableCompletableFutureTest.java index e98812d7d..a804b7dc7 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/StringTrackableCompletableFutureTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/StringTrackableCompletableFutureTest.java @@ -63,10 +63,11 @@ public void futureWithThreeStages() throws Exception { Assertions.assertEquals(id3 + "C[…]<-" + id2 + "B[^]<-" + id1 + "A[^]", stcf3.toString()); Assertions.assertEquals(id3 + "C[…]<-" + id2 + "B[11]<-" + id1 + "A[1]", stcf3.formatAsString(StringTrackableCompletableFutureTest::formatCompletableFuture)); + // A is clipped because of grandparent culling notifyAndCheckNewDiagnosticValue(stcf3, notifier3, - id3 + "C[^]<-" + id2 +"B[^]<-" + id1 + "A[^]"); + id3 + "C[^]<-" + id2 +"B[^]"); Assertions.assertEquals(id1 + "A[^]", stcf1.toString()); - Assertions.assertEquals(id2 + "B[^]<-" + id1 + "A[^]", stcf2.toString()); + Assertions.assertEquals(id2 + "B[^]", stcf2.toString()); } From a6d5dd43bf17f09f9f48243295b686dff5abdf9d Mon Sep 17 00:00:00 2001 From: Greg Schohn Date: Fri, 26 Apr 2024 11:58:00 -0400 Subject: [PATCH 07/11] Make the list of sequences that the object is waiting for in getAwaitingText shorter by doing run-length compression. Signed-off-by: Greg Schohn --- .../java/org/opensearch/migrations/Utils.java | 2 + .../SequentialSpanCompressingReducer.java | 58 +++++++++++++++++++ .../replay/util/OnlineRadixSorter.java | 35 ++++++----- .../replay/util/OnlineRadixSorterTest.java | 13 +++++ 4 files changed, 94 insertions(+), 14 deletions(-) create mode 100644 TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/utils/SequentialSpanCompressingReducer.java diff --git a/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/Utils.java b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/Utils.java index a06519e52..0d5770eb6 100644 --- a/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/Utils.java +++ b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/Utils.java @@ -1,5 +1,7 @@ package org.opensearch.migrations; +import lombok.AllArgsConstructor; + import java.util.function.BiFunction; import java.util.function.Function; import java.util.stream.Collector; diff --git a/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/utils/SequentialSpanCompressingReducer.java b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/utils/SequentialSpanCompressingReducer.java new file mode 100644 index 000000000..a797059ec --- /dev/null +++ b/TrafficCapture/coreUtilities/src/main/java/org/opensearch/migrations/utils/SequentialSpanCompressingReducer.java @@ -0,0 +1,58 @@ +package org.opensearch.migrations.utils; + +import lombok.AllArgsConstructor; + +/** + * This class can be used to reduce a stream of Integers into a string (calling getFinalAccumulation()) To use + * ``` + * Stream<Integer>...reduce(new SequentialSpanCompressingReducer(-1), SequentialSpanCompressingReducer::addNext, + * (c, d) -> { throw new IllegalStateException("parallel streams aren't allowed"); }) + * ``` + */ +@AllArgsConstructor +public class SequentialSpanCompressingReducer { + private static final int IGNORED_SENTINEL_VALUE = -1; + private static final char RUN_CHARACTER = '-'; + + private final int shift; + private final int last; + private final StringBuilder accumulatedText; + + public SequentialSpanCompressingReducer(int shift) { + this.shift = shift; + this.last = IGNORED_SENTINEL_VALUE; + this.accumulatedText = new StringBuilder(); + } + + private boolean lastWasSpan() { + var len = accumulatedText.length(); + return len > 0 && accumulatedText.charAt(len-1) == RUN_CHARACTER; + } + + public SequentialSpanCompressingReducer addNext(int b) { + if (last+shift == b) { + if (lastWasSpan()) { + return new SequentialSpanCompressingReducer(shift, b, + accumulatedText); + } else { + return new SequentialSpanCompressingReducer(shift, b, + accumulatedText.append(RUN_CHARACTER)); + } + } else { + if (lastWasSpan()) { + return new SequentialSpanCompressingReducer(shift, b, + accumulatedText.append(last).append(",").append(b)); + } else { + return new SequentialSpanCompressingReducer(shift, b, + accumulatedText.append(last == IGNORED_SENTINEL_VALUE ? "" : ",").append(b)); + } + } + } + + public String getFinalAccumulation() { + if (lastWasSpan()) { + accumulatedText.append(last); + } + return accumulatedText.toString(); + } +} \ No newline at end of file diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java index 70bbb3239..7979e95a0 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java @@ -1,14 +1,15 @@ package org.opensearch.migrations.replay.util; import lombok.AllArgsConstructor; +import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.opensearch.migrations.replay.datatypes.FutureTransformer; +import org.opensearch.migrations.utils.SequentialSpanCompressingReducer; import java.util.Optional; import java.util.SortedMap; import java.util.TreeMap; import java.util.concurrent.CompletableFuture; -import java.util.stream.Collectors; import java.util.stream.IntStream; /** @@ -26,14 +27,18 @@ * is responsible for setting up any work necessary when the future is signaled (compose, whenComplete, etc) * and returning the resultant future. That resultant future's completion will block the OnlineRadixSorter * instance from proceeding to signal any subsequent signals. + * + * This class is NOT thread safe and is only meant to be called from a single thread. */ @Slf4j public class OnlineRadixSorter { + @AllArgsConstructor + @Getter private static class IndexedWork { - public final DiagnosticTrackableCompletableFuture signalingToStartFuture; - public DiagnosticTrackableCompletableFuture workCompletedFuture; - public final DiagnosticTrackableCompletableFuture signalWorkCompletedFuture; + private final DiagnosticTrackableCompletableFuture signalingToStartFuture; + private DiagnosticTrackableCompletableFuture workCompletedFuture; + private final DiagnosticTrackableCompletableFuture signalWorkCompletedFuture; public DiagnosticTrackableCompletableFuture addWorkFuture(FutureTransformer processor, int index) { @@ -86,7 +91,7 @@ public OnlineRadixSorter(int startingOffset) { ()->"Kickoff for slot #" + finalNextKey); oldWorkItem = new IndexedWork(signalFuture, null, new StringTrackableCompletableFuture(()->"Work to finish for slot #" + finalNextKey + - " is awaiting [" + getAwaitingTextUpTo(index) + "]")); + " is awaiting [" + getAwaitingText() + "]")); oldWorkItem.signalWorkCompletedFuture.whenComplete((v,t)->{ ++currentOffset; items.remove(finalNextKey); @@ -97,16 +102,18 @@ public OnlineRadixSorter(int startingOffset) { return oldWorkItem.addWorkFuture(processor, index); } - public String getAwaitingTextUpTo(int upTo) { - return "slotsOutstanding:" + + public String getAwaitingText() { + final var upTo = items.lastKey(); + return "slotsOutstanding: >" + (upTo) + "," + IntStream.range(0, upTo-currentOffset) - .map(i->upTo-i-1) - .mapToObj(i -> Optional.ofNullable(items.get(i)) - .flatMap(wi->Optional.ofNullable(wi.workCompletedFuture)) - .map(ignored->"") - .orElse(i+"")) - .filter(s->!s.isEmpty()) - .collect(Collectors.joining(",")); + .map(i->upTo-i-1) + .filter(i->Optional.ofNullable(items.get(i)) + .flatMap(wi->Optional.ofNullable(wi.workCompletedFuture)) + .isEmpty()) + .boxed() + .reduce(new SequentialSpanCompressingReducer(-1), SequentialSpanCompressingReducer::addNext, + (c, d) -> { throw new IllegalStateException("parallel streams aren't allowed"); }) + .getFinalAccumulation(); } @Override diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/OnlineRadixSorterTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/OnlineRadixSorterTest.java index 1058b1d7f..8eea70092 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/OnlineRadixSorterTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/OnlineRadixSorterTest.java @@ -52,4 +52,17 @@ void testOnlineRadixSorter_outOfOrder() { Assertions.assertEquals("", add(radixSorter, dcfMap, receiverList, 7)); receiverList.clear(); } + + @Test + void testGetAwaitingStrings() { + var radixSorter = new OnlineRadixSorter(1); + radixSorter.addFutureForWork(4, x -> x); + Assertions.assertEquals("slotsOutstanding: >4,3-1", radixSorter.getAwaitingText()); + radixSorter.addFutureForWork(6, x -> x); + Assertions.assertEquals("slotsOutstanding: >6,5,3-1", radixSorter.getAwaitingText()); + for (int i = 9; i < 20; ++i) { + radixSorter.addFutureForWork(i, x -> x); + } + Assertions.assertEquals("slotsOutstanding: >19,8-7,5,3-1", radixSorter.getAwaitingText()); + } } \ No newline at end of file From b540e91d3c042185777094528128b804316cd305 Mon Sep 17 00:00:00 2001 From: Greg Schohn Date: Sat, 27 Apr 2024 09:52:13 -0400 Subject: [PATCH 08/11] Make the activity monitor output the future dependency graph as json. The old format is still present and is used to print outstanding work that's outstanding when the replayer is shut down. Json should make it easier to pretty print the dependency graph. I think it might make more sense to re-reverse the order of the graph for json output though. I should also shift the LEVEL line to the beginning so that it's easier to copy-paste. Signed-off-by: Greg Schohn --- .../migrations/replay/TrafficReplayer.java | 8 +- .../DiagnosticTrackableCompletableFuture.java | 40 ++------ ...ackableCompletableFutureJsonFormatter.java | 65 +++++++++++++ ...kableCompletableFutureStringFormatter.java | 39 ++++++++ .../StringTrackableCompletableFutureTest.java | 94 ++++++++++++++----- 5 files changed, 184 insertions(+), 62 deletions(-) create mode 100644 TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFutureJsonFormatter.java create mode 100644 TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFutureStringFormatter.java 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 d2f20d72e..de4cbc52f 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 @@ -8,18 +8,17 @@ 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.DiagnosticTrackableCompletableFutureJsonFormatter; import org.opensearch.migrations.replay.util.OrderedWorkerTracker; import org.opensearch.migrations.tracing.ActiveContextTracker; import org.opensearch.migrations.tracing.ActiveContextTrackerByActivityType; import org.opensearch.migrations.tracing.CompositeContextTracker; -import org.opensearch.migrations.tracing.IScopedInstrumentationAttributes; import org.opensearch.migrations.tracing.RootOtelContext; import org.opensearch.migrations.transform.IAuthTransformer; import org.opensearch.migrations.transform.IAuthTransformerFactory; import org.opensearch.migrations.transform.IHttpMessage; import org.opensearch.migrations.transform.RemovingAuthTransformerFactory; import org.opensearch.migrations.transform.StaticAuthTransformerFactory; -import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.event.Level; import software.amazon.awssdk.arns.Arn; @@ -33,13 +32,10 @@ import java.nio.file.Files; import java.nio.file.Paths; import java.time.Duration; -import java.util.Iterator; import java.util.List; -import java.util.NoSuchElementException; import java.util.Optional; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; -import java.util.function.Function; @Slf4j public class TrafficReplayer { @@ -317,7 +313,7 @@ public static void main(String[] args) throws Exception { new TrafficStreamLimiter(params.maxConcurrentRequests), orderedRequestTracker); activeContextMonitor = new ActiveContextMonitor( globalContextTracker, perContextTracker, orderedRequestTracker, 64, - cf->cf.formatAsString(TrafficReplayerTopLevel::formatWorkItem), activeContextLogger); + cf-> DiagnosticTrackableCompletableFutureJsonFormatter.format(cf, TrafficReplayerTopLevel::formatWorkItem), activeContextLogger); ActiveContextMonitor finalActiveContextMonitor = activeContextMonitor; scheduledExecutorService.scheduleAtFixedRate(()->{ activeContextLogger.atInfo().setMessage(()->"Total requests outstanding: " + tr.requestWorkTracker.size()).log(); diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFuture.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFuture.java index 234e788f7..56227d8ab 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFuture.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFuture.java @@ -2,7 +2,6 @@ import lombok.Getter; import lombok.NonNull; -import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import java.time.Duration; @@ -17,7 +16,6 @@ import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; -import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.Stream; @@ -215,16 +213,6 @@ public T get(@NonNull Duration timeout) throws ExecutionException, InterruptedEx public boolean isDone() { return future.isDone(); } - @Override - public String toString() { - return formatAsString(x->null); - } - - public String formatAsString(@NonNull Function,String> resultFormatter) { - return walkParentsAsStream().map(kvp->formatFutureWithDiagnostics(kvp, resultFormatter)) - .collect(Collectors.joining("<-")); - } - public Stream> walkParentsAsStream() { AtomicReference> chainHeadReference = new AtomicReference<>(this); @@ -237,30 +225,16 @@ public Stream> walkParentsAsStream() { }); } - @SneakyThrows - protected String formatFutureWithDiagnostics( - @NonNull DiagnosticTrackableCompletableFuture dcf, - @NonNull Function,String> resultFormatter) { - var diagnosticInfo = dcf.diagnosticSupplier.get(); - var isDone = dcf.isDone(); - return "[" + System.identityHashCode(dcf) + "] " + diagnosticInfo + - (isDone ? formatWithDefault(resultFormatter, dcf) : - getPendingString(dcf, resultFormatter)); + @Override + public String toString() { + return formatAsString(x->null); } - private static String - getPendingString(DiagnosticTrackableCompletableFuture dcf, - Function, String> resultFormatter) { - return Optional.ofNullable(dcf.innerComposedPendingCompletableFutureReference) - .map(r -> (DiagnosticTrackableCompletableFuture) r.get()) - .map(df -> " --[[" + df.formatAsString(resultFormatter) + " ]] ") - .orElse("[…]"); + public String formatAsString(@NonNull Function,String> resultFormatter) { + return DiagnosticTrackableCompletableFutureStringFormatter.format(this, resultFormatter); } - private static String formatWithDefault( - @NonNull Function,String> formatter, - DiagnosticTrackableCompletableFuture df) { - var str = formatter.apply(df); - return "[" + (str == null ? "^" : str) + "]"; + public String formatAsJson(@NonNull Function,String> resultFormatter) { + return DiagnosticTrackableCompletableFutureJsonFormatter.format(this, resultFormatter); } } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFutureJsonFormatter.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFutureJsonFormatter.java new file mode 100644 index 000000000..52940b06e --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFutureJsonFormatter.java @@ -0,0 +1,65 @@ +package org.opensearch.migrations.replay.util; + +import com.fasterxml.jackson.databind.ObjectMapper; +import lombok.Lombok; +import lombok.NonNull; +import lombok.SneakyThrows; + +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.Function; +import java.util.stream.Collectors; + +public class DiagnosticTrackableCompletableFutureJsonFormatter { + + static ObjectMapper objectMapper = new ObjectMapper(); + + private DiagnosticTrackableCompletableFutureJsonFormatter() {} + + public static String format(DiagnosticTrackableCompletableFuture tf) { + return format(tf, x->null); + } + + public static String format(DiagnosticTrackableCompletableFuture tf, + @NonNull Function,String> resultFormatter) { + try { + return objectMapper.writeValueAsString(makeJson(tf, resultFormatter)); + } catch (Exception e) { + throw Lombok.sneakyThrow(e); + } + } + + public static List makeJson(DiagnosticTrackableCompletableFuture tf, + @NonNull Function,String> resultFormatter) { + return tf.walkParentsAsStream().map(kvp->jsonFormatFutureWithDiagnostics(kvp, resultFormatter)) + .collect(Collectors.toList()); + } + + @SneakyThrows + protected static Map jsonFormatFutureWithDiagnostics( + @NonNull DiagnosticTrackableCompletableFuture dcf, + @NonNull Function,String> resultFormatter) { + var diagnosticInfo = dcf.diagnosticSupplier.get(); + var isDone = dcf.isDone(); + var map = new LinkedHashMap(); + map.put("idHash", System.identityHashCode(dcf)); + map.put("label", diagnosticInfo); + if (isDone) { + map.put("value", Optional.ofNullable(resultFormatter.apply(dcf)).orElse("^")); + } else { + var innerResult = Optional.ofNullable(dcf.innerComposedPendingCompletableFutureReference) + .map(r -> (DiagnosticTrackableCompletableFuture) r.get()) + .map(df -> makeJson(df, resultFormatter)) + .orElse(null); + if (innerResult == null) { + map.put("value", "…"); + } else { + map.put("pending", innerResult); + } + } + return map; + } +} diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFutureStringFormatter.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFutureStringFormatter.java new file mode 100644 index 000000000..f5a070930 --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFutureStringFormatter.java @@ -0,0 +1,39 @@ +package org.opensearch.migrations.replay.util; + +import lombok.NonNull; +import lombok.SneakyThrows; + +import java.util.Optional; +import java.util.function.Function; +import java.util.stream.Collectors; + +public class DiagnosticTrackableCompletableFutureStringFormatter { + + private DiagnosticTrackableCompletableFutureStringFormatter() {} + + public static String format(DiagnosticTrackableCompletableFuture f) { + return format(f, x->null); + } + + public static String format(DiagnosticTrackableCompletableFuture f, + @NonNull Function,String> resultFormatter) { + return f.walkParentsAsStream().map(kvp-> stringFormatFutureWithDiagnostics(f, kvp, resultFormatter)) + .collect(Collectors.joining("<-")); + } + + @SneakyThrows + protected static String stringFormatFutureWithDiagnostics( + DiagnosticTrackableCompletableFuture f, + @NonNull DiagnosticTrackableCompletableFuture dcf, + @NonNull Function,String> resultFormatter) { + var diagnosticInfo = dcf.diagnosticSupplier.get(); + var isDone = dcf.isDone(); + return "[" + System.identityHashCode(dcf) + "] " + diagnosticInfo + + (isDone ? + "[" + Optional.ofNullable(resultFormatter.apply(dcf)).orElse("^") + "]" : + Optional.ofNullable(dcf.innerComposedPendingCompletableFutureReference) + .map(r -> (DiagnosticTrackableCompletableFuture) r.get()) + .map(df -> " --[[" + format(df, resultFormatter) + " ]] ") + .orElse("[…]")); + } +} diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/StringTrackableCompletableFutureTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/StringTrackableCompletableFutureTest.java index a804b7dc7..414d0b7cc 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/StringTrackableCompletableFutureTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/StringTrackableCompletableFutureTest.java @@ -31,44 +31,93 @@ public void futureWithThreeStages() throws Exception { return 1; }), ()->"A"); - var id1 = "[" + System.identityHashCode(stcf1) + "] "; - Assertions.assertEquals(id1 + "A[…]", stcf1.toString()); + final var id1 = System.identityHashCode(stcf1); + final var id1Bktd = "[" + id1 + "] "; + Assertions.assertEquals(id1Bktd + "A[…]", DiagnosticTrackableCompletableFutureStringFormatter.format(stcf1)); var stcf2 = stcf1.map(f->f.thenApplyAsync(x->{ sneakyWait(notifier2); return x*10+1; }), ()->"B"); - var id2 = "[" + System.identityHashCode(stcf2) + "] "; - Assertions.assertEquals(id2 + "B[…]<-" + id1 + "A[…]", stcf2.toString()); + final var id2 = System.identityHashCode(stcf2); + final var id2Bktd = "[" + id2 + "] "; + Assertions.assertEquals(id2Bktd + "B[…]<-" + id1Bktd + "A[…]", DiagnosticTrackableCompletableFutureStringFormatter.format(stcf2)); var stcf3 = stcf2.map(f->f.thenApplyAsync(x->{ sneakyWait(notifier3); return x*10+1; }), ()->"C"); - var id3 = "[" + System.identityHashCode(stcf3) + "] "; + final var id3 = System.identityHashCode(stcf3); + final var id3Bktd = "[" + id3 + "] "; - Assertions.assertEquals(id1 + "A[…]", stcf1.toString()); - Assertions.assertEquals(id2 + "B[…]<-" + id1 + "A[…]", stcf2.toString()); - Assertions.assertEquals(id3 + "C[…]<-" + id2 + "B[…]<-" + id1 + "A[…]", stcf3.toString()); + Assertions.assertEquals(id1Bktd + "A[…]", DiagnosticTrackableCompletableFutureStringFormatter.format(stcf1)); + Assertions.assertEquals(id2Bktd + "B[…]<-" + id1Bktd + "A[…]", DiagnosticTrackableCompletableFutureStringFormatter.format(stcf2)); + Assertions.assertEquals(id3Bktd + "C[…]<-" + id2Bktd + "B[…]<-" + id1Bktd + "A[…]", DiagnosticTrackableCompletableFutureStringFormatter.format(stcf3)); - notifyAndCheckNewDiagnosticValue(stcf1, notifier1, id1 + "A[^]"); - Assertions.assertEquals(id2 + "B[…]<-" + id1 + "A[^]", stcf2.toString()); - Assertions.assertEquals(id3 + "C[…]<-" + id2 + "B[…]<-" + id1 + "A[^]", stcf3.toString()); - Assertions.assertEquals(id3 + "C[…]<-" + id2 + "B[…]<-" + id1 + "A[1]", + Assertions.assertEquals("[{\"idHash\":" + id1 + ",\"label\":\"A\",\"value\":\"…\"}]", + DiagnosticTrackableCompletableFutureJsonFormatter.format(stcf1)); + Assertions.assertEquals("[" + + "{\"idHash\":" + id2 + ",\"label\":\"B\",\"value\":\"…\"}," + + "{\"idHash\":" + id1 + ",\"label\":\"A\",\"value\":\"…\"}]", DiagnosticTrackableCompletableFutureJsonFormatter.format(stcf2)); + Assertions.assertEquals("[" + + "{\"idHash\":" + id3 + ",\"label\":\"C\",\"value\":\"…\"}," + + "{\"idHash\":" + id2 + ",\"label\":\"B\",\"value\":\"…\"}," + + "{\"idHash\":" + id1 + ",\"label\":\"A\",\"value\":\"…\"}]", DiagnosticTrackableCompletableFutureJsonFormatter.format(stcf3)); + + + notifyAndWaitForGet(stcf1, notifier1); + Assertions.assertEquals(id1Bktd + "A[^]", DiagnosticTrackableCompletableFutureStringFormatter.format(stcf1)); + Assertions.assertEquals(id2Bktd + "B[…]<-" + id1Bktd + "A[^]", DiagnosticTrackableCompletableFutureStringFormatter.format(stcf2)); + Assertions.assertEquals(id3Bktd + "C[…]<-" + id2Bktd + "B[…]<-" + id1Bktd + "A[^]", DiagnosticTrackableCompletableFutureStringFormatter.format(stcf3)); + Assertions.assertEquals(id3Bktd + "C[…]<-" + id2Bktd + "B[…]<-" + id1Bktd + "A[1]", stcf3.formatAsString(StringTrackableCompletableFutureTest::formatCompletableFuture)); - notifyAndCheckNewDiagnosticValue(stcf2, notifier2, id2 + "B[^]<-" + id1 + "A[^]"); - Assertions.assertEquals(id1 + "A[^]", stcf1.toString()); - Assertions.assertEquals(id3 + "C[…]<-" + id2 + "B[^]<-" + id1 + "A[^]", stcf3.toString()); - Assertions.assertEquals(id3 + "C[…]<-" + id2 + "B[11]<-" + id1 + "A[1]", + + Assertions.assertEquals("[{\"idHash\":" + id1 + ",\"label\":\"A\",\"value\":\"^\"}]", + DiagnosticTrackableCompletableFutureJsonFormatter.format(stcf1)); + Assertions.assertEquals("[" + + "{\"idHash\":" + id2 + ",\"label\":\"B\",\"value\":\"…\"}," + + "{\"idHash\":" + id1 + ",\"label\":\"A\",\"value\":\"^\"}]", DiagnosticTrackableCompletableFutureJsonFormatter.format(stcf2)); + Assertions.assertEquals("[" + + "{\"idHash\":" + id3 + ",\"label\":\"C\",\"value\":\"…\"}," + + "{\"idHash\":" + id2 + ",\"label\":\"B\",\"value\":\"…\"}," + + "{\"idHash\":" + id1 + ",\"label\":\"A\",\"value\":\"^\"}]", DiagnosticTrackableCompletableFutureJsonFormatter.format(stcf3)); + + notifyAndWaitForGet(stcf2, notifier2); + Assertions.assertEquals(id2Bktd + "B[^]<-" + id1Bktd + "A[^]", DiagnosticTrackableCompletableFutureStringFormatter.format(stcf2)); + Assertions.assertEquals(id1Bktd + "A[^]", DiagnosticTrackableCompletableFutureStringFormatter.format(stcf1)); + Assertions.assertEquals(id3Bktd + "C[…]<-" + id2Bktd + "B[^]<-" + id1Bktd + "A[^]", DiagnosticTrackableCompletableFutureStringFormatter.format(stcf3)); + Assertions.assertEquals(id3Bktd + "C[…]<-" + id2Bktd + "B[11]<-" + id1Bktd + "A[1]", stcf3.formatAsString(StringTrackableCompletableFutureTest::formatCompletableFuture)); + + + + Assertions.assertEquals("[{\"idHash\":" + id1 + ",\"label\":\"A\",\"value\":\"^\"}]", + DiagnosticTrackableCompletableFutureJsonFormatter.format(stcf1)); + Assertions.assertEquals("[" + + "{\"idHash\":" + id2 + ",\"label\":\"B\",\"value\":\"^\"}," + + "{\"idHash\":" + id1 + ",\"label\":\"A\",\"value\":\"^\"}]", DiagnosticTrackableCompletableFutureJsonFormatter.format(stcf2)); + Assertions.assertEquals("[" + + "{\"idHash\":" + id3 + ",\"label\":\"C\",\"value\":\"…\"}," + + "{\"idHash\":" + id2 + ",\"label\":\"B\",\"value\":\"^\"}," + + "{\"idHash\":" + id1 + ",\"label\":\"A\",\"value\":\"^\"}]", DiagnosticTrackableCompletableFutureJsonFormatter.format(stcf3)); + Assertions.assertEquals("[" + + "{\"idHash\":" + id3 + ",\"label\":\"C\",\"value\":\"…\"}," + + "{\"idHash\":" + id2 + ",\"label\":\"B\",\"value\":\"11\"}," + + "{\"idHash\":" + id1 + ",\"label\":\"A\",\"value\":\"1\"}]", + stcf3.formatAsJson(StringTrackableCompletableFutureTest::formatCompletableFuture)); + // A is clipped because of grandparent culling - notifyAndCheckNewDiagnosticValue(stcf3, notifier3, - id3 + "C[^]<-" + id2 +"B[^]"); - Assertions.assertEquals(id1 + "A[^]", stcf1.toString()); - Assertions.assertEquals(id2 + "B[^]", stcf2.toString()); + notifyAndWaitForGet(stcf3, notifier3); + Assertions.assertEquals(id3Bktd + "C[^]<-" + id2Bktd +"B[^]", DiagnosticTrackableCompletableFutureStringFormatter.format(stcf3)); + Assertions.assertEquals(id1Bktd + "A[^]", DiagnosticTrackableCompletableFutureStringFormatter.format(stcf1)); + Assertions.assertEquals(id2Bktd + "B[^]", DiagnosticTrackableCompletableFutureStringFormatter.format(stcf2)); + Assertions.assertEquals("[" + + "{\"idHash\":" + id3 + ",\"label\":\"C\",\"value\":\"111\"}," + + "{\"idHash\":" + id2 + ",\"label\":\"B\",\"value\":\"11\"}]", + stcf3.formatAsJson(StringTrackableCompletableFutureTest::formatCompletableFuture)); } public static String formatCompletableFuture(DiagnosticTrackableCompletableFuture cf) { @@ -82,10 +131,9 @@ public static String formatCompletableFuture(DiagnosticTrackableCompletableFutur } } - private void notifyAndCheckNewDiagnosticValue(DiagnosticTrackableCompletableFuture stcf, - CompletableFuture lockObject, String expectedValue) throws Exception { + private void notifyAndWaitForGet(DiagnosticTrackableCompletableFuture stcf, + CompletableFuture lockObject) throws Exception { notify(lockObject); stcf.get(); - Assertions.assertEquals(expectedValue, stcf.toString()); } } \ No newline at end of file From e35ca8cd3e024c618c0c18ccb22025f09c91b456 Mon Sep 17 00:00:00 2001 From: Greg Schohn Date: Sun, 28 Apr 2024 17:21:43 -0400 Subject: [PATCH 09/11] Convert remaining netty Future 'addListener' calls to use NettyToCompletableFutureBinders + other assorted tweaks. I've also stripped the longRunningActivity file of the log levels and timestamps. They were really just getting in the way of understanding what was going on and being able to quickly copy-paste from there into a json file for further analysis. Signed-off-by: Greg Schohn --- .../NettyToCompletableFutureBinders.java | 5 +- .../replay/ClientConnectionPool.java | 90 +++------- .../replay/RequestSenderOrchestrator.java | 4 +- .../migrations/replay/TrafficReplayer.java | 4 +- .../NettyPacketToHttpConsumer.java | 166 +++++++----------- .../datatypes/ConnectionReplaySession.java | 11 +- .../TimeToResponseFulfillmentFutureMap.java | 11 +- .../replay/util/OnlineRadixSorter.java | 16 +- .../src/main/resources/log4j2.properties | 2 +- .../NettyPacketToHttpConsumerTest.java | 7 +- .../e2etests/FullTrafficReplayerTest.java | 6 +- 11 files changed, 120 insertions(+), 202 deletions(-) diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/NettyToCompletableFutureBinders.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/NettyToCompletableFutureBinders.java index 155de099a..7a945f0f9 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/NettyToCompletableFutureBinders.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/NettyToCompletableFutureBinders.java @@ -12,6 +12,9 @@ import java.util.function.Supplier; public class NettyToCompletableFutureBinders { + + private NettyToCompletableFutureBinders() {} + public static CompletableFuture bindNettyFutureToCompletableFuture(Future nettyFuture, CompletableFuture cf) { nettyFuture.addListener(f -> { @@ -54,7 +57,7 @@ public class NettyToCompletableFutureBinders { bindNettyScheduleToCompletableFuture(EventLoop eventLoop, Duration delay) { var delayMs = Math.max(0, delay.toMillis()); return bindNettyFutureToTrackableFuture(eventLoop.schedule(() -> {}, delayMs, TimeUnit.MILLISECONDS), - "scheduling to run next send at " + delay + " in " + delayMs +" ms (clipped)"); + "scheduling to run next send in " + delay + " (clipped: " + delayMs + "ms)"); } public static CompletableFuture 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 d3f30a550..783712600 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 @@ -8,25 +8,21 @@ import io.netty.channel.EventLoop; import io.netty.channel.nio.NioEventLoopGroup; import io.netty.handler.ssl.SslContext; -import io.netty.util.concurrent.DefaultPromise; 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; import lombok.extern.slf4j.Slf4j; +import org.opensearch.migrations.NettyToCompletableFutureBinders; import org.opensearch.migrations.replay.datahandlers.NettyPacketToHttpConsumer; import org.opensearch.migrations.replay.datatypes.ConnectionReplaySession; import org.opensearch.migrations.replay.tracing.IReplayContexts; import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; -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 { @@ -81,41 +77,24 @@ public ConnectionReplaySession buildConnectionReplaySession(IReplayContexts.ICha private DiagnosticTrackableCompletableFuture getResilientClientChannelProducer(EventLoop eventLoop, IReplayContexts.IChannelKeyContext connectionContext) { return new AdaptiveRateLimiter() - .get(() -> { - var channelFuture = NettyPacketToHttpConsumer.createClientConnection(eventLoop, - sslContext, serverUri, connectionContext, timeout); - return getCompletedChannelFutureAsCompletableFuture(connectionContext, channelFuture); - }); - } - - public static StringTrackableCompletableFuture - getCompletedChannelFutureAsCompletableFuture(IReplayContexts.IChannelKeyContext connectionContext, - ChannelFuture channelFuture) { - var clientConnectionChannelCreatedFuture = - new StringTrackableCompletableFuture("waiting for createClientConnection to finish"); - channelFuture.addListener(f -> { - log.atInfo().setMessage(()-> - "New network connection result for " + connectionContext + "=" + f.isSuccess()).log(); - if (f.isSuccess()) { - clientConnectionChannelCreatedFuture.future.complete(channelFuture); - } else { - clientConnectionChannelCreatedFuture.future.completeExceptionally(f.cause()); - } - }); - return clientConnectionChannelCreatedFuture; + .get(() -> + NettyPacketToHttpConsumer.createClientConnection(eventLoop, + sslContext, serverUri, connectionContext, timeout) + .whenComplete((v,t)-> { + if (t == null) { + log.atDebug().setMessage(() -> "New network connection result for " + + connectionContext + " =" + v).log(); + } else { + log.atInfo().setMessage(() -> "got exception for " + connectionContext) + .setCause(t).log(); + } + }, () -> "waiting for createClientConnection to finish")); } public CompletableFuture shutdownNow() { CompletableFuture shutdownFuture = new CompletableFuture<>(); connectionId2ChannelCache.invalidateAll(); - eventLoopGroup.shutdownGracefully().addListener(f->{ - if (f.isSuccess()) { - shutdownFuture.complete(null); - } else { - shutdownFuture.completeExceptionally(f.cause()); - } - }); - return shutdownFuture; + return NettyToCompletableFutureBinders.bindNettyFutureToCompletableFuture(eventLoopGroup.shutdownGracefully()); } public void closeConnection(IReplayContexts.IChannelKeyContext ctx, int sessionNumber) { @@ -143,25 +122,16 @@ public void closeConnection(IReplayContexts.IChannelKeyContext ctx, int sessionN private DiagnosticTrackableCompletableFuture closeClientConnectionChannel(ConnectionReplaySession channelAndFutureWork) { - var channelClosedFuture = - new StringTrackableCompletableFuture("Waiting for closeFuture() on channel"); - - channelAndFutureWork.getFutureThatReturnsChannelFuture(false) - .thenAccept(channelFuture-> { - if (channelFuture == null) { - return; - } + return channelAndFutureWork.getFutureThatReturnsChannelFutureInAnyState(false) + .thenCompose(channelFuture-> { log.atTrace().setMessage(() -> "closing channel " + channelFuture.channel() + "(" + channelAndFutureWork.getChannelKeyContext() + ")...").log(); - channelFuture.channel().close() - .addListener(closeFuture -> { + return NettyToCompletableFutureBinders.bindNettyFutureToTrackableFuture( + channelFuture.channel().close(), + "calling channel.close()") + .thenApply(v -> { log.atTrace().setMessage(() -> "channel.close() has finished for " + - channelAndFutureWork.getChannelKeyContext()).log(); - if (closeFuture.isSuccess()) { - channelClosedFuture.future.complete(channelFuture.channel()); - } else { - channelClosedFuture.future.completeExceptionally(closeFuture.cause()); - } + channelAndFutureWork.getChannelKeyContext() + " with value=" + v).log(); if (channelAndFutureWork.hasWorkRemaining()) { log.atWarn().setMessage(() -> "Work items are still remaining for this connection session" + @@ -170,19 +140,9 @@ public void closeConnection(IReplayContexts.IChannelKeyContext ctx, int sessionN "). " + channelAndFutureWork.calculateSizeSlowly() + " requests that were enqueued won't be run").log(); } - var schedule = channelAndFutureWork.schedule; - while (channelAndFutureWork.schedule.hasPendingTransmissions()) { - var scheduledItemToKill = schedule.peekFirstItem(); - schedule.removeFirstItem(); - } - }); - }, () -> "calling channel.close()") - .exceptionally(t->{ - log.atWarn().setMessage(()->"client connection encountered an exception while closing") - .setCause(t).log(); - channelClosedFuture.future.completeExceptionally(t); - return null; - }, () -> "handling any potential exceptions"); - return channelClosedFuture; + channelAndFutureWork.schedule.clear(); + return channelFuture.channel(); + }, () -> "clearing work"); + }, () -> ""); } } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/RequestSenderOrchestrator.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/RequestSenderOrchestrator.java index 094e36e1d..6709b863c 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/RequestSenderOrchestrator.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/RequestSenderOrchestrator.java @@ -244,8 +244,8 @@ private Duration getDelayFromNowMs(Instant to) { return consumeFuture.thenCompose(cf -> NettyToCompletableFutureBinders.bindNettyScheduleToCompletableFuture(eventLoop, Duration.between(now(), startAt.plus(interval.multipliedBy(counter.get())))) - .getDeferredFutureThroughHandle((v,t)-> sendSendingRestOfPackets(packetReceiver, eventLoop, - iterator, startAt, interval, counter), () -> "sending next packet"), + .thenCompose(v -> sendSendingRestOfPackets(packetReceiver, eventLoop, iterator, + startAt, interval, counter), () -> "sending next packet"), () -> "recursing, once ready"); } else { return consumeFuture.getDeferredFutureThroughHandle((v,t) -> packetReceiver.finalizeRequest(), 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 de4cbc52f..e149cd847 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 @@ -32,6 +32,7 @@ import java.nio.file.Files; import java.nio.file.Paths; import java.time.Duration; +import java.time.Instant; import java.util.List; import java.util.Optional; import java.util.concurrent.Executors; @@ -316,7 +317,8 @@ public static void main(String[] args) throws Exception { cf-> DiagnosticTrackableCompletableFutureJsonFormatter.format(cf, TrafficReplayerTopLevel::formatWorkItem), activeContextLogger); ActiveContextMonitor finalActiveContextMonitor = activeContextMonitor; scheduledExecutorService.scheduleAtFixedRate(()->{ - activeContextLogger.atInfo().setMessage(()->"Total requests outstanding: " + tr.requestWorkTracker.size()).log(); + activeContextLogger.atInfo().setMessage(()->"Total requests outstanding at " + Instant.now() + + ": " + tr.requestWorkTracker.size()).log(); finalActiveContextMonitor.run(); }, ACTIVE_WORK_MONITOR_CADENCE_MS, ACTIVE_WORK_MONITOR_CADENCE_MS, TimeUnit.MILLISECONDS); 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 17aa3db3f..e64aac458 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 @@ -3,7 +3,6 @@ import io.netty.bootstrap.Bootstrap; import io.netty.buffer.ByteBuf; import io.netty.channel.Channel; -import io.netty.channel.ChannelDuplexHandler; import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelFutureListener; import io.netty.channel.ChannelHandlerContext; @@ -19,8 +18,10 @@ import io.netty.handler.ssl.SslContext; import io.netty.handler.ssl.SslHandler; import io.netty.handler.timeout.ReadTimeoutHandler; +import lombok.Lombok; import lombok.NonNull; import lombok.extern.slf4j.Slf4j; +import org.opensearch.migrations.NettyToCompletableFutureBinders; import org.opensearch.migrations.replay.AggregatedRawResponse; import org.opensearch.migrations.replay.datahandlers.http.helpers.ReadMeteringHandler; import org.opensearch.migrations.replay.datahandlers.http.helpers.WriteMeteringHandler; @@ -41,7 +42,6 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; @Slf4j public class NettyPacketToHttpConsumer implements IPacketFinalizingConsumer { @@ -89,43 +89,40 @@ public NettyPacketToHttpConsumer(ConnectionReplaySession replaySession, var parentContext = ctx.createTargetRequestContext(); this.setCurrentMessageContext(parentContext.createHttpSendingContext()); responseBuilder = AggregatedRawResponse.builder(Instant.now()); - this.activeChannelFuture = new StringTrackableCompletableFuture<>( - () -> "incoming connection is ready for " + replaySession); - var initialFuture = this.activeChannelFuture; - - log.atDebug().setMessage(() -> - "C'tor: incoming session=" + replaySession).log(); - activateLiveChannel(initialFuture); + log.atDebug().setMessage(() -> "C'tor: incoming session=" + replaySession).log(); + this.activeChannelFuture = activateLiveChannel(); } - private void activateLiveChannel(DiagnosticTrackableCompletableFuture initialFuture) { - replaySession.getFutureThatReturnsChannelFuture(true).thenAccept(channelFuture-> { - channelFuture.addListener(connectFuture -> { - final var ctx = replaySession.getChannelKeyContext(); - if (connectFuture.isSuccess()) { - final var c = channelFuture.channel(); - if (c.isActive()) { - this.channel = c; - initializeChannelPipeline(); - log.atDebug().setMessage(()->"Channel initialized for " + ctx + " signaling future").log(); - initialFuture.future.complete(null); - } else { - // this may loop forever - until the event loop is shutdown - // (see the ClientConnectionPool::shutdownNow()) - ctx.addFailedChannelCreation(); - log.atWarn().setMessage(()->"Channel wasn't active, trying to create another for this request") - .log(); - activateLiveChannel(initialFuture); - } - } else { - ctx.addFailedChannelCreation(); - ctx.addTraceException(channelFuture.cause(), true); - log.atWarn().setMessage(()->"error creating channel, not retrying") - .setCause(connectFuture.cause()).log(); - initialFuture.future.completeExceptionally(connectFuture.cause()); - } - }); - }, () -> "creating an alive connection"); + private DiagnosticTrackableCompletableFuture activateLiveChannel() { + final var ctx = replaySession.getChannelKeyContext(); + return replaySession.getFutureThatReturnsChannelFutureInAnyState(true) + .thenCompose(channelFuture -> NettyToCompletableFutureBinders.bindNettyFutureToTrackableFuture(channelFuture, + "waiting for newly acquired channel to be ready") + .getDeferredFutureThroughHandle((connectFuture,t)->{ + if (t != null) { + ctx.addFailedChannelCreation(); + ctx.addTraceException(channelFuture.cause(), true); + log.atWarn().setMessage(()->"error creating channel, not retrying") + .setCause(t).log(); + throw Lombok.sneakyThrow(t); + } + + final var c = channelFuture.channel(); + if (c.isActive()) { + this.channel = c; + initializeChannelPipeline(); + log.atDebug().setMessage(()->"Channel initialized for " + ctx + " signaling future").log(); + return StringTrackableCompletableFuture.completedFuture(null, ()->"Done"); + } else { + // this may recurse forever - until the event loop is shutdown + // (see the ClientConnectionPool::shutdownNow()) + ctx.addFailedChannelCreation(); + log.atWarn().setMessage(()->"Channel wasn't active, trying to create another for this request") + .log(); + return activateLiveChannel(); + } + }, () -> "acting on ready channelFuture to retry if inactive or to return"), + () -> "taking newly acquired channel and making it active"); } private & @@ -142,11 +139,12 @@ public IReplayContexts.ITargetRequestContext getParentContext() { return currentRequestContextUnion.getLogicalEnclosingScope(); } - public static ChannelFuture createClientConnection(EventLoopGroup eventLoopGroup, - SslContext sslContext, - URI serverUri, - IReplayContexts.IChannelKeyContext channelKeyContext, - Duration timeout) { + public static DiagnosticTrackableCompletableFuture + createClientConnection(EventLoopGroup eventLoopGroup, + SslContext sslContext, + URI serverUri, + 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(); @@ -165,38 +163,29 @@ protected void initChannel(@NonNull Channel ch) throws Exception { var outboundChannelFuture = b.connect(host, port); - var rval = new DefaultChannelPromise(outboundChannelFuture.channel()); - outboundChannelFuture.addListener((ChannelFutureListener) connectFuture -> { - if (connectFuture.isSuccess()) { - final var channel = connectFuture.channel(); - log.atTrace().setMessage(()-> channelKeyContext.getChannelKey() + - " Done setting up client channel & it was successful for " + channel).log(); - var pipeline = channel.pipeline(); - if (sslContext != null) { - var sslEngine = sslContext.newEngine(channel.alloc()); - sslEngine.setUseClientMode(true); - var sslHandler = new SslHandler(sslEngine); - addLoggingHandlerLast(pipeline, "A"); - pipeline.addLast(SSL_HANDLER_NAME, sslHandler); - sslHandler.handshakeFuture().addListener(handshakeFuture -> { - if (handshakeFuture.isSuccess()) { - rval.setSuccess(); + return NettyToCompletableFutureBinders.bindNettyFutureToTrackableFuture(outboundChannelFuture, "") + .thenCompose(voidVal-> { + if (outboundChannelFuture.isSuccess()) { + final var channel = outboundChannelFuture.channel(); + log.atTrace().setMessage(() -> channelKeyContext.getChannelKey() + + " Done setting up client channel & it was successful for " + channel).log(); + var pipeline = channel.pipeline(); + if (sslContext != null) { + var sslEngine = sslContext.newEngine(channel.alloc()); + sslEngine.setUseClientMode(true); + var sslHandler = new SslHandler(sslEngine); + addLoggingHandlerLast(pipeline, "A"); + pipeline.addLast(SSL_HANDLER_NAME, sslHandler); + return NettyToCompletableFutureBinders.bindNettyFutureToTrackableFuture(sslHandler.handshakeFuture(), + ()->"") + .thenApply(voidVal2->outboundChannelFuture, ()->""); } else { - rval.setFailure(handshakeFuture.cause()); + return StringTrackableCompletableFuture.completedFuture(outboundChannelFuture, ()->""); } - }); - } else { - rval.setSuccess(); - } - } else { - // Close the connection if the connection attempt has failed. - log.atWarn().setCause(connectFuture.cause()) - .setMessage(() -> channelKeyContext.getChannelKey() + " CONNECT future was not successful, " + - "so setting the channel future's result to an exception").log(); - rval.setFailure(connectFuture.cause()); - } - }); - return rval; + } else { + return StringTrackableCompletableFuture.failedFuture(outboundChannelFuture.cause(), ()->""); + } + }, () -> ""); } private static boolean channelIsInUse(Channel c) { @@ -309,37 +298,8 @@ private IReplayContexts.IReplayerHttpTransactionContext httpContext() { private DiagnosticTrackableCompletableFuture writePacketAndUpdateFuture(ByteBuf packetData) { - final var completableFuture = new DiagnosticTrackableCompletableFuture(new CompletableFuture<>(), - ()->"CompletableFuture that will wait for the netty future to fill in the completion value"); - channel.writeAndFlush(packetData) - .addListener((ChannelFutureListener) future -> { - Throwable cause = null; - try { - if (!future.isSuccess()) { - log.atWarn().setMessage(()-> httpContext().getReplayerRequestKey() + "closing outbound channel " + - "because WRITE future was not successful " + future.cause() + " hash=" + - System.identityHashCode(packetData) + " will be sending the exception to " + - completableFuture).log(); - future.channel().close(); // close the backside - cause = future.cause(); - } - } catch (Exception e) { - cause = e; - } - if (cause == null) { - log.atTrace().setMessage(()->"Previously returned CompletableFuture packet write was " + - "successful: " + packetData + " hash=" + System.identityHashCode(packetData)).log(); - completableFuture.future.complete(null); - } else { - log.atInfo().setMessage(()->"Previously returned CompletableFuture packet write had " + - " an exception :" + packetData + " hash=" + System.identityHashCode(packetData)).log(); - completableFuture.future.completeExceptionally(cause); - channel.close(); - } - }); - log.atTrace().setMessage(()->"Writing packet data=" + packetData + - ". Created future for writing data="+completableFuture).log(); - return completableFuture; + return NettyToCompletableFutureBinders.bindNettyFutureToTrackableFuture(channel.writeAndFlush(packetData), + "CompletableFuture that will wait for the netty future to fill in the completion value"); } @Override diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/ConnectionReplaySession.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/ConnectionReplaySession.java index a1d1f0df7..4fe33d8ab 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/ConnectionReplaySession.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/ConnectionReplaySession.java @@ -2,7 +2,6 @@ import io.netty.channel.ChannelFuture; import io.netty.channel.EventLoop; -import lombok.AllArgsConstructor; import lombok.Getter; import lombok.NonNull; import lombok.SneakyThrows; @@ -13,13 +12,7 @@ import org.opensearch.migrations.replay.util.StringTrackableCompletableFuture; import java.io.IOException; -import java.util.Comparator; -import java.util.Map; -import java.util.Optional; -import java.util.PriorityQueue; -import java.util.concurrent.CompletableFuture; import java.util.function.Supplier; -import java.util.function.UnaryOperator; /** * This class contains everything that is needed to replay packets to a specific channel. @@ -61,7 +54,7 @@ public ConnectionReplaySession(EventLoop eventLoop, IReplayContexts.IChannelKeyC } public DiagnosticTrackableCompletableFuture - getFutureThatReturnsChannelFuture(boolean requireActiveChannel) { + getFutureThatReturnsChannelFutureInAnyState(boolean requireActiveChannel) { StringTrackableCompletableFuture eventLoopFuture = new StringTrackableCompletableFuture<>("procuring a connection"); eventLoop.submit(() -> { @@ -113,6 +106,6 @@ public boolean hasWorkRemaining() { } public long calculateSizeSlowly() { - return schedule.calculateSizeSlowly() + scheduleSequencer.size(); + return (long) schedule.timeToRunnableMap.size() + scheduleSequencer.size(); } } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/TimeToResponseFulfillmentFutureMap.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/TimeToResponseFulfillmentFutureMap.java index 255952d86..da1fb9bf3 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/TimeToResponseFulfillmentFutureMap.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/TimeToResponseFulfillmentFutureMap.java @@ -10,6 +10,7 @@ public class TimeToResponseFulfillmentFutureMap { + public static class FutureWorkPoint { public final Instant startTime; public final DiagnosticTrackableCompletableFuture scheduleFuture; @@ -42,6 +43,10 @@ public boolean isEmpty() { return timeToRunnableMap.isEmpty(); } + public void clear() { + timeToRunnableMap.clear(); + } + public boolean hasPendingTransmissions() { if (timeToRunnableMap.isEmpty()) { return false; @@ -50,13 +55,9 @@ public boolean hasPendingTransmissions() { } } - public long calculateSizeSlowly() { - return timeToRunnableMap.size(); - } - @Override public String toString() { - return "[" + this.calculateSizeSlowly() + "]: {" + formatBookends() + "}"; + return "[" + (long) timeToRunnableMap.size() + "]: {" + formatBookends() + "}"; } private String formatBookends() { diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java index 7979e95a0..e96e8a158 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java @@ -72,13 +72,13 @@ public OnlineRadixSorter(int startingOffset) { */ public DiagnosticTrackableCompletableFuture addFutureForWork(final int index, FutureTransformer processor) { - var oldWorkItem = items.get(index); - if (oldWorkItem == null) { + var workItem = items.get(index); + if (workItem == null) { if (index < currentOffset) { throw new IllegalArgumentException("index (" + index + ")" + " must be > last processed item (" + currentOffset + ")"); } - for (int nextKey = Math.max(currentOffset, items.isEmpty() ? 0 : items.lastKey()+1); + for (int nextKey = Math.max(currentOffset, items.isEmpty() ? Integer.MIN_VALUE : items.lastKey()+1); nextKey<=index; ++nextKey) { int finalNextKey = nextKey; @@ -86,20 +86,20 @@ public OnlineRadixSorter(int startingOffset) { new StringTrackableCompletableFuture( CompletableFuture.completedFuture(null), "unlinked signaling future for slot #" + finalNextKey) : - items.get(items.lastKey()).signalWorkCompletedFuture + items.get(finalNextKey-1).signalWorkCompletedFuture .thenAccept(v-> {}, ()->"Kickoff for slot #" + finalNextKey); - oldWorkItem = new IndexedWork(signalFuture, null, + workItem = new IndexedWork(signalFuture, null, new StringTrackableCompletableFuture(()->"Work to finish for slot #" + finalNextKey + " is awaiting [" + getAwaitingText() + "]")); - oldWorkItem.signalWorkCompletedFuture.whenComplete((v,t)->{ + workItem.signalWorkCompletedFuture.whenComplete((v,t)->{ ++currentOffset; items.remove(finalNextKey); }, ()->"cleaning up spent work for idx #" + finalNextKey); - items.put(nextKey, oldWorkItem); + items.put(nextKey, workItem); } } - return oldWorkItem.addWorkFuture(processor, index); + return workItem.addWorkFuture(processor, index); } public String getAwaitingText() { diff --git a/TrafficCapture/trafficReplayer/src/main/resources/log4j2.properties b/TrafficCapture/trafficReplayer/src/main/resources/log4j2.properties index 36a75d032..1fb20bd93 100644 --- a/TrafficCapture/trafficReplayer/src/main/resources/log4j2.properties +++ b/TrafficCapture/trafficReplayer/src/main/resources/log4j2.properties @@ -63,7 +63,7 @@ appender.ALL_ACTIVE_WORK_MONITOR_LOGFILE.name = AllActiveWorkMonitorFile appender.ALL_ACTIVE_WORK_MONITOR_LOGFILE.fileName = logs/longRunningActivity.log appender.ALL_ACTIVE_WORK_MONITOR_LOGFILE.filePattern = logs/%d{yyyy-MM}{UTC}/longRunningActivity-%d{yyyy-MM-dd-HH-mm}{UTC}-%i.log.gz appender.ALL_ACTIVE_WORK_MONITOR_LOGFILE.layout.type = PatternLayout -appender.ALL_ACTIVE_WORK_MONITOR_LOGFILE.layout.pattern = %msg ([%-5level] %d{yyyy-MM-dd HH:mm:ss,SSS}{UTC})%n +appender.ALL_ACTIVE_WORK_MONITOR_LOGFILE.layout.pattern = %msg%n appender.ALL_ACTIVE_WORK_MONITOR_LOGFILE.policies.type = Policies appender.ALL_ACTIVE_WORK_MONITOR_LOGFILE.policies.time.type = TimeBasedTriggeringPolicy appender.ALL_ACTIVE_WORK_MONITOR_LOGFILE.policies.time.interval = 60 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 8a01d9dae..5a44c3054 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 @@ -15,6 +15,7 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.ValueSource; +import org.opensearch.migrations.NettyToCompletableFutureBinders; import org.opensearch.migrations.replay.AggregatedRawResponse; import org.opensearch.migrations.replay.ClientConnectionPool; import org.opensearch.migrations.replay.PacketToTransformingHttpHandlerFactory; @@ -149,10 +150,8 @@ public void testHttpResponseIsSuccessfullyCaptured(boolean useTls, boolean large var channelContext = httpContext.getChannelKeyContext(); var eventLoop = new NioEventLoopGroup(1, new DefaultThreadFactory("test")).next(); var replaySession = new ConnectionReplaySession(eventLoop, channelContext, - () -> ClientConnectionPool.getCompletedChannelFutureAsCompletableFuture( - httpContext.getChannelKeyContext(), - NettyPacketToHttpConsumer.createClientConnection(eventLoop, sslContext, - testServer.localhostEndpoint(), channelContext, REGULAR_RESPONSE_TIMEOUT))); + () -> NettyPacketToHttpConsumer.createClientConnection(eventLoop, sslContext, + 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(); 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 d49e5ac9a..1020d9757 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 @@ -300,9 +300,9 @@ public void makeSureThatCollateralDamageDoesntFreezeTests() throws Throwable { @ParameterizedTest @CsvSource(value = { -// "3,false", -// "-1,false", -// "3,true", + "3,false", + "-1,false", + "3,true", "-1,true", }) @Tag("longTest") From 3d76386e9937c17ce321fa3bb99d3b01de4ba710 Mon Sep 17 00:00:00 2001 From: Greg Schohn Date: Mon, 29 Apr 2024 09:26:27 -0400 Subject: [PATCH 10/11] Syntactic refactoring. No behavioral changes. Rename long classes to be shorter. s/DiagnosticTrackableCompletableFuture/TrackedFuture/ s/NettyToCompletableFutureBinders/NettyFutureBinders/ Identifier names were also updated. Where types could be implicitly deduced, code was updated so that these type names didn't need to be specified. Signed-off-by: Greg Schohn --- .../replay/AddCompressionEncodingTest.java | 9 +- ...reBinders.java => NettyFutureBinders.java} | 22 ++--- .../replay/AdaptiveRateLimiter.java | 6 +- .../replay/ClientConnectionPool.java | 12 +-- .../migrations/replay/ReplayEngine.java | 14 +-- .../replay/RequestSenderOrchestrator.java | 57 ++++++----- .../migrations/replay/TrafficReplayer.java | 4 +- .../replay/TrafficReplayerCore.java | 26 ++--- .../replay/TrafficReplayerTopLevel.java | 33 +++---- .../replay/datahandlers/IPacketConsumer.java | 6 +- .../IPacketFinalizingConsumer.java | 4 +- .../NettyPacketToHttpConsumer.java | 38 ++++---- .../TransformedPacketReceiver.java | 10 +- .../http/HttpJsonTransformingConsumer.java | 39 ++++---- ...ttySendByteBufsToPacketHandlerHandler.java | 30 +++--- .../datatypes/ConnectionReplaySession.java | 18 ++-- .../replay/datatypes/FutureTransformer.java | 5 +- .../TimeToResponseFulfillmentFutureMap.java | 8 +- .../replay/util/ActiveContextMonitor.java | 8 +- ...kableCompletableFutureStringFormatter.java | 39 -------- .../replay/util/OnlineRadixSorter.java | 21 ++-- .../replay/util/OrderedWorkerTracker.java | 6 +- .../StringTrackableCompletableFuture.java | 51 ---------- .../replay/util/TextTrackedFuture.java | 51 ++++++++++ ...pletableFuture.java => TrackedFuture.java} | 97 +++++++++---------- ...r.java => TrackedFutureJsonFormatter.java} | 31 +++--- .../util/TrackedFutureStringFormatter.java | 39 ++++++++ .../src/main/resources/log4j2.properties | 3 +- .../replay/HeaderTransformerTest.java | 6 +- .../replay/RequestSenderOrchestratorTest.java | 32 +++--- .../NettyPacketToHttpConsumerTest.java | 1 - .../replay/util/ActiveContextMonitorTest.java | 12 +-- .../replay/util/OnlineRadixSorterTest.java | 32 +++--- ...reTest.java => TextTrackedFutureTest.java} | 62 ++++++------ ...FutureTest.java => TrackedFutureTest.java} | 27 +++--- .../TestCapturePacketToHttpHandler.java | 12 +-- .../migrations/replay/TestUtils.java | 16 +-- .../OnlineRadixSorterForIntegratedKeys.java | 2 +- 38 files changed, 439 insertions(+), 450 deletions(-) rename TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/{NettyToCompletableFutureBinders.java => NettyFutureBinders.java} (71%) delete mode 100644 TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFutureStringFormatter.java delete mode 100644 TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/StringTrackableCompletableFuture.java create mode 100644 TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/TextTrackedFuture.java rename TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/{DiagnosticTrackableCompletableFuture.java => TrackedFuture.java} (67%) rename TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/{DiagnosticTrackableCompletableFutureJsonFormatter.java => TrackedFutureJsonFormatter.java} (53%) create mode 100644 TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/TrackedFutureStringFormatter.java rename TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/{StringTrackableCompletableFutureTest.java => TextTrackedFutureTest.java} (64%) rename TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/{DiagnosticTrackableCompletableFutureTest.java => TrackedFutureTest.java} (52%) diff --git a/TrafficCapture/replayerPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/java/org/opensearch/migrations/replay/AddCompressionEncodingTest.java b/TrafficCapture/replayerPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/java/org/opensearch/migrations/replay/AddCompressionEncodingTest.java index 3eac3e974..f520d9907 100644 --- a/TrafficCapture/replayerPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/java/org/opensearch/migrations/replay/AddCompressionEncodingTest.java +++ b/TrafficCapture/replayerPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/java/org/opensearch/migrations/replay/AddCompressionEncodingTest.java @@ -10,7 +10,7 @@ import org.junit.jupiter.api.Test; import org.opensearch.migrations.replay.datahandlers.http.HttpJsonTransformingConsumer; import org.opensearch.migrations.replay.datatypes.HttpRequestTransformationStatus; -import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; +import org.opensearch.migrations.replay.util.TrackedFuture; import org.opensearch.migrations.tracing.InstrumentationTest; import org.opensearch.migrations.transform.JsonJoltTransformBuilder; import org.opensearch.migrations.transform.JsonJoltTransformer; @@ -49,10 +49,9 @@ public void addingCompressionRequestHeaderCompressesPayload() throws ExecutionEx "host: localhost\n" + "content-length: " + (numParts * payloadPartSize) + "\n"; - DiagnosticTrackableCompletableFuture tail = - compressingTransformer.consumeBytes(sourceHeaders.getBytes(StandardCharsets.UTF_8)) - .thenCompose(v -> compressingTransformer.consumeBytes("\n".getBytes(StandardCharsets.UTF_8)), - () -> "AddCompressionEncodingTest.compressingTransformer"); + var tail = compressingTransformer.consumeBytes(sourceHeaders.getBytes(StandardCharsets.UTF_8)) + .thenCompose(v -> compressingTransformer.consumeBytes("\n".getBytes(StandardCharsets.UTF_8)), + () -> "AddCompressionEncodingTest.compressingTransformer"); final byte[] payloadPart = new byte[payloadPartSize]; Arrays.fill(payloadPart, BYTE_FILL_VALUE); for (var i = new AtomicInteger(numParts); i.get() > 0; i.decrementAndGet()) { diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/NettyToCompletableFutureBinders.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/NettyFutureBinders.java similarity index 71% rename from TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/NettyToCompletableFutureBinders.java rename to TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/NettyFutureBinders.java index 7a945f0f9..c4d2ce1ae 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/NettyToCompletableFutureBinders.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/NettyFutureBinders.java @@ -2,8 +2,8 @@ import io.netty.channel.EventLoop; import io.netty.util.concurrent.Future; -import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; -import org.opensearch.migrations.replay.util.StringTrackableCompletableFuture; +import org.opensearch.migrations.replay.util.TrackedFuture; +import org.opensearch.migrations.replay.util.TextTrackedFuture; import java.time.Duration; import java.util.concurrent.CompletableFuture; @@ -11,9 +11,9 @@ import java.util.function.Function; import java.util.function.Supplier; -public class NettyToCompletableFutureBinders { +public class NettyFutureBinders { - private NettyToCompletableFutureBinders() {} + private NettyFutureBinders() {} public static CompletableFuture bindNettyFutureToCompletableFuture(Future nettyFuture, CompletableFuture cf) { @@ -32,28 +32,28 @@ private NettyToCompletableFutureBinders() {} return bindNettyFutureToCompletableFuture(nettyFuture, new CompletableFuture<>()); } - public static DiagnosticTrackableCompletableFuture + public static TrackedFuture bindNettyFutureToTrackableFuture(Future nettyFuture, String label) { - return new StringTrackableCompletableFuture<>(bindNettyFutureToCompletableFuture(nettyFuture), label); + return new TextTrackedFuture<>(bindNettyFutureToCompletableFuture(nettyFuture), label); } - public static DiagnosticTrackableCompletableFuture + public static TrackedFuture bindNettyFutureToTrackableFuture(Future nettyFuture, Supplier labelProvider) { - return new StringTrackableCompletableFuture<>(bindNettyFutureToCompletableFuture(nettyFuture), labelProvider); + return new TextTrackedFuture<>(bindNettyFutureToCompletableFuture(nettyFuture), labelProvider); } - public static DiagnosticTrackableCompletableFuture + public static TrackedFuture bindNettyFutureToTrackableFuture(Function> nettyFutureGenerator, String label) { return bindNettyFutureToTrackableFuture(nettyFutureGenerator.apply(() -> { }), label); } - public static DiagnosticTrackableCompletableFuture + public static TrackedFuture bindNettySubmitToTrackableFuture(EventLoop eventLoop) { return bindNettyFutureToTrackableFuture(eventLoop::submit, "waiting for event loop submission"); } - public static DiagnosticTrackableCompletableFuture + public static TrackedFuture bindNettyScheduleToCompletableFuture(EventLoop eventLoop, Duration delay) { var delayMs = Math.max(0, delay.toMillis()); return bindNettyFutureToTrackableFuture(eventLoop.schedule(() -> {}, delayMs, TimeUnit.MILLISECONDS), diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/AdaptiveRateLimiter.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/AdaptiveRateLimiter.java index 2a1f669f9..0e0b67255 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/AdaptiveRateLimiter.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/AdaptiveRateLimiter.java @@ -5,7 +5,7 @@ import io.github.resilience4j.ratelimiter.RateLimiterConfig; import io.github.resilience4j.retry.Retry; import io.github.resilience4j.retry.RetryConfig; -import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; +import org.opensearch.migrations.replay.util.TrackedFuture; import java.time.Duration; import java.util.function.Supplier; @@ -22,8 +22,8 @@ */ public class AdaptiveRateLimiter { - public DiagnosticTrackableCompletableFuture - get(Supplier> producer) { + public TrackedFuture + get(Supplier> producer) { var intervalFunction = IntervalFunction.ofExponentialBackoff(Duration.ofMillis(1),2,Duration.ofSeconds(1)); RetryConfig retryConfig = RetryConfig.custom() .maxAttempts(Integer.MAX_VALUE) 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 783712600..336009e98 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 @@ -14,11 +14,11 @@ import lombok.NonNull; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; -import org.opensearch.migrations.NettyToCompletableFutureBinders; +import org.opensearch.migrations.NettyFutureBinders; import org.opensearch.migrations.replay.datahandlers.NettyPacketToHttpConsumer; import org.opensearch.migrations.replay.datatypes.ConnectionReplaySession; import org.opensearch.migrations.replay.tracing.IReplayContexts; -import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; +import org.opensearch.migrations.replay.util.TrackedFuture; import java.net.URI; import java.time.Duration; @@ -74,7 +74,7 @@ public ConnectionReplaySession buildConnectionReplaySession(IReplayContexts.ICha ()->getResilientClientChannelProducer(eventLoop, channelKeyCtx)); } - private DiagnosticTrackableCompletableFuture + private TrackedFuture getResilientClientChannelProducer(EventLoop eventLoop, IReplayContexts.IChannelKeyContext connectionContext) { return new AdaptiveRateLimiter() .get(() -> @@ -94,7 +94,7 @@ public ConnectionReplaySession buildConnectionReplaySession(IReplayContexts.ICha public CompletableFuture shutdownNow() { CompletableFuture shutdownFuture = new CompletableFuture<>(); connectionId2ChannelCache.invalidateAll(); - return NettyToCompletableFutureBinders.bindNettyFutureToCompletableFuture(eventLoopGroup.shutdownGracefully()); + return NettyFutureBinders.bindNettyFutureToCompletableFuture(eventLoopGroup.shutdownGracefully()); } public void closeConnection(IReplayContexts.IChannelKeyContext ctx, int sessionNumber) { @@ -120,13 +120,13 @@ public void closeConnection(IReplayContexts.IChannelKeyContext ctx, int sessionN return crs; } - private DiagnosticTrackableCompletableFuture + private TrackedFuture closeClientConnectionChannel(ConnectionReplaySession channelAndFutureWork) { return channelAndFutureWork.getFutureThatReturnsChannelFutureInAnyState(false) .thenCompose(channelFuture-> { log.atTrace().setMessage(() -> "closing channel " + channelFuture.channel() + "(" + channelAndFutureWork.getChannelKeyContext() + ")...").log(); - return NettyToCompletableFutureBinders.bindNettyFutureToTrackableFuture( + return NettyFutureBinders.bindNettyFutureToTrackableFuture( channelFuture.channel().close(), "calling channel.close()") .thenApply(v -> { diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/ReplayEngine.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/ReplayEngine.java index d2afe7bec..d6c6288b4 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/ReplayEngine.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/ReplayEngine.java @@ -6,7 +6,7 @@ import org.opensearch.migrations.replay.datatypes.IndexedChannelInteraction; import org.opensearch.migrations.replay.tracing.IReplayContexts; import org.opensearch.migrations.replay.traffic.source.BufferedFlowController; -import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; +import org.opensearch.migrations.replay.util.TrackedFuture; import java.time.Duration; import java.time.Instant; @@ -101,8 +101,8 @@ public boolean isWorkOutstanding() { return totalCountOfScheduledTasksOutstanding.get() > 0; } - private DiagnosticTrackableCompletableFuture - hookWorkFinishingUpdates(DiagnosticTrackableCompletableFuture future, Instant timestamp, + private TrackedFuture + hookWorkFinishingUpdates(TrackedFuture future, Instant timestamp, Object stringableKey, String taskDescription) { return future.map(f->f .whenComplete((v,t)->Utils.setIfLater(lastCompletedSourceTimeEpochMs, timestamp.toEpochMilli())) @@ -124,9 +124,9 @@ private static void logStartOfWork(Object stringableKey, long newCount, Instant ") to run at " + start + " incremented tasksOutstanding to "+ newCount).log(); } - public DiagnosticTrackableCompletableFuture + public TrackedFuture scheduleTransformationWork(IReplayContexts.IReplayerHttpTransactionContext requestCtx, Instant originalStart, - Supplier> task) { + Supplier> task) { var newCount = totalCountOfScheduledTasksOutstanding.incrementAndGet(); final String label = "processing"; var start = timeShifter.transformSourceTimeToRealTime(originalStart); @@ -136,7 +136,7 @@ private static void logStartOfWork(Object stringableKey, long newCount, Instant return hookWorkFinishingUpdates(result, originalStart, requestCtx, label); } - public DiagnosticTrackableCompletableFuture + public TrackedFuture scheduleRequest(IReplayContexts.IReplayerHttpTransactionContext ctx, Instant originalStart, Instant originalEnd, int numPackets, Stream packets) { @@ -154,7 +154,7 @@ private static void logStartOfWork(Object stringableKey, long newCount, Instant return hookWorkFinishingUpdates(sendResult, originalStart, requestKey, label); } - public DiagnosticTrackableCompletableFuture + public TrackedFuture closeConnection(int channelInteractionNum, IReplayContexts.IChannelKeyContext ctx, int channelSessionNumber, Instant timestamp) { diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/RequestSenderOrchestrator.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/RequestSenderOrchestrator.java index 6709b863c..ec5e38f6a 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/RequestSenderOrchestrator.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/RequestSenderOrchestrator.java @@ -3,17 +3,16 @@ import io.netty.buffer.ByteBuf; import io.netty.channel.EventLoop; import lombok.extern.slf4j.Slf4j; -import org.opensearch.migrations.NettyToCompletableFutureBinders; +import org.opensearch.migrations.NettyFutureBinders; import org.opensearch.migrations.replay.datahandlers.IPacketFinalizingConsumer; -import org.opensearch.migrations.replay.datahandlers.NettyPacketToHttpConsumer; import org.opensearch.migrations.replay.datatypes.ChannelTask; import org.opensearch.migrations.replay.datatypes.ChannelTaskType; import org.opensearch.migrations.replay.datatypes.ConnectionReplaySession; import org.opensearch.migrations.replay.datatypes.IndexedChannelInteraction; import org.opensearch.migrations.replay.datatypes.UniqueReplayerRequestKey; import org.opensearch.migrations.replay.tracing.IReplayContexts; -import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; -import org.opensearch.migrations.replay.util.StringTrackableCompletableFuture; +import org.opensearch.migrations.replay.util.TrackedFuture; +import org.opensearch.migrations.replay.util.TextTrackedFuture; import java.time.Duration; import java.time.Instant; @@ -29,24 +28,24 @@ @Slf4j public class RequestSenderOrchestrator { - public DiagnosticTrackableCompletableFuture + public TrackedFuture bindNettyScheduleToCompletableFuture(EventLoop eventLoop, Instant timestamp) { - return NettyToCompletableFutureBinders.bindNettyScheduleToCompletableFuture(eventLoop, getDelayFromNowMs(timestamp)); + return NettyFutureBinders.bindNettyScheduleToCompletableFuture(eventLoop, getDelayFromNowMs(timestamp)); } - public StringTrackableCompletableFuture + public TextTrackedFuture bindNettyScheduleToCompletableFuture(EventLoop eventLoop, Instant timestamp, - DiagnosticTrackableCompletableFuture existingFuture) { + TrackedFuture existingFuture) { var delayMs = getDelayFromNowMs(timestamp); - NettyToCompletableFutureBinders.bindNettyScheduleToCompletableFuture(eventLoop, delayMs, existingFuture.future); - return new StringTrackableCompletableFuture<>(existingFuture.future, + NettyFutureBinders.bindNettyScheduleToCompletableFuture(eventLoop, delayMs, existingFuture.future); + return new TextTrackedFuture<>(existingFuture.future, "scheduling to run next send at " + timestamp + " in " + delayMs + "ms"); } public CompletableFuture bindNettyScheduleToCompletableFuture(EventLoop eventLoop, Instant timestamp, CompletableFuture cf) { - return NettyToCompletableFutureBinders.bindNettyScheduleToCompletableFuture(eventLoop, + return NettyFutureBinders.bindNettyScheduleToCompletableFuture(eventLoop, getDelayFromNowMs(timestamp), cf); } @@ -64,9 +63,9 @@ public RequestSenderOrchestrator(ClientConnectionPool clientConnectionPool, this.packetConsumerFactory = packetConsumerFactory; } - public DiagnosticTrackableCompletableFuture + public TrackedFuture scheduleWork(IReplayContexts.IReplayerHttpTransactionContext ctx, Instant timestamp, - Supplier> task) { + Supplier> task) { var connectionSession = clientConnectionPool.getCachedSession(ctx.getChannelKeyContext(), ctx.getReplayerRequestKey().sourceRequestIndexSessionIdentifier); log.atDebug().setMessage(()->"Scheduling work for "+ctx.getConnectionId()+" at time "+timestamp).log(); @@ -90,12 +89,12 @@ public RequestSenderOrchestrator(ClientConnectionPool clientConnectionPool, if (scheduleFailure == null) { return task.get(); } else { - return StringTrackableCompletableFuture.failedFuture(scheduleFailure, ()->"netty scheduling failure"); + return TextTrackedFuture.failedFuture(scheduleFailure, ()->"netty scheduling failure"); } }, ()->"The scheduled callback is running work for " + ctx); } - public DiagnosticTrackableCompletableFuture + public TrackedFuture scheduleRequest(UniqueReplayerRequestKey requestKey, IReplayContexts.IReplayerHttpTransactionContext ctx, Instant start, Duration interval, Stream packets) { var sessionNumber = requestKey.sourceRequestIndexSessionIdentifier; @@ -111,10 +110,10 @@ public RequestSenderOrchestrator(ClientConnectionPool clientConnectionPool, connectionReplaySession, start, interval, packets)); } - public DiagnosticTrackableCompletableFuture scheduleClose(IReplayContexts.IChannelKeyContext ctx, - int sessionNumber, - int channelInteractionNum, - Instant timestamp) { + public TrackedFuture scheduleClose(IReplayContexts.IChannelKeyContext ctx, + int sessionNumber, + int channelInteractionNum, + Instant timestamp) { var channelKey = ctx.getChannelKey(); var channelInteraction = new IndexedChannelInteraction(channelKey, channelInteractionNum); log.atDebug().setMessage(() -> "Scheduling CLOSE for " + channelInteraction + " at time " + timestamp).log(); @@ -131,14 +130,14 @@ public DiagnosticTrackableCompletableFuture scheduleClose(IReplayCo * have been called. This method isn't concerned with scheduling items to run at a specific time, that is * left up to the callback. */ - private DiagnosticTrackableCompletableFuture + private TrackedFuture submitUnorderedWorkToEventLoop(IReplayContexts.IChannelKeyContext ctx, int sessionNumber, int channelInteractionNumber, - Function> + Function> onSessionCallback) { final var replaySession = clientConnectionPool.getCachedSession(ctx, sessionNumber); - return NettyToCompletableFutureBinders.bindNettySubmitToTrackableFuture(replaySession.eventLoop) + return NettyFutureBinders.bindNettySubmitToTrackableFuture(replaySession.eventLoop) .getDeferredFutureThroughHandle((v,t) -> { log.atTrace().setMessage(() -> "adding work item at slot " + channelInteractionNumber + " for " + replaySession.getChannelKeyContext() + " with " + @@ -149,7 +148,7 @@ public DiagnosticTrackableCompletableFuture scheduleClose(IReplayCo }, () -> "Waiting for sequencer to finish for slot " + channelInteractionNumber); } - private DiagnosticTrackableCompletableFuture + private TrackedFuture scheduleSendRequestOnConnectionReplaySession(IReplayContexts.IReplayerHttpTransactionContext ctx, ConnectionReplaySession connectionReplaySession, Instant startTime, @@ -169,7 +168,7 @@ public DiagnosticTrackableCompletableFuture scheduleClose(IReplayCo }, ()->"sending packets for request"))); } - private DiagnosticTrackableCompletableFuture + private TrackedFuture scheduleCloseOnConnectionReplaySession(IReplayContexts.IChannelKeyContext ctx, ConnectionReplaySession connectionReplaySession, Instant timestamp, @@ -179,14 +178,14 @@ public DiagnosticTrackableCompletableFuture scheduleClose(IReplayCo var diagnosticCtx = new IndexedChannelInteraction(ctx.getChannelKey(), channelInteractionNum); return scheduleOnConnectionReplaySession(diagnosticCtx, connectionReplaySession, timestamp, new ChannelTask<>(ChannelTaskType.CLOSE, - dcf -> dcf.whenComplete((v,t) -> { + tf -> tf.whenComplete((v,t) -> { log.trace("Calling closeConnection at slot " + channelInteraction); clientConnectionPool.closeConnection(ctx, connectionReplaySessionNum); }, () -> "Close connection") )); } - private DiagnosticTrackableCompletableFuture + private TrackedFuture scheduleOnConnectionReplaySession(IndexedChannelInteraction channelInteraction, ConnectionReplaySession channelFutureAndRequestSchedule, Instant atTime, @@ -228,7 +227,7 @@ private Duration getDelayFromNowMs(Instant to) { return Duration.ofMillis(Math.max(0, Duration.between(now(), to).toMillis())); } - private DiagnosticTrackableCompletableFuture + private TrackedFuture sendSendingRestOfPackets(IPacketFinalizingConsumer packetReceiver, EventLoop eventLoop, Iterator iterator, @@ -241,8 +240,8 @@ private Duration getDelayFromNowMs(Instant to) { var consumeFuture = packetReceiver.consumeBytes(iterator.next()); if (iterator.hasNext()) { - return consumeFuture.thenCompose(cf -> - NettyToCompletableFutureBinders.bindNettyScheduleToCompletableFuture(eventLoop, + return consumeFuture.thenCompose(tf -> + NettyFutureBinders.bindNettyScheduleToCompletableFuture(eventLoop, Duration.between(now(), startAt.plus(interval.multipliedBy(counter.get())))) .thenCompose(v -> sendSendingRestOfPackets(packetReceiver, eventLoop, iterator, startAt, interval, counter), () -> "sending next packet"), 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 e149cd847..85450010d 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 @@ -8,7 +8,7 @@ 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.DiagnosticTrackableCompletableFutureJsonFormatter; +import org.opensearch.migrations.replay.util.TrackedFutureJsonFormatter; import org.opensearch.migrations.replay.util.OrderedWorkerTracker; import org.opensearch.migrations.tracing.ActiveContextTracker; import org.opensearch.migrations.tracing.ActiveContextTrackerByActivityType; @@ -314,7 +314,7 @@ public static void main(String[] args) throws Exception { new TrafficStreamLimiter(params.maxConcurrentRequests), orderedRequestTracker); activeContextMonitor = new ActiveContextMonitor( globalContextTracker, perContextTracker, orderedRequestTracker, 64, - cf-> DiagnosticTrackableCompletableFutureJsonFormatter.format(cf, TrafficReplayerTopLevel::formatWorkItem), activeContextLogger); + cf-> TrackedFutureJsonFormatter.format(cf, TrafficReplayerTopLevel::formatWorkItem), activeContextLogger); ActiveContextMonitor finalActiveContextMonitor = activeContextMonitor; scheduledExecutorService.scheduleAtFixedRate(()->{ activeContextLogger.atInfo().setMessage(()->"Total requests outstanding at " + Instant.now() + 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 11c3e492b..c304d9ee4 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 @@ -16,8 +16,8 @@ import org.opensearch.migrations.replay.traffic.source.ITrafficCaptureSource; import org.opensearch.migrations.replay.traffic.source.ITrafficStreamWithKey; import org.opensearch.migrations.replay.traffic.source.TrafficStreamLimiter; -import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; -import org.opensearch.migrations.replay.util.StringTrackableCompletableFuture; +import org.opensearch.migrations.replay.util.TrackedFuture; +import org.opensearch.migrations.replay.util.TextTrackedFuture; import org.opensearch.migrations.trafficcapture.protos.TrafficStreamUtils; import org.opensearch.migrations.transform.IAuthTransformerFactory; import org.opensearch.migrations.transform.IJsonTransformer; @@ -46,7 +46,7 @@ public abstract class TrafficReplayerCore { public interface IWorkTracker { void put(UniqueReplayerRequestKey uniqueReplayerRequestKey, - DiagnosticTrackableCompletableFuture completableFuture); + TrackedFuture completableFuture); void remove(UniqueReplayerRequestKey uniqueReplayerRequestKey); boolean isEmpty(); int size(); @@ -108,15 +108,15 @@ class TrafficReplayerAccumulationCallbacks implements AccumulationCallbacks { var requestKey = ctx.getReplayerRequestKey(); var finishedAccumulatingResponseFuture = - new StringTrackableCompletableFuture( + new TextTrackedFuture( ()->"waiting for response to be accumulated for " + ctx); - finishedAccumulatingResponseFuture.future.whenComplete((v,t)-> log.atInfo() + finishedAccumulatingResponseFuture.future.whenComplete((v,t)-> log.atDebug() .setMessage(() -> "Done receiving captured stream for " + ctx + ":" + v.requestData).log()); var allWorkFinishedForTransactionFuture = sendRequestAfterGoingThroughWorkQueue(ctx, request, requestKey) .getDeferredFutureThroughHandle((arr,httpRequestException) -> finishedAccumulatingResponseFuture .thenCompose(rrPair-> - StringTrackableCompletableFuture.completedFuture( + TextTrackedFuture.completedFuture( handleCompletedTransaction(ctx, rrPair, arr, httpRequestException), ()->"Synchronously committed results"), () -> "logging summary"), @@ -129,12 +129,12 @@ class TrafficReplayerAccumulationCallbacks implements AccumulationCallbacks { return finishedAccumulatingResponseFuture.future::complete; } - private DiagnosticTrackableCompletableFuture + private TrackedFuture sendRequestAfterGoingThroughWorkQueue(IReplayContexts.IReplayerHttpTransactionContext ctx, HttpMessageAndTimestamp request, UniqueReplayerRequestKey requestKey) { var workDequeuedByLimiterFuture = - new StringTrackableCompletableFuture( + new TextTrackedFuture( ()->"waiting for " + ctx + " to be queued and run through TrafficStreamLimiter"); var wi = liveTrafficStreamLimiter.queueWork(1, ctx, workDequeuedByLimiterFuture.future::complete); var httpSentRequestFuture = workDequeuedByLimiterFuture @@ -240,7 +240,7 @@ private void packageAndWriteResponse(IReplayContexts.ITupleHandlingContext tuple log.trace("done sending and finalizing data to the packet handler"); try (var requestResponseTuple = getSourceTargetCaptureTuple(tupleHandlingContext, rrPair, summary, t)) { - log.atInfo().setMessage(()->"Source/Target Request/Response tuple: " + requestResponseTuple).log(); + log.atDebug().setMessage(()->"Source/Target Request/Response tuple: " + requestResponseTuple).log(); tupleWriter.accept(requestResponseTuple); } @@ -284,7 +284,7 @@ private void packageAndWriteResponse(IReplayContexts.ITupleHandlingContext tuple return requestResponseTuple; } - public DiagnosticTrackableCompletableFuture + public TrackedFuture transformAndSendRequest(ReplayEngine replayEngine, HttpMessageAndTimestamp request, IReplayContexts.IReplayerHttpTransactionContext ctx) { return transformAndSendRequest(inputRequestTransformerFactory, replayEngine, ctx, @@ -292,7 +292,7 @@ private void packageAndWriteResponse(IReplayContexts.ITupleHandlingContext tuple request.packetBytes::stream); } - public static DiagnosticTrackableCompletableFuture + public static TrackedFuture transformAndSendRequest(PacketToTransformingHttpHandlerFactory inputRequestTransformerFactory, ReplayEngine replayEngine, IReplayContexts.IReplayerHttpTransactionContext ctx, @@ -324,11 +324,11 @@ private void packageAndWriteResponse(IReplayContexts.ITupleHandlingContext tuple ()->"Checking for exception out of sending data to the target server"); } catch (Exception e) { log.debug("Caught exception in writeToSocket, so failing future"); - return StringTrackableCompletableFuture.failedFuture(e, ()->"TrafficReplayer.writeToSocketAndClose"); + return TextTrackedFuture.failedFuture(e, ()->"TrafficReplayer.writeToSocketAndClose"); } } - private static DiagnosticTrackableCompletableFuture + private static TrackedFuture transformAllData(IPacketFinalizingConsumer packetHandler, Supplier> packetSupplier) { try { var logLabel = packetHandler.getClass().getSimpleName(); 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 5b9553660..d24dd9e34 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 @@ -9,11 +9,10 @@ import org.opensearch.migrations.replay.datahandlers.NettyPacketToHttpConsumer; import org.opensearch.migrations.replay.datatypes.UniqueReplayerRequestKey; import org.opensearch.migrations.replay.tracing.IRootReplayerContext; -import org.opensearch.migrations.replay.tracing.RootReplayerContext; import org.opensearch.migrations.replay.traffic.source.BlockingTrafficSource; import org.opensearch.migrations.replay.traffic.source.TrafficStreamLimiter; -import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; -import org.opensearch.migrations.replay.util.StringTrackableCompletableFuture; +import org.opensearch.migrations.replay.util.TrackedFuture; +import org.opensearch.migrations.replay.util.TextTrackedFuture; import org.opensearch.migrations.transform.IAuthTransformerFactory; import org.opensearch.migrations.transform.IJsonTransformer; import org.slf4j.event.Level; @@ -44,17 +43,17 @@ public class TrafficReplayerTopLevel extends TrafficReplayerCore implements Auto public static final AtomicInteger targetConnectionPoolUniqueCounter = new AtomicInteger(); public interface IStreamableWorkTracker extends IWorkTracker { - public Stream>> + public Stream>> getRemainingItems(); } static class ConcurrentHashMapWorkTracker implements IStreamableWorkTracker { - ConcurrentHashMap> map = + ConcurrentHashMap> map = new ConcurrentHashMap<>(); @Override public void put(UniqueReplayerRequestKey uniqueReplayerRequestKey, - DiagnosticTrackableCompletableFuture completableFuture) { + TrackedFuture completableFuture) { map.put(uniqueReplayerRequestKey, completableFuture); } @@ -73,13 +72,13 @@ public int size() { return map.size(); } - public Stream>> + public Stream>> getRemainingItems() { return map.entrySet().stream(); } } - private final AtomicReference> allRemainingWorkFutureOrShutdownSignalRef; + private final AtomicReference> allRemainingWorkFutureOrShutdownSignalRef; private final AtomicReference shutdownReasonRef; private final AtomicReference> shutdownFutureRef; @@ -238,16 +237,16 @@ protected void waitForRemainingWork(Level logLevel, @NonNull Duration timeout) } var workTracker = (IStreamableWorkTracker) requestWorkTracker; - Map.Entry>[] + Map.Entry>[] allRemainingWorkArray = workTracker.getRemainingItems().toArray(Map.Entry[]::new); writeStatusLogsForRemainingWork(logLevel, allRemainingWorkArray); // remember, this block is ONLY for the leftover items. Lots of other items have been processed // and were removed from the live map (hopefully) - DiagnosticTrackableCompletableFuture[] allCompletableFuturesArray = + TrackedFuture[] allCompletableFuturesArray = Arrays.stream(allRemainingWorkArray) - .map(Map.Entry::getValue).toArray(DiagnosticTrackableCompletableFuture[]::new); - var allWorkFuture = StringTrackableCompletableFuture.allOf(allCompletableFuturesArray, + .map(Map.Entry::getValue).toArray(TrackedFuture[]::new); + var allWorkFuture = TextTrackedFuture.allOf(allCompletableFuturesArray, () -> "TrafficReplayer.AllWorkFinished"); try { if (allRemainingWorkFutureOrShutdownSignalRef.compareAndSet(null, allWorkFuture)) { @@ -292,8 +291,8 @@ private void handleAlreadySetFinishedSignal() throws InterruptedException, Execu protected static void writeStatusLogsForRemainingWork(Level logLevel, Map.Entry>[] + TrackedFuture>[] allRemainingWorkArray) { log.atLevel(logLevel).log("All remaining work to wait on " + allRemainingWorkArray.length); if (log.isInfoEnabled()) { @@ -309,7 +308,7 @@ protected static void writeStatusLogsForRemainingWork(Level logLevel, } } - static String formatWorkItem(DiagnosticTrackableCompletableFuture cf) { + static String formatWorkItem(TrackedFuture cf) { try { var resultValue = cf.get(); if (resultValue instanceof TransformedTargetRequestAndResponse) { @@ -350,8 +349,8 @@ static String formatWorkItem(DiagnosticTrackableCompletableFuture cf) }); Optional.ofNullable(this.nextChunkFutureRef.get()).ifPresent(f -> f.cancel(true)); var shutdownWasSignalledFuture = error == null ? - StringTrackableCompletableFuture.completedFuture(null, () -> "TrafficReplayer shutdown") : - StringTrackableCompletableFuture.failedFuture(error, () -> "TrafficReplayer shutdown"); + TextTrackedFuture.completedFuture(null, () -> "TrafficReplayer shutdown") : + TextTrackedFuture.failedFuture(error, () -> "TrafficReplayer shutdown"); while (!allRemainingWorkFutureOrShutdownSignalRef.compareAndSet(null, shutdownWasSignalledFuture)) { var otherRemainingWorkObj = allRemainingWorkFutureOrShutdownSignalRef.get(); if (otherRemainingWorkObj != null) { diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/IPacketConsumer.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/IPacketConsumer.java index 8108e0df4..8b7b22a70 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/IPacketConsumer.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/IPacketConsumer.java @@ -2,7 +2,7 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; -import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; +import org.opensearch.migrations.replay.util.TrackedFuture; /** * This class consumes arrays of bytes or ByteBufs, potentially asynchronously, @@ -10,12 +10,12 @@ */ public interface IPacketConsumer { - default DiagnosticTrackableCompletableFuture consumeBytes(byte[] nextRequestPacket) { + default TrackedFuture consumeBytes(byte[] nextRequestPacket) { var bb = Unpooled.wrappedBuffer(nextRequestPacket).retain(); var rval = consumeBytes(bb); bb.release(); return rval; } - DiagnosticTrackableCompletableFuture consumeBytes(ByteBuf nextRequestPacket); + TrackedFuture consumeBytes(ByteBuf nextRequestPacket); } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/IPacketFinalizingConsumer.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/IPacketFinalizingConsumer.java index 3b57ac70d..20ba55008 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/IPacketFinalizingConsumer.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/IPacketFinalizingConsumer.java @@ -1,6 +1,6 @@ package org.opensearch.migrations.replay.datahandlers; -import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; +import org.opensearch.migrations.replay.util.TrackedFuture; /** * This class consumes arrays of bytes or ByteBufs, potentially asynchronously, @@ -11,5 +11,5 @@ */ public interface IPacketFinalizingConsumer extends IPacketConsumer { - DiagnosticTrackableCompletableFuture finalizeRequest(); + TrackedFuture finalizeRequest(); } 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 e64aac458..be046c885 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 @@ -4,12 +4,10 @@ import io.netty.buffer.ByteBuf; import io.netty.channel.Channel; import io.netty.channel.ChannelFuture; -import io.netty.channel.ChannelFutureListener; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelInitializer; import io.netty.channel.ChannelOption; import io.netty.channel.ChannelPipeline; -import io.netty.channel.DefaultChannelPromise; import io.netty.channel.EventLoopGroup; import io.netty.channel.socket.nio.NioSocketChannel; import io.netty.handler.codec.http.HttpResponseDecoder; @@ -21,7 +19,7 @@ import lombok.Lombok; import lombok.NonNull; import lombok.extern.slf4j.Slf4j; -import org.opensearch.migrations.NettyToCompletableFutureBinders; +import org.opensearch.migrations.NettyFutureBinders; import org.opensearch.migrations.replay.AggregatedRawResponse; import org.opensearch.migrations.replay.datahandlers.http.helpers.ReadMeteringHandler; import org.opensearch.migrations.replay.datahandlers.http.helpers.WriteMeteringHandler; @@ -29,8 +27,8 @@ import org.opensearch.migrations.replay.netty.BacksideHttpWatcherHandler; import org.opensearch.migrations.replay.netty.BacksideSnifferHandler; import org.opensearch.migrations.replay.tracing.IReplayContexts; -import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; -import org.opensearch.migrations.replay.util.StringTrackableCompletableFuture; +import org.opensearch.migrations.replay.util.TrackedFuture; +import org.opensearch.migrations.replay.util.TextTrackedFuture; import org.opensearch.migrations.tracing.IScopedInstrumentationAttributes; import org.opensearch.migrations.tracing.IWithTypedEnclosingScope; @@ -63,7 +61,7 @@ public class NettyPacketToHttpConsumer implements IPacketFinalizingConsumer activeChannelFuture; + TrackedFuture activeChannelFuture; ConnectionReplaySession replaySession; private Channel channel; AggregatedRawResponse.Builder responseBuilder; @@ -93,10 +91,10 @@ public NettyPacketToHttpConsumer(ConnectionReplaySession replaySession, this.activeChannelFuture = activateLiveChannel(); } - private DiagnosticTrackableCompletableFuture activateLiveChannel() { + private TrackedFuture activateLiveChannel() { final var ctx = replaySession.getChannelKeyContext(); return replaySession.getFutureThatReturnsChannelFutureInAnyState(true) - .thenCompose(channelFuture -> NettyToCompletableFutureBinders.bindNettyFutureToTrackableFuture(channelFuture, + .thenCompose(channelFuture -> NettyFutureBinders.bindNettyFutureToTrackableFuture(channelFuture, "waiting for newly acquired channel to be ready") .getDeferredFutureThroughHandle((connectFuture,t)->{ if (t != null) { @@ -112,7 +110,7 @@ private DiagnosticTrackableCompletableFuture activateLiveChannel() this.channel = c; initializeChannelPipeline(); log.atDebug().setMessage(()->"Channel initialized for " + ctx + " signaling future").log(); - return StringTrackableCompletableFuture.completedFuture(null, ()->"Done"); + return TextTrackedFuture.completedFuture(null, ()->"Done"); } else { // this may recurse forever - until the event loop is shutdown // (see the ClientConnectionPool::shutdownNow()) @@ -139,7 +137,7 @@ public IReplayContexts.ITargetRequestContext getParentContext() { return currentRequestContextUnion.getLogicalEnclosingScope(); } - public static DiagnosticTrackableCompletableFuture + public static TrackedFuture createClientConnection(EventLoopGroup eventLoopGroup, SslContext sslContext, URI serverUri, @@ -163,7 +161,7 @@ protected void initChannel(@NonNull Channel ch) throws Exception { var outboundChannelFuture = b.connect(host, port); - return NettyToCompletableFutureBinders.bindNettyFutureToTrackableFuture(outboundChannelFuture, "") + return NettyFutureBinders.bindNettyFutureToTrackableFuture(outboundChannelFuture, "") .thenCompose(voidVal-> { if (outboundChannelFuture.isSuccess()) { final var channel = outboundChannelFuture.channel(); @@ -176,14 +174,14 @@ protected void initChannel(@NonNull Channel ch) throws Exception { var sslHandler = new SslHandler(sslEngine); addLoggingHandlerLast(pipeline, "A"); pipeline.addLast(SSL_HANDLER_NAME, sslHandler); - return NettyToCompletableFutureBinders.bindNettyFutureToTrackableFuture(sslHandler.handshakeFuture(), + return NettyFutureBinders.bindNettyFutureToTrackableFuture(sslHandler.handshakeFuture(), ()->"") .thenApply(voidVal2->outboundChannelFuture, ()->""); } else { - return StringTrackableCompletableFuture.completedFuture(outboundChannelFuture, ()->""); + return TextTrackedFuture.completedFuture(outboundChannelFuture, ()->""); } } else { - return StringTrackableCompletableFuture.failedFuture(outboundChannelFuture.cause(), ()->""); + return TextTrackedFuture.failedFuture(outboundChannelFuture.cause(), ()->""); } }, () -> ""); } @@ -270,7 +268,7 @@ private void deactivateChannel() { @Override - public DiagnosticTrackableCompletableFuture consumeBytes(ByteBuf packetData) { + public TrackedFuture consumeBytes(ByteBuf packetData) { activeChannelFuture = activeChannelFuture.getDeferredFutureThroughHandle((v, channelException) -> { if (channelException == null) { log.atTrace().setMessage(()->"outboundChannelFuture is ready. Writing packets (hash=" + @@ -284,7 +282,7 @@ public DiagnosticTrackableCompletableFuture consumeBytes(ByteBuf pa "outbound channel was not set up successfully, NOT writing bytes hash=" + System.identityHashCode(packetData)).log(); channel.close(); - return DiagnosticTrackableCompletableFuture.Factory.failedFuture(channelException, ()->"exception"); + return TrackedFuture.Factory.failedFuture(channelException, ()->"exception"); } }, ()->"consumeBytes - after channel is fully initialized (potentially waiting on TLS handshake)"); log.atTrace().setMessage(()->"Setting up write of packetData["+packetData+"] hash=" + @@ -296,14 +294,14 @@ private IReplayContexts.IReplayerHttpTransactionContext httpContext() { return getParentContext().getLogicalEnclosingScope(); } - private DiagnosticTrackableCompletableFuture + private TrackedFuture writePacketAndUpdateFuture(ByteBuf packetData) { - return NettyToCompletableFutureBinders.bindNettyFutureToTrackableFuture(channel.writeAndFlush(packetData), + return NettyFutureBinders.bindNettyFutureToTrackableFuture(channel.writeAndFlush(packetData), "CompletableFuture that will wait for the netty future to fill in the completion value"); } @Override - public DiagnosticTrackableCompletableFuture + public TrackedFuture finalizeRequest() { var ff = activeChannelFuture.getDeferredFutureThroughHandle((v,t)-> { log.atDebug().setMessage(()->"finalization running since all prior work has completed for " + @@ -314,7 +312,7 @@ private IReplayContexts.IReplayerHttpTransactionContext httpContext() { } var future = new CompletableFuture(); - var rval = new DiagnosticTrackableCompletableFuture(future, + var rval = new TrackedFuture(future, ()->"NettyPacketToHttpConsumer.finalizeRequest()"); if (t == null) { var responseWatchHandler = diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/TransformedPacketReceiver.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/TransformedPacketReceiver.java index b6718b2ff..63b96499e 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/TransformedPacketReceiver.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/TransformedPacketReceiver.java @@ -2,22 +2,22 @@ import io.netty.buffer.ByteBuf; import org.opensearch.migrations.replay.datatypes.TransformedPackets; -import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; +import org.opensearch.migrations.replay.util.TrackedFuture; public class TransformedPacketReceiver implements IPacketFinalizingConsumer { public final TransformedPackets packets = new TransformedPackets(); @Override - public DiagnosticTrackableCompletableFuture consumeBytes(ByteBuf nextRequestPacket) { + public TrackedFuture consumeBytes(ByteBuf nextRequestPacket) { packets.add(nextRequestPacket); - return DiagnosticTrackableCompletableFuture.Factory.completedFuture(null, + return TrackedFuture.Factory.completedFuture(null, ()->"TransformedPacketReceiver.consume..."); } @Override - public DiagnosticTrackableCompletableFuture finalizeRequest() { - return DiagnosticTrackableCompletableFuture.Factory.completedFuture(packets, + public TrackedFuture finalizeRequest() { + return TrackedFuture.Factory.completedFuture(packets, ()->"TransformedPacketReceiver.finalize..."); } } 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 5b767c995..3bdb529b6 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 @@ -9,8 +9,8 @@ import org.opensearch.migrations.replay.datatypes.HttpRequestTransformationStatus; import org.opensearch.migrations.replay.datatypes.TransformedOutputAndResult; import org.opensearch.migrations.replay.datahandlers.IPacketFinalizingConsumer; -import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; -import org.opensearch.migrations.replay.util.StringTrackableCompletableFuture; +import org.opensearch.migrations.replay.util.TrackedFuture; +import org.opensearch.migrations.replay.util.TextTrackedFuture; import org.opensearch.migrations.transform.IAuthTransformerFactory; import org.opensearch.migrations.transform.IJsonTransformer; import org.slf4j.event.Level; @@ -89,7 +89,7 @@ private HttpRequestDecoder getHttpRequestDecoderHandler() { } @Override - public DiagnosticTrackableCompletableFuture consumeBytes(ByteBuf nextRequestPacket) { + public TrackedFuture consumeBytes(ByteBuf nextRequestPacket) { chunks.add(nextRequestPacket.duplicate().readerIndex(0).retain()); chunkSizes.get(chunkSizes.size() - 1).add(nextRequestPacket.readableBytes()); if (log.isTraceEnabled()) { @@ -98,12 +98,12 @@ public DiagnosticTrackableCompletableFuture consumeBytes(ByteBuf n log.trace("HttpJsonTransformingConsumer[" + this + "]: writing into embedded channel: " + new String(copy, StandardCharsets.UTF_8)); } - return StringTrackableCompletableFuture.completedFuture(null, ()->"initialValue") + return TextTrackedFuture.completedFuture(null, ()->"initialValue") .map(cf->cf.thenAccept(x -> channel.writeInbound(nextRequestPacket)), ()->"HttpJsonTransformingConsumer sending bytes to its EmbeddedChannel"); } - public DiagnosticTrackableCompletableFuture> finalizeRequest() { + public TrackedFuture> finalizeRequest() { var offloadingHandler = getOffloadingHandler(); try { channel.checkException(); @@ -145,7 +145,7 @@ public DiagnosticTrackableCompletableFuture"transformedHttpMessageValue"); + return TextTrackedFuture.completedFuture(v, ()->"transformedHttpMessageValue"); } }, ()->"HttpJsonTransformingConsumer.finalizeRequest() is waiting to handle"); } @@ -157,25 +157,26 @@ private static Throwable unwindPossibleCompletionException(Throwable t) { return t; } - private DiagnosticTrackableCompletableFuture> + private TrackedFuture> redriveWithoutTransformation(IPacketFinalizingConsumer packetConsumer, Throwable reason) { - DiagnosticTrackableCompletableFuture consumptionChainedFuture = + var consumptionChainedFuture = chunks.stream().collect( - Utils.foldLeft(DiagnosticTrackableCompletableFuture.Factory. - completedFuture(null, ()->"Initial value"), - (dcf, bb) -> dcf.thenCompose(v -> packetConsumer.consumeBytes(bb), + Utils.foldLeft(TrackedFuture.Factory.completedFuture((Void) null, ()->"Initial value"), + (tf, bb) -> tf.thenCompose(v -> packetConsumer.consumeBytes(bb), ()->"HttpJsonTransformingConsumer.redriveWithoutTransformation collect()"))); - DiagnosticTrackableCompletableFuture finalizedFuture = + var finalizedFuture = consumptionChainedFuture.thenCompose(v -> packetConsumer.finalizeRequest(), ()->"HttpJsonTransformingConsumer.redriveWithoutTransformation.compose()"); - return finalizedFuture.map(f->f.thenApply(r -> reason == null ? - new TransformedOutputAndResult<>(r, HttpRequestTransformationStatus.SKIPPED, null) : - new TransformedOutputAndResult<>(r, HttpRequestTransformationStatus.ERROR, reason) - ) - .whenComplete((v,t)->{ + return finalizedFuture.thenApply(r -> new TransformedOutputAndResult<>(r, makeStatus(reason), reason), + () -> "redrive final packaging") + .whenComplete((v,t)->{ transformationContext.onTransformSkip(); transformationContext.close(); - }), - ()->"HttpJsonTransformingConsumer.redriveWithoutTransformation().map()"); + }, + ()->"HttpJsonTransformingConsumer.redriveWithoutTransformation().map()"); + } + + private static HttpRequestTransformationStatus makeStatus(Throwable reason) { + return reason == null ? HttpRequestTransformationStatus.SKIPPED : HttpRequestTransformationStatus.ERROR; } } \ No newline at end of file diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettySendByteBufsToPacketHandlerHandler.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettySendByteBufsToPacketHandlerHandler.java index 0d7fd432e..8121eea94 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettySendByteBufsToPacketHandlerHandler.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datahandlers/http/NettySendByteBufsToPacketHandlerHandler.java @@ -9,8 +9,8 @@ import org.opensearch.migrations.replay.datatypes.TransformedOutputAndResult; import org.opensearch.migrations.replay.datahandlers.IPacketFinalizingConsumer; import org.opensearch.migrations.replay.tracing.IReplayContexts; -import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; -import org.opensearch.migrations.replay.util.StringTrackableCompletableFuture; +import org.opensearch.migrations.replay.util.TrackedFuture; +import org.opensearch.migrations.replay.util.TextTrackedFuture; import java.util.Optional; import java.util.concurrent.atomic.AtomicReference; @@ -26,8 +26,8 @@ public class NettySendByteBufsToPacketHandlerHandler extends ChannelInboundHandlerAdapter { final IPacketFinalizingConsumer packetReceiver; // final Boolean value indicates if the handler received a LastHttpContent or EndOfInput message - DiagnosticTrackableCompletableFuture currentFuture; - private AtomicReference>> + TrackedFuture currentFuture; + private AtomicReference>> packetReceiverCompletionFutureRef; IReplayContexts.IReplayerHttpTransactionContext httpTransactionContext; @@ -36,7 +36,7 @@ public NettySendByteBufsToPacketHandlerHandler(IPacketFinalizingConsumer pack this.packetReceiver = packetReceiver; this.packetReceiverCompletionFutureRef = new AtomicReference<>(); this.httpTransactionContext = httpTransactionContext; - currentFuture = DiagnosticTrackableCompletableFuture.Factory.completedFuture(null, + currentFuture = TrackedFuture.Factory.completedFuture(null, ()->"currentFuture for NettySendByteBufsToPacketHandlerHandler initialized to the base case for " + httpTransactionContext); } @@ -47,14 +47,14 @@ public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { if (currentFuture.future.isDone()) { if (currentFuture.future.isCompletedExceptionally()) { packetReceiverCompletionFutureRef.set(currentFuture.getDeferredFutureThroughHandle((v,t)-> - StringTrackableCompletableFuture.failedFuture(t, ()->"fixed failure"), + TextTrackedFuture.failedFuture(t, ()->"fixed failure"), ()->"handlerRemoved: packetReceiverCompletionFuture receiving exceptional value")); return; } else if (currentFuture.get() == null) { log.info("The handler responsible for writing data to the server was detached before writing " + "bytes. Throwing a NoContentException so that the calling context can handle appropriately."); packetReceiverCompletionFutureRef.set( - StringTrackableCompletableFuture.failedFuture(new NoContentException(), + TextTrackedFuture.failedFuture(new NoContentException(), ()->"Setting NoContentException to the exposed CompletableFuture" + " of NettySendByteBufsToPacketHandlerHandler")); return; @@ -76,7 +76,7 @@ public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { }, () -> "handlerRemoved: waiting for the currentFuture to finish"); currentFuture = packetReceiverCompletionFuture.getDeferredFutureThroughHandle((v,t)-> - StringTrackableCompletableFuture.completedFuture(true, + TextTrackedFuture.completedFuture(true, ()->"ignoring return type of packetReceiver.finalizeRequest() but waiting for it to finish"), ()->"Waiting for packetReceiver.finalizeRequest() and will return once that is done"); packetReceiverCompletionFutureRef.set(packetReceiverCompletionFuture); @@ -84,17 +84,17 @@ public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { super.handlerRemoved(ctx); } - private static StringTrackableCompletableFuture> + private static TextTrackedFuture> wrapFinalizedResultWithExceptionHandling(Throwable t1, R v2, Throwable t2, HttpRequestTransformationStatus transformationStatus) { if (t1 != null) { - return StringTrackableCompletableFuture.>failedFuture(t1, + return TextTrackedFuture.>failedFuture(t1, () -> "fixed failure from currentFuture.getDeferredFutureThroughHandle()"); } else if (t2 != null) { - return StringTrackableCompletableFuture.>failedFuture(t2, + return TextTrackedFuture.>failedFuture(t2, () -> "fixed failure from packetReceiver.finalizeRequest()"); } else { - return StringTrackableCompletableFuture.completedFuture(Optional.ofNullable(v2) + return TextTrackedFuture.completedFuture(Optional.ofNullable(v2) .map(r -> new TransformedOutputAndResult(r, transformationStatus, null)) .orElse(null), @@ -103,7 +103,7 @@ public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { } } - public DiagnosticTrackableCompletableFuture> + public TrackedFuture> getPacketReceiverCompletionFuture() { assert packetReceiverCompletionFutureRef.get() != null : "expected close() to have removed the handler and for this to be non-null"; @@ -112,7 +112,7 @@ public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { - currentFuture = DiagnosticTrackableCompletableFuture.Factory.failedFuture(cause, + currentFuture = TrackedFuture.Factory.failedFuture(cause, () -> "NettySendByteBufsToPacketHandlerHandler got an exception"); super.exceptionCaught(ctx, cause); } @@ -133,7 +133,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { if (t != null) { log.atInfo().setCause(t).setMessage(() -> "got exception from a previous future that " + "will prohibit sending any more data to the packetReceiver").log(); - return StringTrackableCompletableFuture.failedFuture(t, () -> "failed previous future"); + return TextTrackedFuture.failedFuture(t, () -> "failed previous future"); } else { log.atTrace().setMessage(() -> "chaining consumingBytes with " + msg + " lastFuture=" + preexistingFutureForCapture).log(); diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/ConnectionReplaySession.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/ConnectionReplaySession.java index 4fe33d8ab..a813a1f8c 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/ConnectionReplaySession.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/ConnectionReplaySession.java @@ -7,9 +7,9 @@ import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import org.opensearch.migrations.replay.tracing.IReplayContexts; -import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; +import org.opensearch.migrations.replay.util.TrackedFuture; import org.opensearch.migrations.replay.util.OnlineRadixSorter; -import org.opensearch.migrations.replay.util.StringTrackableCompletableFuture; +import org.opensearch.migrations.replay.util.TextTrackedFuture; import java.io.IOException; import java.util.function.Supplier; @@ -35,7 +35,7 @@ public class ConnectionReplaySession { public final EventLoop eventLoop; public final OnlineRadixSorter scheduleSequencer; @Getter - private Supplier> channelFutureFutureFactory; + private Supplier> channelFutureFutureFactory; private ChannelFuture cachedChannel; // only can be accessed from the eventLoop thread public final TimeToResponseFulfillmentFutureMap schedule; @Getter @@ -43,7 +43,7 @@ public class ConnectionReplaySession { @SneakyThrows public ConnectionReplaySession(EventLoop eventLoop, IReplayContexts.IChannelKeyContext channelKeyContext, - Supplier> + Supplier> channelFutureFutureFactory) { this.eventLoop = eventLoop; @@ -53,10 +53,10 @@ public ConnectionReplaySession(EventLoop eventLoop, IReplayContexts.IChannelKeyC this.channelFutureFutureFactory = channelFutureFutureFactory; } - public DiagnosticTrackableCompletableFuture + public TrackedFuture getFutureThatReturnsChannelFutureInAnyState(boolean requireActiveChannel) { - StringTrackableCompletableFuture eventLoopFuture = - new StringTrackableCompletableFuture<>("procuring a connection"); + TextTrackedFuture eventLoopFuture = + new TextTrackedFuture<>("procuring a connection"); eventLoop.submit(() -> { if (!requireActiveChannel || (cachedChannel != null && cachedChannel.channel().isActive())) { eventLoopFuture.future.complete(cachedChannel); @@ -68,12 +68,12 @@ public ConnectionReplaySession(EventLoop eventLoop, IReplayContexts.IChannelKeyC } private void createNewChannelFuture(boolean requireActiveChannel, - StringTrackableCompletableFuture eventLoopFuture) { + TextTrackedFuture eventLoopFuture) { createNewChannelFuture(requireActiveChannel, MAX_CHANNEL_CREATE_RETRIES, eventLoopFuture); } private void createNewChannelFuture(boolean requireActiveChannel, int retries, - StringTrackableCompletableFuture eventLoopFuture) + TextTrackedFuture eventLoopFuture) { channelFutureFutureFactory.get().future.whenComplete((v,t)-> { if (requireActiveChannel && retries > 0 && (t == null || exceptionIsRetryable(t))) { diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/FutureTransformer.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/FutureTransformer.java index 0f63a5015..39fa6217e 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/FutureTransformer.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/FutureTransformer.java @@ -1,6 +1,7 @@ package org.opensearch.migrations.replay.datatypes; -import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; + +import org.opensearch.migrations.replay.util.TrackedFuture; import java.util.function.Function; @@ -9,5 +10,5 @@ * chained to its logical parent dependency. */ public interface FutureTransformer extends - Function, DiagnosticTrackableCompletableFuture> { + Function, TrackedFuture> { } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/TimeToResponseFulfillmentFutureMap.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/TimeToResponseFulfillmentFutureMap.java index da1fb9bf3..014a83308 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/TimeToResponseFulfillmentFutureMap.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/TimeToResponseFulfillmentFutureMap.java @@ -5,19 +5,19 @@ import java.util.Deque; import java.util.StringJoiner; -import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; -import org.opensearch.migrations.replay.util.StringTrackableCompletableFuture; +import org.opensearch.migrations.replay.util.TrackedFuture; +import org.opensearch.migrations.replay.util.TextTrackedFuture; public class TimeToResponseFulfillmentFutureMap { public static class FutureWorkPoint { public final Instant startTime; - public final DiagnosticTrackableCompletableFuture scheduleFuture; + public final TrackedFuture scheduleFuture; private final ChannelTaskType channelTaskType; public FutureWorkPoint(Instant forTime, ChannelTaskType taskType) { startTime = forTime; - scheduleFuture = new StringTrackableCompletableFuture<>("scheduled start for " + forTime); + scheduleFuture = new TextTrackedFuture<>("scheduled start for " + forTime); channelTaskType = taskType; } } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/ActiveContextMonitor.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/ActiveContextMonitor.java index f45396ad1..e858c3e61 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/ActiveContextMonitor.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/ActiveContextMonitor.java @@ -39,7 +39,7 @@ public class ActiveContextMonitor implements Runnable { private final ActiveContextTrackerByActivityType perActivityContextTracker; private final OrderedWorkerTracker orderedRequestTracker; private final int totalItemsToOutputLimit; - private final Function,String> formatWorkItem; + private final Function,String> formatWorkItem; private final Predicate logLevelIsEnabled; private final AtomicReference> ageToLevelEdgeMapRef; @@ -48,7 +48,7 @@ public ActiveContextMonitor(ActiveContextTracker globalContextTracker, ActiveContextTrackerByActivityType perActivityContextTracker, OrderedWorkerTracker orderedRequestTracker, int totalItemsToOutputLimit, - Function, String> formatWorkItem, + Function, String> formatWorkItem, Logger logger) { this(globalContextTracker, perActivityContextTracker, orderedRequestTracker, totalItemsToOutputLimit, formatWorkItem, (level, supplier)->logger.atLevel(level).setMessage(supplier).log(), @@ -59,7 +59,7 @@ public ActiveContextMonitor(ActiveContextTracker globalContextTracker, ActiveContextTrackerByActivityType perActivityContextTracker, OrderedWorkerTracker orderedRequestTracker, int totalItemsToOutputLimit, - Function, String> formatWorkItem, + Function, String> formatWorkItem, BiConsumer> logger, Predicate logLevelIsEnabled) { this(globalContextTracker, perActivityContextTracker, orderedRequestTracker, totalItemsToOutputLimit, @@ -75,7 +75,7 @@ public ActiveContextMonitor(ActiveContextTracker globalContextTracker, ActiveContextTrackerByActivityType perActivityContextTracker, OrderedWorkerTracker orderedRequestTracker, int totalItemsToOutputLimit, - Function, String> formatWorkItem, + Function, String> formatWorkItem, BiConsumer> logger, Predicate logLevelIsEnabled, Map levelShowsAgeOlderThanMap) { diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFutureStringFormatter.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFutureStringFormatter.java deleted file mode 100644 index f5a070930..000000000 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFutureStringFormatter.java +++ /dev/null @@ -1,39 +0,0 @@ -package org.opensearch.migrations.replay.util; - -import lombok.NonNull; -import lombok.SneakyThrows; - -import java.util.Optional; -import java.util.function.Function; -import java.util.stream.Collectors; - -public class DiagnosticTrackableCompletableFutureStringFormatter { - - private DiagnosticTrackableCompletableFutureStringFormatter() {} - - public static String format(DiagnosticTrackableCompletableFuture f) { - return format(f, x->null); - } - - public static String format(DiagnosticTrackableCompletableFuture f, - @NonNull Function,String> resultFormatter) { - return f.walkParentsAsStream().map(kvp-> stringFormatFutureWithDiagnostics(f, kvp, resultFormatter)) - .collect(Collectors.joining("<-")); - } - - @SneakyThrows - protected static String stringFormatFutureWithDiagnostics( - DiagnosticTrackableCompletableFuture f, - @NonNull DiagnosticTrackableCompletableFuture dcf, - @NonNull Function,String> resultFormatter) { - var diagnosticInfo = dcf.diagnosticSupplier.get(); - var isDone = dcf.isDone(); - return "[" + System.identityHashCode(dcf) + "] " + diagnosticInfo + - (isDone ? - "[" + Optional.ofNullable(resultFormatter.apply(dcf)).orElse("^") + "]" : - Optional.ofNullable(dcf.innerComposedPendingCompletableFutureReference) - .map(r -> (DiagnosticTrackableCompletableFuture) r.get()) - .map(df -> " --[[" + format(df, resultFormatter) + " ]] ") - .orElse("[…]")); - } -} diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java index e96e8a158..42bb63841 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OnlineRadixSorter.java @@ -36,15 +36,16 @@ public class OnlineRadixSorter { @AllArgsConstructor @Getter private static class IndexedWork { - private final DiagnosticTrackableCompletableFuture signalingToStartFuture; - private DiagnosticTrackableCompletableFuture workCompletedFuture; - private final DiagnosticTrackableCompletableFuture signalWorkCompletedFuture; + private final TrackedFuture signalingToStartFuture; + private TrackedFuture workCompletedFuture; + private final TrackedFuture signalWorkCompletedFuture; - public DiagnosticTrackableCompletableFuture + public TrackedFuture addWorkFuture(FutureTransformer processor, int index) { - var rval = processor.apply(signalingToStartFuture).propagateCompletionToDependentFuture(signalWorkCompletedFuture, - (processedCf, dependentCf) -> dependentCf.complete(null), - ()->"Caller-task completion for idx=" + index); + var rval = processor.apply(signalingToStartFuture) + .propagateCompletionToDependentFuture(signalWorkCompletedFuture, (processedCf, dependentCf) -> + dependentCf.complete(null), + ()->"Caller-task completion for idx=" + index); workCompletedFuture = rval; return rval; } @@ -70,7 +71,7 @@ public OnlineRadixSorter(int startingOffset) { * @param processor * @return */ - public DiagnosticTrackableCompletableFuture + public TrackedFuture addFutureForWork(final int index, FutureTransformer processor) { var workItem = items.get(index); if (workItem == null) { @@ -83,14 +84,14 @@ public OnlineRadixSorter(int startingOffset) { ++nextKey) { int finalNextKey = nextKey; var signalFuture = items.isEmpty() ? - new StringTrackableCompletableFuture( + new TextTrackedFuture( CompletableFuture.completedFuture(null), "unlinked signaling future for slot #" + finalNextKey) : items.get(finalNextKey-1).signalWorkCompletedFuture .thenAccept(v-> {}, ()->"Kickoff for slot #" + finalNextKey); workItem = new IndexedWork(signalFuture, null, - new StringTrackableCompletableFuture(()->"Work to finish for slot #" + finalNextKey + + new TextTrackedFuture(()->"Work to finish for slot #" + finalNextKey + " is awaiting [" + getAwaitingText() + "]")); workItem.signalWorkCompletedFuture.whenComplete((v,t)->{ ++currentOffset; diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OrderedWorkerTracker.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OrderedWorkerTracker.java index 304b26b65..b9f7276c6 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OrderedWorkerTracker.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/OrderedWorkerTracker.java @@ -17,7 +17,7 @@ public class OrderedWorkerTracker implements TrafficReplayerTopLevel.IStreama static class TimeKeyAndFuture { @Getter final long nanoTimeKey; - final DiagnosticTrackableCompletableFuture future; + final TrackedFuture future; } ConcurrentHashMap> primaryMap = new ConcurrentHashMap<>(); ConcurrentSkipListSet> orderedSet = @@ -26,7 +26,7 @@ static class TimeKeyAndFuture { @Override public void put(UniqueReplayerRequestKey uniqueReplayerRequestKey, - DiagnosticTrackableCompletableFuture completableFuture) { + TrackedFuture completableFuture) { var timedValue = new TimeKeyAndFuture<>(System.nanoTime(), completableFuture); primaryMap.put(uniqueReplayerRequestKey, timedValue); orderedSet.add(timedValue); @@ -49,7 +49,7 @@ public int size() { return primaryMap.size(); } - public Stream>> + public Stream>> getRemainingItems() { return primaryMap.entrySet().stream().map(kvp->Map.entry(kvp.getKey(), kvp.getValue().future)); } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/StringTrackableCompletableFuture.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/StringTrackableCompletableFuture.java deleted file mode 100644 index 1ac8d8c82..000000000 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/StringTrackableCompletableFuture.java +++ /dev/null @@ -1,51 +0,0 @@ -package org.opensearch.migrations.replay.util; - -import lombok.NonNull; - -import java.util.Arrays; -import java.util.concurrent.CompletableFuture; -import java.util.function.Supplier; -import java.util.stream.Stream; - -public class StringTrackableCompletableFuture - extends DiagnosticTrackableCompletableFuture { - - public StringTrackableCompletableFuture(String diagnosticLabel) { - this(new CompletableFuture<>(), () -> diagnosticLabel); - } - - public StringTrackableCompletableFuture(Supplier diagnosticSupplier) { - this(new CompletableFuture<>(), diagnosticSupplier); - } - - public StringTrackableCompletableFuture(@NonNull CompletableFuture future, Supplier diagnosticSupplier) { - super(future, diagnosticSupplier); - } - - public StringTrackableCompletableFuture(@NonNull CompletableFuture future, String diagnosticLabel) { - super(future, () -> diagnosticLabel); - } - - public static StringTrackableCompletableFuture - failedFuture(Throwable e, Supplier diagnosticSupplier) { - return new StringTrackableCompletableFuture<>(CompletableFuture.failedFuture(e), diagnosticSupplier); - } - - public static StringTrackableCompletableFuture completedFuture(U v, Supplier diagnosticSupplier) { - return new StringTrackableCompletableFuture<>(CompletableFuture.completedFuture(v), diagnosticSupplier); - } - - public static StringTrackableCompletableFuture - allOf(DiagnosticTrackableCompletableFuture[] allRemainingWorkArray, Supplier diagnosticSupplier) { - return allOf(Arrays.stream(allRemainingWorkArray), diagnosticSupplier); - } - - public static StringTrackableCompletableFuture - allOf(Stream> allRemainingWorkStream, Supplier diagnosticSupplier) { - return new StringTrackableCompletableFuture<>( - CompletableFuture.allOf(allRemainingWorkStream - .map(tcf->tcf.future).toArray(CompletableFuture[]::new)), - diagnosticSupplier); - - } -} diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/TextTrackedFuture.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/TextTrackedFuture.java new file mode 100644 index 000000000..1605e67c2 --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/TextTrackedFuture.java @@ -0,0 +1,51 @@ +package org.opensearch.migrations.replay.util; + +import lombok.NonNull; + +import java.util.Arrays; +import java.util.concurrent.CompletableFuture; +import java.util.function.Supplier; +import java.util.stream.Stream; + +public class TextTrackedFuture + extends TrackedFuture { + + public TextTrackedFuture(String diagnosticLabel) { + this(new CompletableFuture<>(), () -> diagnosticLabel); + } + + public TextTrackedFuture(Supplier diagnosticSupplier) { + this(new CompletableFuture<>(), diagnosticSupplier); + } + + public TextTrackedFuture(@NonNull CompletableFuture future, Supplier diagnosticSupplier) { + super(future, diagnosticSupplier); + } + + public TextTrackedFuture(@NonNull CompletableFuture future, String diagnosticLabel) { + super(future, () -> diagnosticLabel); + } + + public static TextTrackedFuture + failedFuture(Throwable e, Supplier diagnosticSupplier) { + return new TextTrackedFuture<>(CompletableFuture.failedFuture(e), diagnosticSupplier); + } + + public static TextTrackedFuture completedFuture(U v, Supplier diagnosticSupplier) { + return new TextTrackedFuture<>(CompletableFuture.completedFuture(v), diagnosticSupplier); + } + + public static TextTrackedFuture + allOf(TrackedFuture[] allRemainingWorkArray, Supplier diagnosticSupplier) { + return allOf(Arrays.stream(allRemainingWorkArray), diagnosticSupplier); + } + + public static TextTrackedFuture + allOf(Stream> allRemainingWorkStream, Supplier diagnosticSupplier) { + return new TextTrackedFuture<>( + CompletableFuture.allOf(allRemainingWorkStream + .map(tcf->tcf.future).toArray(CompletableFuture[]::new)), + diagnosticSupplier); + + } +} diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFuture.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/TrackedFuture.java similarity index 67% rename from TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFuture.java rename to TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/TrackedFuture.java index 56227d8ab..da9c46472 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFuture.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/TrackedFuture.java @@ -30,15 +30,15 @@ * @param The type of value of the underlying (internal) CompletableFuture's result */ @Slf4j -public class DiagnosticTrackableCompletableFuture { +public class TrackedFuture { public final CompletableFuture future; - protected AtomicReference> innerComposedPendingCompletableFutureReference; + protected AtomicReference> innerComposedPendingCompletableFutureReference; @Getter public final Supplier diagnosticSupplier; - protected final AtomicReference> parentDiagnosticFutureRef; + protected final AtomicReference> parentDiagnosticFutureRef; - private DiagnosticTrackableCompletableFuture() { + private TrackedFuture() { throw new IllegalCallerException(); } @@ -48,35 +48,35 @@ private DiagnosticTrackableCompletableFuture() { */ public static class Factory { private Factory() {} - public static DiagnosticTrackableCompletableFuture + public static TrackedFuture failedFuture(@NonNull Throwable e, @NonNull Supplier diagnosticSupplier) { - return new DiagnosticTrackableCompletableFuture<>(CompletableFuture.failedFuture(e), + return new TrackedFuture<>(CompletableFuture.failedFuture(e), diagnosticSupplier, null); } - public static DiagnosticTrackableCompletableFuture + public static TrackedFuture completedFuture(U v, @NonNull Supplier diagnosticSupplier) { - return new DiagnosticTrackableCompletableFuture<>(CompletableFuture.completedFuture(v), diagnosticSupplier, + return new TrackedFuture<>(CompletableFuture.completedFuture(v), diagnosticSupplier, null); } } - private DiagnosticTrackableCompletableFuture( + private TrackedFuture( @NonNull CompletableFuture future, @NonNull Supplier diagnosticSupplier, - DiagnosticTrackableCompletableFuture parentFuture) { + TrackedFuture parentFuture) { this.future = future; this.diagnosticSupplier = diagnosticSupplier; this.parentDiagnosticFutureRef = new AtomicReference<>(); setParentDiagnosticFuture(parentFuture); } - public DiagnosticTrackableCompletableFuture(@NonNull CompletableFuture future, - @NonNull Supplier diagnosticSupplier) { + public TrackedFuture(@NonNull CompletableFuture future, + @NonNull Supplier diagnosticSupplier) { this(future, diagnosticSupplier, null); } - public DiagnosticTrackableCompletableFuture getParentDiagnosticFuture() { + public TrackedFuture getParentDiagnosticFuture() { var p = parentDiagnosticFutureRef.get(); if (future.isDone() && p != null) { p.setParentDiagnosticFuture(null); @@ -84,7 +84,7 @@ public DiagnosticTrackableCompletableFuture(@NonNull CompletableFuture future return p; } - protected void setParentDiagnosticFuture(DiagnosticTrackableCompletableFuture parent) { + protected void setParentDiagnosticFuture(TrackedFuture parent) { if (parent == null) { parentDiagnosticFutureRef.set(null); return; @@ -96,67 +96,66 @@ protected void setParentDiagnosticFuture(DiagnosticTrackableCompletableFuture { - Optional.ofNullable(getParentDiagnosticFuture()) - .ifPresent(p->p.setParentDiagnosticFuture(null)); - }); + future.whenComplete((v, t) -> + Optional.ofNullable(getParentDiagnosticFuture()).ifPresent(p->p.setParentDiagnosticFuture(null)) + ); } /** * @throws IllegalStateException if the dependentFuture has already been passed to this method * before or if it has already been marked as completed or was initialized with a parent. */ - public DiagnosticTrackableCompletableFuture - propagateCompletionToDependentFuture(DiagnosticTrackableCompletableFuture dependentFuture, + public TrackedFuture + propagateCompletionToDependentFuture(TrackedFuture dependentFuture, BiConsumer, CompletableFuture> consume, @NonNull Supplier diagnosticSupplier) { dependentFuture.setParentDiagnosticFuture(this); return this.whenComplete((v,t) -> consume.accept(this.future, dependentFuture.future), diagnosticSupplier); } - public DiagnosticTrackableCompletableFuture getInnerComposedPendingCompletableFuture() { + public TrackedFuture getInnerComposedPendingCompletableFuture() { return Optional.ofNullable(innerComposedPendingCompletableFutureReference) .map(AtomicReference::get).orElse(null); } - public DiagnosticTrackableCompletableFuture + public TrackedFuture map(@NonNull Function, CompletableFuture> fn, @NonNull Supplier diagnosticSupplier) { var newCf = fn.apply(future); - return new DiagnosticTrackableCompletableFuture<>(newCf, diagnosticSupplier, this); + return new TrackedFuture<>(newCf, diagnosticSupplier, this); } - public DiagnosticTrackableCompletableFuture + public TrackedFuture thenAccept(Consumer fn, @NonNull Supplier diagnosticSupplier) { - return this.map(dcf->dcf.thenAccept(fn), diagnosticSupplier); + return this.map(cf->cf.thenAccept(fn), diagnosticSupplier); } - public DiagnosticTrackableCompletableFuture + public TrackedFuture thenApply(Function fn, @NonNull Supplier diagnosticSupplier) { - return this.map(dcf->dcf.thenApply(fn), diagnosticSupplier); + return this.map(cf->cf.thenApply(fn), diagnosticSupplier); } - public DiagnosticTrackableCompletableFuture + public TrackedFuture exceptionally(Function fn, @NonNull Supplier diagnosticSupplier) { return this.map(cf->cf.exceptionally(fn), diagnosticSupplier); } - public DiagnosticTrackableCompletableFuture + public TrackedFuture whenComplete(BiConsumer fn, @NonNull Supplier diagnosticSupplier) { return map(cf->cf.whenComplete(fn::accept), diagnosticSupplier); } - public DiagnosticTrackableCompletableFuture - thenCompose(@NonNull Function> fn, + public TrackedFuture + thenCompose(@NonNull Function> fn, @NonNull Supplier diagnosticSupplier) { - var innerComposedCompletableFutureReference = new AtomicReference>(); + var innerComposedCompletableFutureReference = new AtomicReference>(); var newCf = this.future.thenCompose(v->{ var innerFuture = fn.apply(v); innerComposedCompletableFutureReference.set(innerFuture); return innerFuture.future; }); var wrappedDiagnosticFuture = - new DiagnosticTrackableCompletableFuture<>(newCf, diagnosticSupplier, this); + new TrackedFuture<>(newCf, diagnosticSupplier, this); wrappedDiagnosticFuture.innerComposedPendingCompletableFutureReference = innerComposedCompletableFutureReference; wrappedDiagnosticFuture.future.whenComplete((v2,t2)->innerComposedCompletableFutureReference.set(null)); return wrappedDiagnosticFuture; @@ -175,12 +174,12 @@ public DiagnosticTrackableCompletableFuture getInnerComposedPendingCompleta * @return * @param */ - public DiagnosticTrackableCompletableFuture + public TrackedFuture getDeferredFutureThroughHandle( - @NonNull BiFunction> fn, + @NonNull BiFunction> fn, @NonNull Supplier diagnosticSupplier) { - var innerComposedCompletableFutureReference = new AtomicReference>(); - CompletableFuture> handledFuture = + var innerComposedCompletableFutureReference = new AtomicReference>(); + CompletableFuture> handledFuture = this.future.handle((v, t)->{ var innerFuture = fn.apply(v,t); innerComposedCompletableFutureReference.set(innerFuture); @@ -188,17 +187,17 @@ public DiagnosticTrackableCompletableFuture getInnerComposedPendingCompleta }); var newCf = handledFuture.thenCompose(wcf->wcf.future); var wrappedDiagnosticFuture = - new DiagnosticTrackableCompletableFuture<>(newCf, diagnosticSupplier, this); + new TrackedFuture<>(newCf, diagnosticSupplier, this); wrappedDiagnosticFuture.innerComposedPendingCompletableFutureReference = innerComposedCompletableFutureReference; // TODO: Add a count to how many futures have been completed and are falling away? wrappedDiagnosticFuture.future.whenComplete((v2,t2)->innerComposedCompletableFutureReference.set(null)); return wrappedDiagnosticFuture; } - public DiagnosticTrackableCompletableFuture + public TrackedFuture handle(@NonNull BiFunction fn, @NonNull Supplier diagnosticSupplier) { CompletableFuture newCf = this.future.handle(fn); - return new DiagnosticTrackableCompletableFuture<>(newCf, diagnosticSupplier, this); + return new TrackedFuture<>(newCf, diagnosticSupplier, this); } public T get() throws ExecutionException, InterruptedException { @@ -213,15 +212,15 @@ public T get(@NonNull Duration timeout) throws ExecutionException, InterruptedEx public boolean isDone() { return future.isDone(); } - public Stream> walkParentsAsStream() { - AtomicReference> chainHeadReference = + public Stream> walkParentsAsStream() { + AtomicReference> chainHeadReference = new AtomicReference<>(this); return IntStream.generate(() -> chainHeadReference.get() != null ? 1 : 0) .takeWhile(x -> x == 1) .mapToObj(i -> { - var dcf = chainHeadReference.get(); - chainHeadReference.set(dcf.getParentDiagnosticFuture()); - return dcf; + var trackedFuture = chainHeadReference.get(); + chainHeadReference.set(trackedFuture.getParentDiagnosticFuture()); + return trackedFuture; }); } @@ -230,11 +229,11 @@ public String toString() { return formatAsString(x->null); } - public String formatAsString(@NonNull Function,String> resultFormatter) { - return DiagnosticTrackableCompletableFutureStringFormatter.format(this, resultFormatter); + public String formatAsString(@NonNull Function,String> resultFormatter) { + return TrackedFutureStringFormatter.format(this, resultFormatter); } - public String formatAsJson(@NonNull Function,String> resultFormatter) { - return DiagnosticTrackableCompletableFutureJsonFormatter.format(this, resultFormatter); + public String formatAsJson(@NonNull Function,String> resultFormatter) { + return TrackedFutureJsonFormatter.format(this, resultFormatter); } } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFutureJsonFormatter.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/TrackedFutureJsonFormatter.java similarity index 53% rename from TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFutureJsonFormatter.java rename to TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/TrackedFutureJsonFormatter.java index 52940b06e..e3d4fcc1e 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFutureJsonFormatter.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/TrackedFutureJsonFormatter.java @@ -5,7 +5,6 @@ import lombok.NonNull; import lombok.SneakyThrows; -import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -13,18 +12,18 @@ import java.util.function.Function; import java.util.stream.Collectors; -public class DiagnosticTrackableCompletableFutureJsonFormatter { +public class TrackedFutureJsonFormatter { static ObjectMapper objectMapper = new ObjectMapper(); - private DiagnosticTrackableCompletableFutureJsonFormatter() {} + private TrackedFutureJsonFormatter() {} - public static String format(DiagnosticTrackableCompletableFuture tf) { + public static String format(TrackedFuture tf) { return format(tf, x->null); } - public static String format(DiagnosticTrackableCompletableFuture tf, - @NonNull Function,String> resultFormatter) { + public static String format(TrackedFuture tf, + @NonNull Function,String> resultFormatter) { try { return objectMapper.writeValueAsString(makeJson(tf, resultFormatter)); } catch (Exception e) { @@ -32,26 +31,26 @@ public static String format(DiagnosticTrackableCompletableFuture tf, } } - public static List makeJson(DiagnosticTrackableCompletableFuture tf, - @NonNull Function,String> resultFormatter) { + public static List makeJson(TrackedFuture tf, + @NonNull Function,String> resultFormatter) { return tf.walkParentsAsStream().map(kvp->jsonFormatFutureWithDiagnostics(kvp, resultFormatter)) .collect(Collectors.toList()); } @SneakyThrows protected static Map jsonFormatFutureWithDiagnostics( - @NonNull DiagnosticTrackableCompletableFuture dcf, - @NonNull Function,String> resultFormatter) { - var diagnosticInfo = dcf.diagnosticSupplier.get(); - var isDone = dcf.isDone(); + @NonNull TrackedFuture tf, + @NonNull Function,String> resultFormatter) { + var diagnosticInfo = tf.diagnosticSupplier.get(); + var isDone = tf.isDone(); var map = new LinkedHashMap(); - map.put("idHash", System.identityHashCode(dcf)); + map.put("idHash", System.identityHashCode(tf)); map.put("label", diagnosticInfo); if (isDone) { - map.put("value", Optional.ofNullable(resultFormatter.apply(dcf)).orElse("^")); + map.put("value", Optional.ofNullable(resultFormatter.apply(tf)).orElse("^")); } else { - var innerResult = Optional.ofNullable(dcf.innerComposedPendingCompletableFutureReference) - .map(r -> (DiagnosticTrackableCompletableFuture) r.get()) + var innerResult = Optional.ofNullable(tf.innerComposedPendingCompletableFutureReference) + .map(r -> (TrackedFuture) r.get()) .map(df -> makeJson(df, resultFormatter)) .orElse(null); if (innerResult == null) { diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/TrackedFutureStringFormatter.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/TrackedFutureStringFormatter.java new file mode 100644 index 000000000..0090dc166 --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/util/TrackedFutureStringFormatter.java @@ -0,0 +1,39 @@ +package org.opensearch.migrations.replay.util; + +import lombok.NonNull; +import lombok.SneakyThrows; + +import java.util.Optional; +import java.util.function.Function; +import java.util.stream.Collectors; + +public class TrackedFutureStringFormatter { + + private TrackedFutureStringFormatter() {} + + public static String format(TrackedFuture f) { + return format(f, x->null); + } + + public static String format(TrackedFuture f, + @NonNull Function,String> resultFormatter) { + return f.walkParentsAsStream().map(kvp-> stringFormatFutureWithDiagnostics(f, kvp, resultFormatter)) + .collect(Collectors.joining("<-")); + } + + @SneakyThrows + protected static String stringFormatFutureWithDiagnostics( + TrackedFuture f, + @NonNull TrackedFuture tf, + @NonNull Function,String> resultFormatter) { + var diagnosticInfo = tf.diagnosticSupplier.get(); + var isDone = tf.isDone(); + return "[" + System.identityHashCode(tf) + "] " + diagnosticInfo + + (isDone ? + "[" + Optional.ofNullable(resultFormatter.apply(tf)).orElse("^") + "]" : + Optional.ofNullable(tf.innerComposedPendingCompletableFutureReference) + .map(r -> (TrackedFuture) r.get()) + .map(df -> " --[[" + format(df, resultFormatter) + " ]] ") + .orElse("[…]")); + } +} diff --git a/TrafficCapture/trafficReplayer/src/main/resources/log4j2.properties b/TrafficCapture/trafficReplayer/src/main/resources/log4j2.properties index 1fb20bd93..d7f78cd11 100644 --- a/TrafficCapture/trafficReplayer/src/main/resources/log4j2.properties +++ b/TrafficCapture/trafficReplayer/src/main/resources/log4j2.properties @@ -83,7 +83,8 @@ logger.OutputTupleJsonLogger.appenderRef.OUTPUT_TUPLES.ref = OUTPUT_TUPLES logger.TransactionSummaryLogger.name = TransactionSummaryLogger logger.TransactionSummaryLogger.level = info -logger.TransactionSummaryLogger.appenderRef.TRANSACTION_SUMMARIES.ref = TransactionSummariesConsole +logger.TransactionSummaryLogger.additivity = false +#logger.TransactionSummaryLogger.appenderRef.TRANSACTION_SUMMARIES.ref = TransactionSummariesConsole logger.TransactionSummaryLogger.appenderRef.TRANSACTION_SUMMARIES_LOGFILE.ref = TransactionSummariesFile logger.AllActiveWorkMonitor.name = AllActiveWorkMonitor 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 3dfb612f1..861a40558 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 @@ -5,7 +5,7 @@ import org.junit.jupiter.api.Test; import org.opensearch.migrations.replay.datahandlers.http.HttpJsonTransformingConsumer; import org.opensearch.migrations.replay.datatypes.HttpRequestTransformationStatus; -import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; +import org.opensearch.migrations.replay.util.TrackedFuture; import org.opensearch.migrations.testutils.WrapWithNettyLeakDetection; import org.opensearch.migrations.tracing.InstrumentationTest; import org.opensearch.migrations.transform.StaticAuthTransformerFactory; @@ -53,7 +53,7 @@ private void runRandomPayloadWithTransformer(HttpJsonTransformingConsumer allConsumesFuture = + TrackedFuture allConsumesFuture = TestUtils.chainedDualWriteHeaderAndPayloadParts(transformingHandler, stringParts, referenceStringBuilder, @@ -119,7 +119,7 @@ public void testMalformedPayload_andTypeMappingUri_IsPassedThrough() throws Exce var stringParts = IntStream.range(0, 1).mapToObj(i-> TestUtils.makeRandomString(r, 10)).map(o->(String)o) .collect(Collectors.toList()); - DiagnosticTrackableCompletableFuture allConsumesFuture = + TrackedFuture allConsumesFuture = TestUtils.chainedDualWriteHeaderAndPayloadParts(transformingHandler, stringParts, referenceStringBuilder, 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 52d12b796..598585854 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 @@ -19,27 +19,21 @@ import org.junit.jupiter.api.parallel.ExecutionMode; import org.opensearch.migrations.replay.datahandlers.IPacketFinalizingConsumer; import org.opensearch.migrations.replay.datahandlers.NettyPacketToHttpConsumer; -import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; +import org.opensearch.migrations.replay.util.TrackedFuture; import org.opensearch.migrations.replay.util.NettyUtils; import org.opensearch.migrations.replay.util.RefSafeHolder; -import org.opensearch.migrations.replay.util.StringTrackableCompletableFuture; +import org.opensearch.migrations.replay.util.TextTrackedFuture; import org.opensearch.migrations.testutils.SimpleHttpServer; import org.opensearch.migrations.testutils.WrapWithNettyLeakDetection; import org.opensearch.migrations.tracing.InstrumentationTest; import java.net.URI; -import java.nio.charset.StandardCharsets; -import java.time.Duration; -import java.time.Instant; -import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; -import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Semaphore; import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.Collectors; import java.util.stream.IntStream; @Slf4j @@ -58,9 +52,9 @@ public static class BlockingPacketConsumer implements IPacketFinalizingConsumer< } @Override - public DiagnosticTrackableCompletableFuture consumeBytes(ByteBuf nextRequestPacket) { + public TrackedFuture consumeBytes(ByteBuf nextRequestPacket) { var index = calls.getAndIncrement(); - return new StringTrackableCompletableFuture<>(CompletableFuture.supplyAsync(()->{ + return new TextTrackedFuture<>(CompletableFuture.supplyAsync(()->{ try { lastCheckIsReady.release(); log.atDebug().setMessage(()->"trying to acquire semaphore for packet #"+index+" and id="+id).log(); @@ -74,9 +68,9 @@ public DiagnosticTrackableCompletableFuture consumeBytes(ByteBuf n } @Override - public DiagnosticTrackableCompletableFuture finalizeRequest() { + public TrackedFuture finalizeRequest() { var index = calls.getAndIncrement(); - return new StringTrackableCompletableFuture<>(CompletableFuture.supplyAsync(()->{ + return new TextTrackedFuture<>(CompletableFuture.supplyAsync(()->{ try { lastCheckIsReady.release(); log.atDebug().setMessage(()->"trying to acquire semaphore for finalize and id="+id).log(); @@ -103,7 +97,7 @@ public void testFutureGraphBuildout() throws Exception { connectionToConsumerMap.get(c.getSourceRequestIndex())); var baseTime = Instant.EPOCH; Instant lastEndTime = baseTime; - var scheduledRequests = new ArrayList>(); + var scheduledRequests = new ArrayList>(); for (int i = 0; i < NUM_REQUESTS_TO_SCHEDULE; ++i) { connectionToConsumerMap.put((long)i, new BlockingPacketConsumer(i)); var requestContext = rootContext.getTestConnectionRequestContext(i); @@ -151,15 +145,15 @@ public void testFutureGraphBuildout() throws Exception { closeFuture.get(); } - private String getParentsDiagnosticString(DiagnosticTrackableCompletableFuture cf, String indent) { + private String getParentsDiagnosticString(TrackedFuture cf, String indent) { return cf.walkParentsAsStream() - .map(dcf->Optional.ofNullable(dcf.getInnerComposedPendingCompletableFuture()) + .map(tf->Optional.ofNullable(tf.getInnerComposedPendingCompletableFuture()) .map(idf->indent + "<\n" + getParentsDiagnosticString(idf, indent+" ") + "\n" + indent + ">\n") .orElse("") + - indent + dcf.diagnosticSupplier.get() + - " [" + System.identityHashCode(dcf) + "]" + - ": " + dcf.isDone()) + indent + tf.diagnosticSupplier.get() + + " [" + System.identityHashCode(tf) + "]" + + ": " + tf.isDone()) .collect(Collectors.joining(";\n")); } @@ -180,7 +174,7 @@ public void testThatSchedulingWorks() throws Exception { new RequestSenderOrchestrator(clientConnectionPool, NettyPacketToHttpConsumer::new); var baseTime = Instant.now(); Instant lastEndTime = baseTime; - var scheduledItems = new ArrayList>(); + var scheduledItems = new ArrayList>(); for (int i = 0; i < NUM_REQUESTS_TO_SCHEDULE; ++i) { var requestContext = rootContext.getTestConnectionRequestContext(i); // half the time schedule at the same time as the last one, the other half, 10ms later than the previous 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 5a44c3054..b0c749f19 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 @@ -15,7 +15,6 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.ValueSource; -import org.opensearch.migrations.NettyToCompletableFutureBinders; import org.opensearch.migrations.replay.AggregatedRawResponse; import org.opensearch.migrations.replay.ClientConnectionPool; import org.opensearch.migrations.replay.PacketToTransformingHttpHandlerFactory; diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/ActiveContextMonitorTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/ActiveContextMonitorTest.java index 4ea541f42..27d7c54c8 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/ActiveContextMonitorTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/ActiveContextMonitorTest.java @@ -74,7 +74,7 @@ void testThatCommonAncestorsAreShownJustEnough() throws Exception { addContexts(compositeTracker, rc); final var idx = i; orderedWorkerTracker.put(rc.getReplayerRequestKey(), - new DiagnosticTrackableCompletableFuture<>(new CompletableFuture<>(), () -> "dummy #" + idx)); + new TrackedFuture<>(new CompletableFuture<>(), () -> "dummy #" + idx)); } Thread.sleep(10); acm.run(); @@ -108,7 +108,7 @@ void testThatNewerItemsArentInspected() throws Exception { addContexts(compositeTracker, rc); final var idx = i; orderedWorkerTracker.put(rc.getReplayerRequestKey(), - new DiagnosticTrackableCompletableFuture<>(new CompletableFuture<>(), () -> "dummy #" + idx)); + new TrackedFuture<>(new CompletableFuture<>(), () -> "dummy #" + idx)); } var startTime = System.nanoTime(); acm.logTopOpenActivities(false); @@ -126,7 +126,7 @@ void testThatNewerItemsArentInspected() throws Exception { var rc = testContext.getTestConnectionRequestContext(i+TRANCHE_SIZE); addContexts(compositeTracker, rc); orderedWorkerTracker.put(rc.getReplayerRequestKey(), - new DiagnosticTrackableCompletableFuture<>(new CompletableFuture<>(), () -> "dummy obj")); + new TrackedFuture<>(new CompletableFuture<>(), () -> "dummy obj")); } acm.logTopOpenActivities(false); @@ -170,7 +170,7 @@ void test() throws Exception { try (var testContext = TestContext.noOtelTracking()) { var requestContext1 = testContext.getTestConnectionRequestContext(0); orderedWorkerTracker.put(requestContext1.getReplayerRequestKey(), - new DiagnosticTrackableCompletableFuture<>(new CompletableFuture<>(), ()->"dummy 1")); + new TrackedFuture<>(new CompletableFuture<>(), ()->"dummy 1")); addContexts(compositeTracker, requestContext1); Thread.sleep(20); @@ -179,7 +179,7 @@ void test() throws Exception { var requestContext2 = testContext.getTestConnectionRequestContext(0); orderedWorkerTracker.put(requestContext2.getReplayerRequestKey(), - new DiagnosticTrackableCompletableFuture<>(new CompletableFuture<>(), ()->"dummy 2")); + new TrackedFuture<>(new CompletableFuture<>(), ()->"dummy 2")); addContexts(compositeTracker, requestContext2); Thread.sleep(20); @@ -188,7 +188,7 @@ void test() throws Exception { var requestContext3 = testContext.getTestConnectionRequestContext(0); orderedWorkerTracker.put(requestContext3.getReplayerRequestKey(), - new DiagnosticTrackableCompletableFuture<>(new CompletableFuture<>(), ()->"dummy 3")); + new TrackedFuture<>(new CompletableFuture<>(), ()->"dummy 3")); addContexts(compositeTracker, requestContext3); Thread.sleep(20); diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/OnlineRadixSorterTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/OnlineRadixSorterTest.java index 8eea70092..5f416fb85 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/OnlineRadixSorterTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/OnlineRadixSorterTest.java @@ -18,38 +18,38 @@ private static String stringify(Stream stream) { } private static String add(OnlineRadixSorterForIntegratedKeys sorter, - Map> m, + Map> m, ArrayList receivedItems, int v) { - var dcf = sorter.add(v, () -> receivedItems.add(v)); - if (m != null) { m.put(v, dcf); } - log.atInfo().setMessage(()->"after adding work... "+dcf).log(); + var future = sorter.add(v, () -> receivedItems.add(v)); + if (m != null) { m.put(v, future); } + log.atInfo().setMessage(()->"after adding work... "+future).log(); return stringify(receivedItems.stream()); } @Test void testOnlineRadixSorter_inOrder() { var radixSorter = new OnlineRadixSorterForIntegratedKeys(1, i -> (int) i); - Assertions.assertEquals("1", add(radixSorter, null, new ArrayList(), 1)); - Assertions.assertEquals("2", add(radixSorter, null, new ArrayList(), 2)); - Assertions.assertEquals("3", add(radixSorter, null, new ArrayList(), 3)); + Assertions.assertEquals("1", add(radixSorter, null, new ArrayList<>(), 1)); + Assertions.assertEquals("2", add(radixSorter, null, new ArrayList<>(), 2)); + Assertions.assertEquals("3", add(radixSorter, null, new ArrayList<>(), 3)); } @Test void testOnlineRadixSorter_outOfOrder() { var radixSorter = new OnlineRadixSorterForIntegratedKeys(1, i->(int) i); var receiverList = new ArrayList(); - var dcfMap = new HashMap>(); - Assertions.assertEquals("", add(radixSorter, dcfMap, receiverList, 3)); - Assertions.assertEquals("", add(radixSorter, dcfMap, receiverList, 4)); - Assertions.assertEquals("1", add(radixSorter, dcfMap, receiverList, 1)); - log.atInfo().setMessage(()->"after adding work for '1'... dcf[3]=" + dcfMap.get(3)).log(); - log.atInfo().setMessage(()->"after adding work for '1'... dcf[4]=" + dcfMap.get(4)).log(); + var futureMap = new HashMap>(); + Assertions.assertEquals("", add(radixSorter, futureMap, receiverList, 3)); + Assertions.assertEquals("", add(radixSorter, futureMap, receiverList, 4)); + Assertions.assertEquals("1", add(radixSorter, futureMap, receiverList, 1)); + log.atInfo().setMessage(()->"after adding work for '1'... future[3]=" + futureMap.get(3)).log(); + log.atInfo().setMessage(()->"after adding work for '1'... future[4]=" + futureMap.get(4)).log(); receiverList.clear(); - Assertions.assertEquals("2,3,4", add(radixSorter, dcfMap, receiverList, 2)); + Assertions.assertEquals("2,3,4", add(radixSorter, futureMap, receiverList, 2)); receiverList.clear(); - Assertions.assertEquals("5", add(radixSorter, dcfMap, receiverList, 5)); + Assertions.assertEquals("5", add(radixSorter, futureMap, receiverList, 5)); receiverList.clear(); - Assertions.assertEquals("", add(radixSorter, dcfMap, receiverList, 7)); + Assertions.assertEquals("", add(radixSorter, futureMap, receiverList, 7)); receiverList.clear(); } diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/StringTrackableCompletableFutureTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/TextTrackedFutureTest.java similarity index 64% rename from TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/StringTrackableCompletableFutureTest.java rename to TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/TextTrackedFutureTest.java index 414d0b7cc..d5c0935a7 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/StringTrackableCompletableFutureTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/TextTrackedFutureTest.java @@ -10,7 +10,7 @@ import java.util.concurrent.TimeUnit; @WrapWithNettyLeakDetection(disableLeakChecks = true) -class StringTrackableCompletableFutureTest { +class TextTrackedFutureTest { @SneakyThrows private static void sneakyWait(CompletableFuture o) { o.get(5, TimeUnit.MINUTES); @@ -26,14 +26,14 @@ public void futureWithThreeStages() throws Exception { CompletableFuture notifier2 = new CompletableFuture(); CompletableFuture notifier3 = new CompletableFuture(); - var stcf1 = new StringTrackableCompletableFuture<>(CompletableFuture.supplyAsync(()->{ + var stcf1 = new TextTrackedFuture<>(CompletableFuture.supplyAsync(()->{ sneakyWait(notifier1); return 1; }), ()->"A"); final var id1 = System.identityHashCode(stcf1); final var id1Bktd = "[" + id1 + "] "; - Assertions.assertEquals(id1Bktd + "A[…]", DiagnosticTrackableCompletableFutureStringFormatter.format(stcf1)); + Assertions.assertEquals(id1Bktd + "A[…]", TrackedFutureStringFormatter.format(stcf1)); var stcf2 = stcf1.map(f->f.thenApplyAsync(x->{ sneakyWait(notifier2); @@ -42,7 +42,7 @@ public void futureWithThreeStages() throws Exception { ()->"B"); final var id2 = System.identityHashCode(stcf2); final var id2Bktd = "[" + id2 + "] "; - Assertions.assertEquals(id2Bktd + "B[…]<-" + id1Bktd + "A[…]", DiagnosticTrackableCompletableFutureStringFormatter.format(stcf2)); + Assertions.assertEquals(id2Bktd + "B[…]<-" + id1Bktd + "A[…]", TrackedFutureStringFormatter.format(stcf2)); var stcf3 = stcf2.map(f->f.thenApplyAsync(x->{ sneakyWait(notifier3); @@ -52,75 +52,75 @@ public void futureWithThreeStages() throws Exception { final var id3 = System.identityHashCode(stcf3); final var id3Bktd = "[" + id3 + "] "; - Assertions.assertEquals(id1Bktd + "A[…]", DiagnosticTrackableCompletableFutureStringFormatter.format(stcf1)); - Assertions.assertEquals(id2Bktd + "B[…]<-" + id1Bktd + "A[…]", DiagnosticTrackableCompletableFutureStringFormatter.format(stcf2)); - Assertions.assertEquals(id3Bktd + "C[…]<-" + id2Bktd + "B[…]<-" + id1Bktd + "A[…]", DiagnosticTrackableCompletableFutureStringFormatter.format(stcf3)); + Assertions.assertEquals(id1Bktd + "A[…]", TrackedFutureStringFormatter.format(stcf1)); + Assertions.assertEquals(id2Bktd + "B[…]<-" + id1Bktd + "A[…]", TrackedFutureStringFormatter.format(stcf2)); + Assertions.assertEquals(id3Bktd + "C[…]<-" + id2Bktd + "B[…]<-" + id1Bktd + "A[…]", TrackedFutureStringFormatter.format(stcf3)); Assertions.assertEquals("[{\"idHash\":" + id1 + ",\"label\":\"A\",\"value\":\"…\"}]", - DiagnosticTrackableCompletableFutureJsonFormatter.format(stcf1)); + TrackedFutureJsonFormatter.format(stcf1)); Assertions.assertEquals("[" + "{\"idHash\":" + id2 + ",\"label\":\"B\",\"value\":\"…\"}," + - "{\"idHash\":" + id1 + ",\"label\":\"A\",\"value\":\"…\"}]", DiagnosticTrackableCompletableFutureJsonFormatter.format(stcf2)); + "{\"idHash\":" + id1 + ",\"label\":\"A\",\"value\":\"…\"}]", TrackedFutureJsonFormatter.format(stcf2)); Assertions.assertEquals("[" + "{\"idHash\":" + id3 + ",\"label\":\"C\",\"value\":\"…\"}," + "{\"idHash\":" + id2 + ",\"label\":\"B\",\"value\":\"…\"}," + - "{\"idHash\":" + id1 + ",\"label\":\"A\",\"value\":\"…\"}]", DiagnosticTrackableCompletableFutureJsonFormatter.format(stcf3)); + "{\"idHash\":" + id1 + ",\"label\":\"A\",\"value\":\"…\"}]", TrackedFutureJsonFormatter.format(stcf3)); notifyAndWaitForGet(stcf1, notifier1); - Assertions.assertEquals(id1Bktd + "A[^]", DiagnosticTrackableCompletableFutureStringFormatter.format(stcf1)); - Assertions.assertEquals(id2Bktd + "B[…]<-" + id1Bktd + "A[^]", DiagnosticTrackableCompletableFutureStringFormatter.format(stcf2)); - Assertions.assertEquals(id3Bktd + "C[…]<-" + id2Bktd + "B[…]<-" + id1Bktd + "A[^]", DiagnosticTrackableCompletableFutureStringFormatter.format(stcf3)); + Assertions.assertEquals(id1Bktd + "A[^]", TrackedFutureStringFormatter.format(stcf1)); + Assertions.assertEquals(id2Bktd + "B[…]<-" + id1Bktd + "A[^]", TrackedFutureStringFormatter.format(stcf2)); + Assertions.assertEquals(id3Bktd + "C[…]<-" + id2Bktd + "B[…]<-" + id1Bktd + "A[^]", TrackedFutureStringFormatter.format(stcf3)); Assertions.assertEquals(id3Bktd + "C[…]<-" + id2Bktd + "B[…]<-" + id1Bktd + "A[1]", - stcf3.formatAsString(StringTrackableCompletableFutureTest::formatCompletableFuture)); + stcf3.formatAsString(TextTrackedFutureTest::formatCompletableFuture)); Assertions.assertEquals("[{\"idHash\":" + id1 + ",\"label\":\"A\",\"value\":\"^\"}]", - DiagnosticTrackableCompletableFutureJsonFormatter.format(stcf1)); + TrackedFutureJsonFormatter.format(stcf1)); Assertions.assertEquals("[" + "{\"idHash\":" + id2 + ",\"label\":\"B\",\"value\":\"…\"}," + - "{\"idHash\":" + id1 + ",\"label\":\"A\",\"value\":\"^\"}]", DiagnosticTrackableCompletableFutureJsonFormatter.format(stcf2)); + "{\"idHash\":" + id1 + ",\"label\":\"A\",\"value\":\"^\"}]", TrackedFutureJsonFormatter.format(stcf2)); Assertions.assertEquals("[" + "{\"idHash\":" + id3 + ",\"label\":\"C\",\"value\":\"…\"}," + "{\"idHash\":" + id2 + ",\"label\":\"B\",\"value\":\"…\"}," + - "{\"idHash\":" + id1 + ",\"label\":\"A\",\"value\":\"^\"}]", DiagnosticTrackableCompletableFutureJsonFormatter.format(stcf3)); + "{\"idHash\":" + id1 + ",\"label\":\"A\",\"value\":\"^\"}]", TrackedFutureJsonFormatter.format(stcf3)); notifyAndWaitForGet(stcf2, notifier2); - Assertions.assertEquals(id2Bktd + "B[^]<-" + id1Bktd + "A[^]", DiagnosticTrackableCompletableFutureStringFormatter.format(stcf2)); - Assertions.assertEquals(id1Bktd + "A[^]", DiagnosticTrackableCompletableFutureStringFormatter.format(stcf1)); - Assertions.assertEquals(id3Bktd + "C[…]<-" + id2Bktd + "B[^]<-" + id1Bktd + "A[^]", DiagnosticTrackableCompletableFutureStringFormatter.format(stcf3)); + Assertions.assertEquals(id2Bktd + "B[^]<-" + id1Bktd + "A[^]", TrackedFutureStringFormatter.format(stcf2)); + Assertions.assertEquals(id1Bktd + "A[^]", TrackedFutureStringFormatter.format(stcf1)); + Assertions.assertEquals(id3Bktd + "C[…]<-" + id2Bktd + "B[^]<-" + id1Bktd + "A[^]", TrackedFutureStringFormatter.format(stcf3)); Assertions.assertEquals(id3Bktd + "C[…]<-" + id2Bktd + "B[11]<-" + id1Bktd + "A[1]", - stcf3.formatAsString(StringTrackableCompletableFutureTest::formatCompletableFuture)); + stcf3.formatAsString(TextTrackedFutureTest::formatCompletableFuture)); Assertions.assertEquals("[{\"idHash\":" + id1 + ",\"label\":\"A\",\"value\":\"^\"}]", - DiagnosticTrackableCompletableFutureJsonFormatter.format(stcf1)); + TrackedFutureJsonFormatter.format(stcf1)); Assertions.assertEquals("[" + "{\"idHash\":" + id2 + ",\"label\":\"B\",\"value\":\"^\"}," + - "{\"idHash\":" + id1 + ",\"label\":\"A\",\"value\":\"^\"}]", DiagnosticTrackableCompletableFutureJsonFormatter.format(stcf2)); + "{\"idHash\":" + id1 + ",\"label\":\"A\",\"value\":\"^\"}]", TrackedFutureJsonFormatter.format(stcf2)); Assertions.assertEquals("[" + "{\"idHash\":" + id3 + ",\"label\":\"C\",\"value\":\"…\"}," + "{\"idHash\":" + id2 + ",\"label\":\"B\",\"value\":\"^\"}," + - "{\"idHash\":" + id1 + ",\"label\":\"A\",\"value\":\"^\"}]", DiagnosticTrackableCompletableFutureJsonFormatter.format(stcf3)); + "{\"idHash\":" + id1 + ",\"label\":\"A\",\"value\":\"^\"}]", TrackedFutureJsonFormatter.format(stcf3)); Assertions.assertEquals("[" + "{\"idHash\":" + id3 + ",\"label\":\"C\",\"value\":\"…\"}," + "{\"idHash\":" + id2 + ",\"label\":\"B\",\"value\":\"11\"}," + "{\"idHash\":" + id1 + ",\"label\":\"A\",\"value\":\"1\"}]", - stcf3.formatAsJson(StringTrackableCompletableFutureTest::formatCompletableFuture)); + stcf3.formatAsJson(TextTrackedFutureTest::formatCompletableFuture)); // A is clipped because of grandparent culling notifyAndWaitForGet(stcf3, notifier3); - Assertions.assertEquals(id3Bktd + "C[^]<-" + id2Bktd +"B[^]", DiagnosticTrackableCompletableFutureStringFormatter.format(stcf3)); - Assertions.assertEquals(id1Bktd + "A[^]", DiagnosticTrackableCompletableFutureStringFormatter.format(stcf1)); - Assertions.assertEquals(id2Bktd + "B[^]", DiagnosticTrackableCompletableFutureStringFormatter.format(stcf2)); + Assertions.assertEquals(id3Bktd + "C[^]<-" + id2Bktd +"B[^]", TrackedFutureStringFormatter.format(stcf3)); + Assertions.assertEquals(id1Bktd + "A[^]", TrackedFutureStringFormatter.format(stcf1)); + Assertions.assertEquals(id2Bktd + "B[^]", TrackedFutureStringFormatter.format(stcf2)); Assertions.assertEquals("[" + "{\"idHash\":" + id3 + ",\"label\":\"C\",\"value\":\"111\"}," + "{\"idHash\":" + id2 + ",\"label\":\"B\",\"value\":\"11\"}]", - stcf3.formatAsJson(StringTrackableCompletableFutureTest::formatCompletableFuture)); + stcf3.formatAsJson(TextTrackedFutureTest::formatCompletableFuture)); } - public static String formatCompletableFuture(DiagnosticTrackableCompletableFuture cf) { + public static String formatCompletableFuture(TrackedFuture cf) { try { return "" + cf.get(); } catch (InterruptedException e) { @@ -131,7 +131,7 @@ public static String formatCompletableFuture(DiagnosticTrackableCompletableFutur } } - private void notifyAndWaitForGet(DiagnosticTrackableCompletableFuture stcf, + private void notifyAndWaitForGet(TrackedFuture stcf, CompletableFuture lockObject) throws Exception { notify(lockObject); stcf.get(); diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFutureTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/TrackedFutureTest.java similarity index 52% rename from TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFutureTest.java rename to TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/TrackedFutureTest.java index a1a33bd89..2efd4e4fc 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/DiagnosticTrackableCompletableFutureTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/util/TrackedFutureTest.java @@ -5,27 +5,26 @@ import org.junit.jupiter.api.Test; import java.util.concurrent.Semaphore; -import java.util.concurrent.ThreadPoolExecutor; @Slf4j -class DiagnosticTrackableCompletableFutureTest { +class TrackedFutureTest { @Test public void test() throws Exception { final int ITERATIONS = 5; - DiagnosticTrackableCompletableFuture base = - new StringTrackableCompletableFuture<>("initial future"); - var dcf = base; - var dcfSemaphore = new Semaphore(0); + TrackedFuture base = + new TextTrackedFuture<>("initial future"); + var tf = base; + var tfSemaphore = new Semaphore(0); var observerSemaphore = new Semaphore(0); for (int i = 0; i< ITERATIONS; ++i) { int finalI = i; - var lastDcf = dcf; - dcf = dcf.thenApply(v->{ + var lastTf = tf; + tf = tf.thenApply(v->{ try { observerSemaphore.release(); - dcfSemaphore.acquire(); - log.atInfo().setMessage(()->"dcf[" + finalI+"]"+lastDcf).log(); + tfSemaphore.acquire(); + log.atInfo().setMessage(()->"tf[" + finalI+"]"+lastTf).log(); } catch (InterruptedException e) { throw Lombok.sneakyThrow(e); } @@ -33,14 +32,14 @@ public void test() throws Exception { }, ()->"run for "+ finalI); } base.future.completeAsync(()->""); - DiagnosticTrackableCompletableFuture finalDcf = dcf; + TrackedFuture finalTf = tf; for (int i=0; i"top dcf after " + finalI + " releases="+ finalDcf).log(); + log.atInfo().setMessage(()->"top tf after " + finalI + " releases="+ finalTf).log(); } - log.atInfo().setMessage(()->"final dcf after any ancestor culls=" + finalDcf).log(); + log.atInfo().setMessage(()->"final tf after any ancestor culls=" + finalTf).log(); } } \ No newline at end of file diff --git a/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/TestCapturePacketToHttpHandler.java b/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/TestCapturePacketToHttpHandler.java index c7b6c81c5..e8fe4edfd 100644 --- a/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/TestCapturePacketToHttpHandler.java +++ b/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/TestCapturePacketToHttpHandler.java @@ -7,8 +7,8 @@ import lombok.extern.slf4j.Slf4j; import org.junit.jupiter.api.Assertions; import org.opensearch.migrations.replay.datahandlers.IPacketFinalizingConsumer; -import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; -import org.opensearch.migrations.replay.util.StringTrackableCompletableFuture; +import org.opensearch.migrations.replay.util.TrackedFuture; +import org.opensearch.migrations.replay.util.TextTrackedFuture; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -39,11 +39,11 @@ public TestCapturePacketToHttpHandler(Duration consumeDuration, } @Override - public DiagnosticTrackableCompletableFuture consumeBytes(ByteBuf nextRequestPacket) { + public TrackedFuture consumeBytes(ByteBuf nextRequestPacket) { numConsumes.incrementAndGet(); log.info("incoming buffer refcnt="+nextRequestPacket.refCnt()); var duplicatedPacket = nextRequestPacket.duplicate().retain(); - return new DiagnosticTrackableCompletableFuture<>(CompletableFuture.runAsync(() -> { + return new TrackedFuture<>(CompletableFuture.runAsync(() -> { try { log.info("Running async future for " + nextRequestPacket); Thread.sleep(consumeDuration.toMillis()); @@ -64,11 +64,11 @@ public DiagnosticTrackableCompletableFuture consumeBytes(ByteBuf n } @Override - public DiagnosticTrackableCompletableFuture finalizeRequest() { + public TrackedFuture finalizeRequest() { numFinalizations.incrementAndGet(); Assertions.assertEquals(1, numFinalizations.get()); bytesCaptured = byteArrayOutputStream.toByteArray(); - return StringTrackableCompletableFuture.completedFuture(dummyAggregatedResponse, + return TextTrackedFuture.completedFuture(dummyAggregatedResponse, ()->"TestCapturePacketToHttpHandler.dummy"); } diff --git a/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/TestUtils.java b/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/TestUtils.java index d883e7d94..80e11e32f 100644 --- a/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/TestUtils.java +++ b/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/TestUtils.java @@ -14,7 +14,7 @@ import org.opensearch.migrations.Utils; import org.opensearch.migrations.replay.datahandlers.IPacketConsumer; import org.opensearch.migrations.replay.datahandlers.http.HttpJsonTransformingConsumer; -import org.opensearch.migrations.replay.util.DiagnosticTrackableCompletableFuture; +import org.opensearch.migrations.replay.util.TrackedFuture; import org.opensearch.migrations.tracing.TestContext; import org.opensearch.migrations.transform.IAuthTransformerFactory; import org.opensearch.migrations.transform.IJsonTransformer; @@ -58,25 +58,25 @@ static String makeRandomString(Random r, int maxStringSize) { .toString(); } - static DiagnosticTrackableCompletableFuture writeStringToBoth(String s, StringBuilder referenceStringBuilder, - IPacketConsumer transformingHandler) { + static TrackedFuture writeStringToBoth(String s, StringBuilder referenceStringBuilder, + IPacketConsumer transformingHandler) { log.info("Sending string to transformer: "+s); referenceStringBuilder.append(s); var bytes = s.getBytes(StandardCharsets.UTF_8); return transformingHandler.consumeBytes(bytes); } - static DiagnosticTrackableCompletableFuture chainedWriteHeadersAndDualWritePayloadParts(IPacketConsumer packetConsumer, - List stringParts, - StringBuilder referenceStringAccumulator, - String headers) { + static TrackedFuture chainedWriteHeadersAndDualWritePayloadParts(IPacketConsumer packetConsumer, + List stringParts, + StringBuilder referenceStringAccumulator, + String headers) { return stringParts.stream().collect( Utils.foldLeft(packetConsumer.consumeBytes(headers.getBytes(StandardCharsets.UTF_8)), (cf, s) -> cf.thenCompose(v -> writeStringToBoth(s, referenceStringAccumulator, packetConsumer), ()->"TestUtils.chainedWriteHeadersAndDualWritePayloadParts"))); } - public static DiagnosticTrackableCompletableFuture + public static TrackedFuture chainedDualWriteHeaderAndPayloadParts(IPacketConsumer packetConsumer, List stringParts, StringBuilder referenceStringAccumulator, diff --git a/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/util/OnlineRadixSorterForIntegratedKeys.java b/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/util/OnlineRadixSorterForIntegratedKeys.java index 4c04388fb..50c6fed1d 100644 --- a/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/util/OnlineRadixSorterForIntegratedKeys.java +++ b/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/util/OnlineRadixSorterForIntegratedKeys.java @@ -21,7 +21,7 @@ public OnlineRadixSorterForIntegratedKeys(int startingOffset, ToIntFunction r this.radixResolver = radixResolver; } - public DiagnosticTrackableCompletableFuture add(T item, Runnable sortedItemVisitor) { + public TrackedFuture add(T item, Runnable sortedItemVisitor) { return super.addFutureForWork(radixResolver.applyAsInt(item), signalFuture->signalFuture.map( f->f.whenComplete((v,t)->sortedItemVisitor.run()), ()->"OnlineRadixSorterForIntegratedKeys.addFutureForWork")); From 748cccd0b4113ef72badcedf04154e2af8a52944 Mon Sep 17 00:00:00 2001 From: Greg Schohn Date: Tue, 30 Apr 2024 18:54:03 -0400 Subject: [PATCH 11/11] Add a bit of logging instead of an NPE when a channel was already gone and we go to close it Signed-off-by: Greg Schohn --- .../opensearch/migrations/replay/ClientConnectionPool.java | 7 +++++++ 1 file changed, 7 insertions(+) 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 336009e98..7c6cd2912 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 @@ -18,6 +18,7 @@ import org.opensearch.migrations.replay.datahandlers.NettyPacketToHttpConsumer; import org.opensearch.migrations.replay.datatypes.ConnectionReplaySession; import org.opensearch.migrations.replay.tracing.IReplayContexts; +import org.opensearch.migrations.replay.util.TextTrackedFuture; import org.opensearch.migrations.replay.util.TrackedFuture; import java.net.URI; @@ -124,6 +125,12 @@ public void closeConnection(IReplayContexts.IChannelKeyContext ctx, int sessionN closeClientConnectionChannel(ConnectionReplaySession channelAndFutureWork) { return channelAndFutureWork.getFutureThatReturnsChannelFutureInAnyState(false) .thenCompose(channelFuture-> { + if (channelFuture == null) { + log.atTrace().setMessage(() -> "Asked to close channel for " + + channelAndFutureWork.getChannelKeyContext() + " but the channel wasn't found. " + + "It may have already been reset.").log(); + return TextTrackedFuture.completedFuture(null, ()->""); + } log.atTrace().setMessage(() -> "closing channel " + channelFuture.channel() + "(" + channelAndFutureWork.getChannelKeyContext() + ")...").log(); return NettyFutureBinders.bindNettyFutureToTrackableFuture(