From 68ffc5a542eb9ff29d1d05e16c1d9c202cc35649 Mon Sep 17 00:00:00 2001 From: Keran Yang Date: Wed, 13 Mar 2024 09:48:33 -0400 Subject: [PATCH] feat: implement sdk for session reduce (#94) Signed-off-by: Keran Yang --- examples/pom.xml | 17 + .../reducesession/counter/CountFactory.java | 23 + .../reducesession/counter/CountFunction.java | 48 + .../reducestreamer/sum/SumFunction.java | 5 +- pom.xml | 2 + .../numaflow/reducestreamer/OutputActor.java | 2 +- .../OutputStreamObserverImpl.java | 4 +- .../reducestreamer/ReduceStreamerActor.java | 4 +- .../reducestreamer/SupervisorActor.java | 16 +- .../numaflow/sessionreducer/ActorRequest.java | 61 + .../sessionreducer/ActorRequestType.java | 17 + .../sessionreducer/ActorResponse.java | 52 + .../numaflow/sessionreducer/Constants.java | 15 + .../numaflow/sessionreducer/GRPCConfig.java | 25 + .../sessionreducer/GetAccumulatorRequest.java | 18 + .../GetAccumulatorResponse.java | 19 + .../numaflow/sessionreducer/HandlerDatum.java | 28 + .../MergeAccumulatorRequest.java | 16 + .../sessionreducer/MergeDoneResponse.java | 7 + .../numaflow/sessionreducer/OutputActor.java | 54 + .../OutputStreamObserverImpl.java | 51 + .../numaflow/sessionreducer/Server.java | 112 + .../numaflow/sessionreducer/Service.java | 142 + .../sessionreducer/SessionReducerActor.java | 128 + .../sessionreducer/ShutdownActor.java | 65 + .../sessionreducer/SupervisorActor.java | 403 +++ .../sessionreducer/UniqueIdGenerator.java | 29 + .../numaflow/sessionreducer/model/Datum.java | 29 + .../sessionreducer/model/Message.java | 56 + .../model/OutputStreamObserver.java | 13 + .../sessionreducer/model/SessionReducer.java | 55 + .../model/SessionReducerFactory.java | 13 + .../sessionreduce/v1/sessionreduce.proto | 83 + .../sessionreducer/GRPCConfigTest.java | 38 + .../ReduceOutputStreamObserver.java | 70 + .../sessionreducer/ServerErrTest.java | 252 ++ .../numaflow/sessionreducer/ServerTest.java | 2405 +++++++++++++++++ .../sessionreducer/ShutdownActorTest.java | 206 ++ 38 files changed, 4568 insertions(+), 15 deletions(-) create mode 100644 examples/src/main/java/io/numaproj/numaflow/examples/reducesession/counter/CountFactory.java create mode 100644 examples/src/main/java/io/numaproj/numaflow/examples/reducesession/counter/CountFunction.java create mode 100644 src/main/java/io/numaproj/numaflow/sessionreducer/ActorRequest.java create mode 100644 src/main/java/io/numaproj/numaflow/sessionreducer/ActorRequestType.java create mode 100644 src/main/java/io/numaproj/numaflow/sessionreducer/ActorResponse.java create mode 100644 src/main/java/io/numaproj/numaflow/sessionreducer/Constants.java create mode 100644 src/main/java/io/numaproj/numaflow/sessionreducer/GRPCConfig.java create mode 100644 src/main/java/io/numaproj/numaflow/sessionreducer/GetAccumulatorRequest.java create mode 100644 src/main/java/io/numaproj/numaflow/sessionreducer/GetAccumulatorResponse.java create mode 100644 src/main/java/io/numaproj/numaflow/sessionreducer/HandlerDatum.java create mode 100644 src/main/java/io/numaproj/numaflow/sessionreducer/MergeAccumulatorRequest.java create mode 100644 src/main/java/io/numaproj/numaflow/sessionreducer/MergeDoneResponse.java create mode 100644 src/main/java/io/numaproj/numaflow/sessionreducer/OutputActor.java create mode 100644 src/main/java/io/numaproj/numaflow/sessionreducer/OutputStreamObserverImpl.java create mode 100644 src/main/java/io/numaproj/numaflow/sessionreducer/Server.java create mode 100644 src/main/java/io/numaproj/numaflow/sessionreducer/Service.java create mode 100644 src/main/java/io/numaproj/numaflow/sessionreducer/SessionReducerActor.java create mode 100644 src/main/java/io/numaproj/numaflow/sessionreducer/ShutdownActor.java create mode 100644 src/main/java/io/numaproj/numaflow/sessionreducer/SupervisorActor.java create mode 100644 src/main/java/io/numaproj/numaflow/sessionreducer/UniqueIdGenerator.java create mode 100644 src/main/java/io/numaproj/numaflow/sessionreducer/model/Datum.java create mode 100644 src/main/java/io/numaproj/numaflow/sessionreducer/model/Message.java create mode 100644 src/main/java/io/numaproj/numaflow/sessionreducer/model/OutputStreamObserver.java create mode 100644 src/main/java/io/numaproj/numaflow/sessionreducer/model/SessionReducer.java create mode 100644 src/main/java/io/numaproj/numaflow/sessionreducer/model/SessionReducerFactory.java create mode 100644 src/main/proto/sessionreduce/v1/sessionreduce.proto create mode 100644 src/test/java/io/numaproj/numaflow/sessionreducer/GRPCConfigTest.java create mode 100644 src/test/java/io/numaproj/numaflow/sessionreducer/ReduceOutputStreamObserver.java create mode 100644 src/test/java/io/numaproj/numaflow/sessionreducer/ServerErrTest.java create mode 100644 src/test/java/io/numaproj/numaflow/sessionreducer/ServerTest.java create mode 100644 src/test/java/io/numaproj/numaflow/sessionreducer/ShutdownActorTest.java diff --git a/examples/pom.xml b/examples/pom.xml index 63df86f2..d9ed9fd5 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -151,6 +151,23 @@ + + session-reduce-count + package + + dockerBuild + + + + + io.numaproj.numaflow.examples.reducesession.counter.CountFactory + + + + numaflow-java-examples/session-reduce-count + + + diff --git a/examples/src/main/java/io/numaproj/numaflow/examples/reducesession/counter/CountFactory.java b/examples/src/main/java/io/numaproj/numaflow/examples/reducesession/counter/CountFactory.java new file mode 100644 index 00000000..21c3822f --- /dev/null +++ b/examples/src/main/java/io/numaproj/numaflow/examples/reducesession/counter/CountFactory.java @@ -0,0 +1,23 @@ +package io.numaproj.numaflow.examples.reducesession.counter; + +import io.numaproj.numaflow.sessionreducer.Server; +import io.numaproj.numaflow.sessionreducer.model.SessionReducerFactory; +import lombok.extern.slf4j.Slf4j; + +/** + * CountFactory extends SessionReducerFactory to support creating instances of SumFunction. + * It also provides a main function to start a server for handling the session reduce stream. + */ +@Slf4j +public class CountFactory extends SessionReducerFactory { + + public static void main(String[] args) throws Exception { + log.info("count udf was invoked"); + new Server(new CountFactory()).start(); + } + + @Override + public CountFunction createSessionReducer() { + return new CountFunction(); + } +} diff --git a/examples/src/main/java/io/numaproj/numaflow/examples/reducesession/counter/CountFunction.java b/examples/src/main/java/io/numaproj/numaflow/examples/reducesession/counter/CountFunction.java new file mode 100644 index 00000000..7f938047 --- /dev/null +++ b/examples/src/main/java/io/numaproj/numaflow/examples/reducesession/counter/CountFunction.java @@ -0,0 +1,48 @@ +package io.numaproj.numaflow.examples.reducesession.counter; + +import io.numaproj.numaflow.sessionreducer.model.Datum; +import io.numaproj.numaflow.sessionreducer.model.Message; +import io.numaproj.numaflow.sessionreducer.model.SessionReducer; +import lombok.extern.slf4j.Slf4j; + +import java.util.concurrent.atomic.AtomicInteger; + +/** + * CountFunction is a simple session reducer which counts the number of events in a session. + */ +@Slf4j +public class CountFunction extends SessionReducer { + + private final AtomicInteger count = new AtomicInteger(0); + + @Override + public void processMessage( + String[] keys, + Datum datum, + io.numaproj.numaflow.sessionreducer.model.OutputStreamObserver outputStreamObserver) { + this.count.incrementAndGet(); + } + + @Override + public void handleEndOfStream( + String[] keys, + io.numaproj.numaflow.sessionreducer.model.OutputStreamObserver outputStreamObserver) { + outputStreamObserver.send(new Message(String.valueOf(this.count.get()).getBytes())); + } + + @Override + public byte[] accumulator() { + return String.valueOf(this.count.get()).getBytes(); + } + + @Override + public void mergeAccumulator(byte[] accumulator) { + int value = 0; + try { + value = Integer.parseInt(new String(accumulator)); + } catch (NumberFormatException e) { + log.info("error while parsing integer - {}", e.getMessage()); + } + this.count.addAndGet(value); + } +} diff --git a/examples/src/main/java/io/numaproj/numaflow/examples/reducestreamer/sum/SumFunction.java b/examples/src/main/java/io/numaproj/numaflow/examples/reducestreamer/sum/SumFunction.java index 77bfcba7..60f4972b 100644 --- a/examples/src/main/java/io/numaproj/numaflow/examples/reducestreamer/sum/SumFunction.java +++ b/examples/src/main/java/io/numaproj/numaflow/examples/reducestreamer/sum/SumFunction.java @@ -1,5 +1,6 @@ package io.numaproj.numaflow.examples.reducestreamer.sum; +import io.numaproj.numaflow.reducestreamer.model.Datum; import io.numaproj.numaflow.reducestreamer.model.Message; import io.numaproj.numaflow.reducestreamer.model.Metadata; import io.numaproj.numaflow.reducestreamer.model.OutputStreamObserver; @@ -19,9 +20,9 @@ public class SumFunction extends ReduceStreamer { @Override public void processMessage( String[] keys, - io.numaproj.numaflow.reducestreamer.model.Datum datum, + Datum datum, OutputStreamObserver outputStreamObserver, - io.numaproj.numaflow.reducestreamer.model.Metadata md) { + Metadata md) { try { sum += Integer.parseInt(new String(datum.getValue())); } catch (NumberFormatException e) { diff --git a/pom.xml b/pom.xml index 7ce801d6..8a220f11 100644 --- a/pom.xml +++ b/pom.xml @@ -296,6 +296,7 @@ io.numaproj.numaflow.mapstream.v1 io.numaproj.numaflow.map.v1 io.numaproj.numaflow.reduce.v1 + io.numaproj.numaflow.sessionreduce.v1 io.numaproj.numaflow.sourcetransformer.v1 io.numaproj.numaflow.sink.v1 io.numaproj.numaflow.sideinput.v1 @@ -337,6 +338,7 @@ io/numaproj/numaflow/examples/* io/numaproj/numaflow/mapstream/v1/* io/numaproj/numaflow/reduce/v1/* + io/numaproj/numaflow/sessionreduce/v1/* io/numaproj/numaflow/map/v1/* io/numaproj/numaflow/sourcetransformer/v1/* io/numaproj/numaflow/sink/v1/* diff --git a/src/main/java/io/numaproj/numaflow/reducestreamer/OutputActor.java b/src/main/java/io/numaproj/numaflow/reducestreamer/OutputActor.java index afa4e724..43e84ace 100644 --- a/src/main/java/io/numaproj/numaflow/reducestreamer/OutputActor.java +++ b/src/main/java/io/numaproj/numaflow/reducestreamer/OutputActor.java @@ -10,7 +10,7 @@ /** * Output actor is a wrapper around the gRPC output stream. * It ensures synchronized calls to the responseObserver onNext() and invokes onComplete at the end of the stream. - * ALL reduce responses are sent to the response stream actor before getting forwarded to the output gRPC stream. + * ALL reduce responses are sent to the output actor before getting forwarded to the output gRPC stream. *

* More details about gRPC StreamObserver concurrency: https://grpc.github.io/grpc-java/javadoc/io/grpc/stub/StreamObserver.html */ diff --git a/src/main/java/io/numaproj/numaflow/reducestreamer/OutputStreamObserverImpl.java b/src/main/java/io/numaproj/numaflow/reducestreamer/OutputStreamObserverImpl.java index 9a37726d..c7d3197f 100644 --- a/src/main/java/io/numaproj/numaflow/reducestreamer/OutputStreamObserverImpl.java +++ b/src/main/java/io/numaproj/numaflow/reducestreamer/OutputStreamObserverImpl.java @@ -16,11 +16,11 @@ @AllArgsConstructor class OutputStreamObserverImpl implements OutputStreamObserver { private final Metadata md; - private final ActorRef responseStreamActor; + private final ActorRef outputActor; @Override public void send(Message message) { - this.responseStreamActor.tell(buildResponse(message), ActorRef.noSender()); + this.outputActor.tell(buildResponse(message), ActorRef.noSender()); } private ActorResponse buildResponse(Message message) { diff --git a/src/main/java/io/numaproj/numaflow/reducestreamer/ReduceStreamerActor.java b/src/main/java/io/numaproj/numaflow/reducestreamer/ReduceStreamerActor.java index 27bc71f8..58be7e71 100644 --- a/src/main/java/io/numaproj/numaflow/reducestreamer/ReduceStreamerActor.java +++ b/src/main/java/io/numaproj/numaflow/reducestreamer/ReduceStreamerActor.java @@ -27,13 +27,13 @@ class ReduceStreamerActor extends AbstractActor { private OutputStreamObserver outputStream; public static Props props( - String[] keys, Metadata md, ReduceStreamer groupBy, ActorRef responseStreamActor) { + String[] keys, Metadata md, ReduceStreamer groupBy, ActorRef outputActor) { return Props.create( ReduceStreamerActor.class, keys, md, groupBy, - new OutputStreamObserverImpl(md, responseStreamActor)); + new OutputStreamObserverImpl(md, outputActor)); } @Override diff --git a/src/main/java/io/numaproj/numaflow/reducestreamer/SupervisorActor.java b/src/main/java/io/numaproj/numaflow/reducestreamer/SupervisorActor.java index a4d782b4..b5d5f4dd 100644 --- a/src/main/java/io/numaproj/numaflow/reducestreamer/SupervisorActor.java +++ b/src/main/java/io/numaproj/numaflow/reducestreamer/SupervisorActor.java @@ -29,31 +29,31 @@ class SupervisorActor extends AbstractActor { private final ReduceStreamerFactory reduceStreamerFactory; private final Metadata md; private final ActorRef shutdownActor; - private final ActorRef responseStreamActor; + private final ActorRef outputActor; private final Map actorsMap = new HashMap<>(); public SupervisorActor( ReduceStreamerFactory reduceStreamerFactory, Metadata md, ActorRef shutdownActor, - ActorRef responseStreamActor) { + ActorRef outputActor) { this.reduceStreamerFactory = reduceStreamerFactory; this.md = md; this.shutdownActor = shutdownActor; - this.responseStreamActor = responseStreamActor; + this.outputActor = outputActor; } public static Props props( ReduceStreamerFactory reduceStreamerFactory, Metadata md, ActorRef shutdownActor, - ActorRef responseStreamActor) { + ActorRef outputActor) { return Props.create( SupervisorActor.class, reduceStreamerFactory, md, shutdownActor, - responseStreamActor); + outputActor); } // if there is an uncaught exception stop in the supervisor actor, send a signal to shut down @@ -101,7 +101,7 @@ private void invokeActor(ActorRequest actorRequest) { keys, this.md, reduceStreamerHandler, - this.responseStreamActor)); + this.outputActor)); actorsMap.put(uniqueId, actorRef); } HandlerDatum handlerDatum = constructHandlerDatum(actorRequest.getRequest().getPayload()); @@ -122,9 +122,9 @@ private void handleActorResponse(ActorResponse actorResponse) { if (actorsMap.isEmpty()) { // since the actors map is empty, this particular actor response is the last response to forward to output gRPC stream. actorResponse.setLast(true); - this.responseStreamActor.tell(actorResponse, getSelf()); + this.outputActor.tell(actorResponse, getSelf()); } else { - this.responseStreamActor.tell(actorResponse, getSelf()); + this.outputActor.tell(actorResponse, getSelf()); } } diff --git a/src/main/java/io/numaproj/numaflow/sessionreducer/ActorRequest.java b/src/main/java/io/numaproj/numaflow/sessionreducer/ActorRequest.java new file mode 100644 index 00000000..99b2e79e --- /dev/null +++ b/src/main/java/io/numaproj/numaflow/sessionreducer/ActorRequest.java @@ -0,0 +1,61 @@ +package io.numaproj.numaflow.sessionreducer; + +import io.numaproj.numaflow.sessionreduce.v1.Sessionreduce; +import lombok.Builder; +import lombok.Getter; + +/** + * ActorRequest is used by the supervisor actor to distribute session reduce operations to + * individual session reducer actors. One actor request is sent to only one session reducer actor. + */ +@Getter +class ActorRequest { + private final ActorRequestType type; + // the window of the target session the actor request is sent to + private final Sessionreduce.KeyedWindow keyedWindow; + // the new keyed window the target session is to be expanded to + // it is specified only when the actor request is an EXPAND + private final Sessionreduce.KeyedWindow newKeyedWindow; + // the payload of the request + private final Sessionreduce.SessionReduceRequest.Payload payload; + // the id of the merge task this request belongs to, it's equal to the unique id of the merged window, + // it is specified only when the actor request is a GET_ACCUMULATOR + private final String mergeTaskId; + + @Builder + private ActorRequest( + ActorRequestType type, + Sessionreduce.KeyedWindow keyedWindow, + Sessionreduce.KeyedWindow newKeyedWindow, + Sessionreduce.SessionReduceRequest.Payload payload, + String mergeTaskId + ) { + this.type = type; + this.keyedWindow = keyedWindow; + this.newKeyedWindow = newKeyedWindow; + this.payload = payload; + this.mergeTaskId = mergeTaskId; + } + + static class ActorRequestBuilder { + ActorRequest build() { + if (newKeyedWindow != null && type != ActorRequestType.EXPAND) { + throw new IllegalStateException( + "attribute newKeyedWindow can only be set when the request is an EXPAND."); + } + if (newKeyedWindow == null && type == ActorRequestType.EXPAND) { + throw new IllegalStateException( + "attribute newKeyedWindow must be set when the request is an EXPAND."); + } + if (mergeTaskId != null && type != ActorRequestType.GET_ACCUMULATOR) { + throw new IllegalStateException( + "attribute mergeTaskId can only be set when the request is a GET_ACCUMULATOR."); + } + if (mergeTaskId == null && type == ActorRequestType.GET_ACCUMULATOR) { + throw new IllegalStateException( + "attribute mergeTaskId must be set when the request is a GET_ACCUMULATOR."); + } + return new ActorRequest(type, keyedWindow, newKeyedWindow, payload, mergeTaskId); + } + } +} diff --git a/src/main/java/io/numaproj/numaflow/sessionreducer/ActorRequestType.java b/src/main/java/io/numaproj/numaflow/sessionreducer/ActorRequestType.java new file mode 100644 index 00000000..e66a1971 --- /dev/null +++ b/src/main/java/io/numaproj/numaflow/sessionreducer/ActorRequestType.java @@ -0,0 +1,17 @@ +package io.numaproj.numaflow.sessionreducer; + +/** + * ActorRequestType represents the purpose of an ActorRequest. + */ +public enum ActorRequestType { + // open a brand-new session window + OPEN, + // append a message to an existing session window + APPEND, + // close a session window + CLOSE, + // expand a session window + EXPAND, + // ask a to-be-merged session window for it's accumulator + GET_ACCUMULATOR, +} diff --git a/src/main/java/io/numaproj/numaflow/sessionreducer/ActorResponse.java b/src/main/java/io/numaproj/numaflow/sessionreducer/ActorResponse.java new file mode 100644 index 00000000..f5192ca6 --- /dev/null +++ b/src/main/java/io/numaproj/numaflow/sessionreducer/ActorResponse.java @@ -0,0 +1,52 @@ +package io.numaproj.numaflow.sessionreducer; + +import io.numaproj.numaflow.sessionreduce.v1.Sessionreduce; +import lombok.Builder; +import lombok.Getter; +import lombok.Setter; + +/** + * The actor response holds the session reduce response from a particular session window. + */ +@Getter +@Setter +class ActorResponse { + Sessionreduce.SessionReduceResponse response; + // The isLast attribute indicates whether the response is globally the last one to be sent to + // the output gRPC stream, if set to true, it means the response is the very last response among + // all windows. When output actor receives an isLast response, it sends the response to and immediately + // closes the output stream. + boolean isLast; + // The accumulator attribute holds the accumulator of the session. + byte[] accumulator; + // The mergeTaskId attribute holds the id of the merged window that this session is to be merged into. + String mergeTaskId; + + @Builder + private ActorResponse( + Sessionreduce.SessionReduceResponse response, + boolean isLast, + byte[] accumulator, + String mergeTaskId + ) { + this.response = response; + this.isLast = isLast; + this.accumulator = accumulator; + this.mergeTaskId = mergeTaskId; + } + + static class ActorResponseBuilder { + ActorResponse build() { + if ((accumulator != null && mergeTaskId == null) || (accumulator == null + && mergeTaskId != null)) { + throw new IllegalStateException( + "attributes accumulator and mergeTaskId should be either both null or both non-null."); + } + return new ActorResponse(response, isLast, accumulator, mergeTaskId); + } + } + + boolean isEOFResponse() { + return this.accumulator == null && this.mergeTaskId == null; + } +} diff --git a/src/main/java/io/numaproj/numaflow/sessionreducer/Constants.java b/src/main/java/io/numaproj/numaflow/sessionreducer/Constants.java new file mode 100644 index 00000000..b4b9c425 --- /dev/null +++ b/src/main/java/io/numaproj/numaflow/sessionreducer/Constants.java @@ -0,0 +1,15 @@ +package io.numaproj.numaflow.sessionreducer; + +class Constants { + public static final int DEFAULT_MESSAGE_SIZE = 1024 * 1024 * 64; + + public static final String DEFAULT_SOCKET_PATH = "/var/run/numaflow/sessionreduce.sock"; + + public static final String DEFAULT_SERVER_INFO_FILE_PATH = "/var/run/numaflow/sessionreducer-server-info"; + + public static final String EOF = "EOF"; + + public static final String SUCCESS = "SUCCESS"; + + public static final String DELIMITER = ":"; +} diff --git a/src/main/java/io/numaproj/numaflow/sessionreducer/GRPCConfig.java b/src/main/java/io/numaproj/numaflow/sessionreducer/GRPCConfig.java new file mode 100644 index 00000000..e990f5e9 --- /dev/null +++ b/src/main/java/io/numaproj/numaflow/sessionreducer/GRPCConfig.java @@ -0,0 +1,25 @@ +package io.numaproj.numaflow.sessionreducer; + +import lombok.Builder; +import lombok.Getter; + +/** + * GRPCConfig is used to provide configurations for gRPC server. + */ +@Getter +@Builder(builderMethodName = "newBuilder") +public class GRPCConfig { + private String socketPath; + private int maxMessageSize; + private String infoFilePath; + + /** + * Static method to create default GRPCConfig. + */ + static GRPCConfig defaultGrpcConfig() { + return GRPCConfig.newBuilder() + .infoFilePath(Constants.DEFAULT_SERVER_INFO_FILE_PATH) + .maxMessageSize(Constants.DEFAULT_MESSAGE_SIZE) + .socketPath(Constants.DEFAULT_SOCKET_PATH).build(); + } +} diff --git a/src/main/java/io/numaproj/numaflow/sessionreducer/GetAccumulatorRequest.java b/src/main/java/io/numaproj/numaflow/sessionreducer/GetAccumulatorRequest.java new file mode 100644 index 00000000..37a064ef --- /dev/null +++ b/src/main/java/io/numaproj/numaflow/sessionreducer/GetAccumulatorRequest.java @@ -0,0 +1,18 @@ +package io.numaproj.numaflow.sessionreducer; + +import lombok.AllArgsConstructor; +import lombok.Getter; + +/** + * GetAccumulatorRequest is sent by supervisor actor to inform a session reducer actor that + * the window is to be merged with other windows. + *

+ * "I am working on a merge task (mergeTaskId), + * and you are one of the windows to be merged. + * Please give me your accumulator." + */ +@AllArgsConstructor +@Getter +class GetAccumulatorRequest { + private final String mergeTaskId; +} diff --git a/src/main/java/io/numaproj/numaflow/sessionreducer/GetAccumulatorResponse.java b/src/main/java/io/numaproj/numaflow/sessionreducer/GetAccumulatorResponse.java new file mode 100644 index 00000000..4d18d857 --- /dev/null +++ b/src/main/java/io/numaproj/numaflow/sessionreducer/GetAccumulatorResponse.java @@ -0,0 +1,19 @@ +package io.numaproj.numaflow.sessionreducer; + +import io.numaproj.numaflow.sessionreduce.v1.Sessionreduce; +import lombok.AllArgsConstructor; +import lombok.Getter; + +/** + * GetAccumulatorResponse is sent from a session reducer actor back to the supervisor actor, + * containing the accumulator of the session. + *

+ * "Hey supervisor, I am the session window fromKeyedWindow, + * I am returning my accumulator so that you can merge me." + */ +@AllArgsConstructor +@Getter +class GetAccumulatorResponse { + private final Sessionreduce.KeyedWindow fromKeyedWindow; + private final byte[] accumulator; +} diff --git a/src/main/java/io/numaproj/numaflow/sessionreducer/HandlerDatum.java b/src/main/java/io/numaproj/numaflow/sessionreducer/HandlerDatum.java new file mode 100644 index 00000000..26080d9a --- /dev/null +++ b/src/main/java/io/numaproj/numaflow/sessionreducer/HandlerDatum.java @@ -0,0 +1,28 @@ +package io.numaproj.numaflow.sessionreducer; + +import io.numaproj.numaflow.sessionreducer.model.Datum; +import lombok.AllArgsConstructor; + +import java.time.Instant; + +@AllArgsConstructor +class HandlerDatum implements Datum { + private byte[] value; + private Instant watermark; + private Instant eventTime; + + @Override + public Instant getWatermark() { + return this.watermark; + } + + @Override + public byte[] getValue() { + return this.value; + } + + @Override + public Instant getEventTime() { + return this.eventTime; + } +} diff --git a/src/main/java/io/numaproj/numaflow/sessionreducer/MergeAccumulatorRequest.java b/src/main/java/io/numaproj/numaflow/sessionreducer/MergeAccumulatorRequest.java new file mode 100644 index 00000000..921dfd7a --- /dev/null +++ b/src/main/java/io/numaproj/numaflow/sessionreducer/MergeAccumulatorRequest.java @@ -0,0 +1,16 @@ +package io.numaproj.numaflow.sessionreducer; + +import lombok.AllArgsConstructor; +import lombok.Getter; + +/** + * MergeAccumulatorRequest is sent from the supervisor actor to the merged session reducer actor to + * ask the actor to merge an accumulator. + *

+ * "Hey, I received an accumulator from one of the sessions that are merging to you, please merge it with yourself." + */ +@AllArgsConstructor +@Getter +class MergeAccumulatorRequest { + private final byte[] accumulator; +} diff --git a/src/main/java/io/numaproj/numaflow/sessionreducer/MergeDoneResponse.java b/src/main/java/io/numaproj/numaflow/sessionreducer/MergeDoneResponse.java new file mode 100644 index 00000000..ef2cbcf1 --- /dev/null +++ b/src/main/java/io/numaproj/numaflow/sessionreducer/MergeDoneResponse.java @@ -0,0 +1,7 @@ +package io.numaproj.numaflow.sessionreducer; + +/** + * MergeDoneResponse indicates the current MERGE request is done. + */ +public class MergeDoneResponse { +} diff --git a/src/main/java/io/numaproj/numaflow/sessionreducer/OutputActor.java b/src/main/java/io/numaproj/numaflow/sessionreducer/OutputActor.java new file mode 100644 index 00000000..f2fb40d8 --- /dev/null +++ b/src/main/java/io/numaproj/numaflow/sessionreducer/OutputActor.java @@ -0,0 +1,54 @@ +package io.numaproj.numaflow.sessionreducer; + +import akka.actor.AbstractActor; +import akka.actor.Props; +import io.grpc.stub.StreamObserver; +import io.numaproj.numaflow.sessionreduce.v1.Sessionreduce; +import lombok.AllArgsConstructor; +import lombok.extern.slf4j.Slf4j; + +/** + * Output actor is a wrapper around the gRPC output stream. + * It ensures synchronized calls to the responseObserver onNext() and invokes onComplete at the end of the stream. + * ALL session reduce responses are sent to the output actor before getting forwarded to the output gRPC stream. + *

+ * More details about gRPC StreamObserver concurrency: https://grpc.github.io/grpc-java/javadoc/io/grpc/stub/StreamObserver.html + */ +@Slf4j +@AllArgsConstructor +class OutputActor extends AbstractActor { + StreamObserver responseObserver; + + public static Props props( + StreamObserver responseObserver) { + return Props.create(OutputActor.class, responseObserver); + } + + @Override + public Receive createReceive() { + return receiveBuilder() + .match(ActorResponse.class, this::handleResponse) + .match(String.class, this::handleEOF) + .build(); + } + + private void handleResponse(ActorResponse actorResponse) { + responseObserver.onNext(actorResponse.getResponse()); + if (actorResponse.isLast()) { + this.closeSystem(); + } + } + + private void handleEOF(String eof) { + this.closeSystem(); + } + + private void closeSystem() { + // close the output stream. + responseObserver.onCompleted(); + // stop the AKKA system right after we close the output stream. + // note: could make more sense if the supervisor actor stops the system, + // but it requires an extra tell. + getContext().getSystem().stop(getSender()); + } +} diff --git a/src/main/java/io/numaproj/numaflow/sessionreducer/OutputStreamObserverImpl.java b/src/main/java/io/numaproj/numaflow/sessionreducer/OutputStreamObserverImpl.java new file mode 100644 index 00000000..b95605db --- /dev/null +++ b/src/main/java/io/numaproj/numaflow/sessionreducer/OutputStreamObserverImpl.java @@ -0,0 +1,51 @@ +package io.numaproj.numaflow.sessionreducer; + +import akka.actor.ActorRef; +import com.google.protobuf.ByteString; +import io.numaproj.numaflow.sessionreduce.v1.Sessionreduce; +import io.numaproj.numaflow.sessionreducer.model.Message; +import io.numaproj.numaflow.sessionreducer.model.OutputStreamObserver; +import lombok.AllArgsConstructor; +import lombok.Setter; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * OutputStreamObserverImpl transforms a message to an ActorResponse. + * The send method sends the ActorResponse to the output actor to forward to output gRPC stream. + */ +@AllArgsConstructor +class OutputStreamObserverImpl implements OutputStreamObserver { + private final ActorRef outputActor; + @Setter + private Sessionreduce.KeyedWindow keyedWindow; + + @Override + public void send(Message message) { + this.outputActor.tell( + buildResponse(message, this.keyedWindow), + ActorRef.noSender()); + } + + private ActorResponse buildResponse(Message message, Sessionreduce.KeyedWindow keyedWindow) { + Sessionreduce.SessionReduceResponse.Builder responseBuilder = Sessionreduce.SessionReduceResponse.newBuilder(); + // set the window + responseBuilder.setKeyedWindow(keyedWindow); + // set EOF to false + responseBuilder.setEOF(false); + // set the result. + responseBuilder.setResult(Sessionreduce.SessionReduceResponse.Result + .newBuilder() + .setValue(ByteString.copyFrom(message.getValue())) + .addAllKeys(message.getKeys() + == null ? new ArrayList<>():Arrays.asList(message.getKeys())) + .addAllTags( + message.getTags() == null ? new ArrayList<>():List.of(message.getTags())) + .build()); + return ActorResponse.builder() + .response(responseBuilder.build()) + .build(); + } +} diff --git a/src/main/java/io/numaproj/numaflow/sessionreducer/Server.java b/src/main/java/io/numaproj/numaflow/sessionreducer/Server.java new file mode 100644 index 00000000..72047178 --- /dev/null +++ b/src/main/java/io/numaproj/numaflow/sessionreducer/Server.java @@ -0,0 +1,112 @@ +package io.numaproj.numaflow.sessionreducer; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import io.grpc.ServerBuilder; +import io.numaproj.numaflow.info.ServerInfoAccessor; +import io.numaproj.numaflow.info.ServerInfoAccessorImpl; +import io.numaproj.numaflow.sessionreducer.model.SessionReducer; +import io.numaproj.numaflow.sessionreducer.model.SessionReducerFactory; +import io.numaproj.numaflow.shared.GrpcServerUtils; +import lombok.extern.slf4j.Slf4j; + +import java.util.concurrent.TimeUnit; + +/** + * Server is the gRPC server for executing session reduce operations. + */ +@Slf4j +public class Server { + private final GRPCConfig grpcConfig; + private final Service service; + private final ServerInfoAccessor serverInfoAccessor = new ServerInfoAccessorImpl(new ObjectMapper()); + private io.grpc.Server server; + + /** + * constructor to create gRPC server. + * + * @param sessionReducerFactory to process the message + */ + public Server(SessionReducerFactory sessionReducerFactory) { + this(sessionReducerFactory, GRPCConfig.defaultGrpcConfig()); + } + + /** + * constructor to create gRPC server with gRPC config. + * + * @param grpcConfig to configure the max message size for grpc + * @param sessionReducerFactory to process the message + */ + public Server( + SessionReducerFactory sessionReducerFactory, + GRPCConfig grpcConfig) { + this.service = new Service(sessionReducerFactory); + this.grpcConfig = grpcConfig; + } + + /** + * Start serving requests. + * + * @throws Exception if server fails to start + */ + public void start() throws Exception { + GrpcServerUtils.writeServerInfo( + serverInfoAccessor, + grpcConfig.getSocketPath(), + grpcConfig.getInfoFilePath()); + + if (this.server == null) { + // create server builder + ServerBuilder serverBuilder = GrpcServerUtils.createServerBuilder( + grpcConfig.getSocketPath(), grpcConfig.getMaxMessageSize()); + + // build server + this.server = serverBuilder + .addService(this.service) + .build(); + } + + // start server + server.start(); + + log.info( + "Server started, listening on socket path: " + + grpcConfig.getSocketPath()); + + // register shutdown hook + Runtime.getRuntime().addShutdownHook(new Thread(() -> { + // Use stderr here since the logger may have been reset by its JVM shutdown hook. + System.err.println("*** shutting down gRPC server since JVM is shutting down"); + try { + Server.this.stop(); + } catch (InterruptedException e) { + Thread.interrupted(); + e.printStackTrace(System.err); + } + })); + } + + /** + * Stop serving requests and shutdown resources. Await termination on the main thread since the + * grpc library uses daemon threads. + * + * @throws InterruptedException if shutdown is interrupted + */ + public void stop() throws InterruptedException { + if (server != null) { + server.shutdown().awaitTermination(30, TimeUnit.SECONDS); + } + } + + /** + * Set server builder for testing. + * + * @param serverBuilder for building the server + */ + @VisibleForTesting + void setServerBuilder(ServerBuilder serverBuilder) { + this.server = serverBuilder + .addService(this.service) + .build(); + } +} diff --git a/src/main/java/io/numaproj/numaflow/sessionreducer/Service.java b/src/main/java/io/numaproj/numaflow/sessionreducer/Service.java new file mode 100644 index 00000000..b747d600 --- /dev/null +++ b/src/main/java/io/numaproj/numaflow/sessionreducer/Service.java @@ -0,0 +1,142 @@ +package io.numaproj.numaflow.sessionreducer; + +import akka.actor.ActorRef; +import akka.actor.ActorSystem; +import akka.actor.AllDeadLetters; +import akka.pattern.Patterns; +import akka.util.Timeout; +import com.google.protobuf.Empty; +import io.grpc.Status; +import io.grpc.stub.StreamObserver; +import io.numaproj.numaflow.sessionreduce.v1.SessionReduceGrpc; +import io.numaproj.numaflow.sessionreduce.v1.Sessionreduce; +import io.numaproj.numaflow.sessionreducer.model.SessionReducer; +import io.numaproj.numaflow.sessionreducer.model.SessionReducerFactory; +import lombok.extern.slf4j.Slf4j; +import scala.concurrent.Await; +import scala.concurrent.Future; +import scala.concurrent.duration.Duration; + +import java.util.concurrent.CompletableFuture; + +import static io.numaproj.numaflow.reduce.v1.ReduceGrpc.getReduceFnMethod; + +@Slf4j +class Service extends SessionReduceGrpc.SessionReduceImplBase { + public static final ActorSystem sessionReduceActorSystem = ActorSystem.create("sessionreduce"); + + private final SessionReducerFactory sessionReducerFactory; + + public Service(SessionReducerFactory sessionReducerFactory) { + this.sessionReducerFactory = sessionReducerFactory; + } + + static void handleFailure( + CompletableFuture failureFuture, + StreamObserver responseObserver) { + new Thread(() -> { + try { + failureFuture.get(); + } catch (Exception e) { + e.printStackTrace(); + var status = Status.UNKNOWN.withDescription(e.getMessage()).withCause(e); + responseObserver.onError(status.asException()); + } + }).start(); + } + + /** + * Streams input data to the session reducer functions and returns the result. + */ + @Override + public StreamObserver sessionReduceFn(final StreamObserver responseObserver) { + if (this.sessionReducerFactory == null) { + return io.grpc.stub.ServerCalls.asyncUnimplementedStreamingCall( + getReduceFnMethod(), + responseObserver); + } + + CompletableFuture failureFuture = new CompletableFuture<>(); + + // create a shutdown actor that listens to exceptions. + ActorRef shutdownActorRef = sessionReduceActorSystem. + actorOf(ShutdownActor.props(failureFuture)); + + // subscribe for dead letters + sessionReduceActorSystem.getEventStream().subscribe(shutdownActorRef, AllDeadLetters.class); + + handleFailure(failureFuture, responseObserver); + + // create an output actor that ensures synchronized delivery of reduce responses. + ActorRef outputActor = sessionReduceActorSystem.actorOf(OutputActor.props(responseObserver)); + /* + create a supervisor actor which assign the tasks to child actors. + we create a child actor for every unique set of keys in a window. + */ + ActorRef supervisorActor = sessionReduceActorSystem + .actorOf(SupervisorActor.props( + sessionReducerFactory, + shutdownActorRef, + outputActor)); + + return new StreamObserver<>() { + @Override + public void onNext(Sessionreduce.SessionReduceRequest sessionReduceRequest) { + // send the message to parent actor, which takes care of distribution. + if (!supervisorActor.isTerminated()) { + // if the operation is a MERGE, make it a blocking call. + if (sessionReduceRequest.getOperation().getEvent() + == Sessionreduce.SessionReduceRequest.WindowOperation.Event.MERGE) { + // on GO SDK side, we wait forever until the MERGE operation is done. + // on Java side, since the Await function requires a timeout, we are setting it to 1h + // for now, which is long enough for us to determine the system is hanging. + // If a MERGE took more than 1h, the system will panic. + Timeout timeout = new Timeout(Duration.create(1, "hour")); + try { + // ask the supervisor to process a merge request. + Future future = Patterns.ask( + supervisorActor, + sessionReduceRequest, + timeout); + // await for the merge done response. + MergeDoneResponse response = (MergeDoneResponse) Await.result( + future, + timeout.duration()); + } catch (Exception e) { + responseObserver.onError(new Throwable( + "Supervisor actor failed processing a MERGE request: " + + e.getMessage())); + } + } else { + supervisorActor.tell(sessionReduceRequest, ActorRef.noSender()); + } + } else { + responseObserver.onError(new Throwable("Supervisor actor was terminated")); + } + } + + @Override + public void onError(Throwable throwable) { + log.error("Error from the client - {}", throwable.getMessage()); + responseObserver.onError(throwable); + } + + @Override + public void onCompleted() { + // indicate the end of input to the supervisor + supervisorActor.tell(Constants.EOF, ActorRef.noSender()); + } + }; + } + + /** + * IsReady is the heartbeat endpoint for gRPC. + */ + @Override + public void isReady( + Empty request, + StreamObserver responseObserver) { + responseObserver.onNext(Sessionreduce.ReadyResponse.newBuilder().setReady(true).build()); + responseObserver.onCompleted(); + } +} diff --git a/src/main/java/io/numaproj/numaflow/sessionreducer/SessionReducerActor.java b/src/main/java/io/numaproj/numaflow/sessionreducer/SessionReducerActor.java new file mode 100644 index 00000000..b59eea85 --- /dev/null +++ b/src/main/java/io/numaproj/numaflow/sessionreducer/SessionReducerActor.java @@ -0,0 +1,128 @@ +package io.numaproj.numaflow.sessionreducer; + +import akka.actor.AbstractActor; +import akka.actor.ActorRef; +import akka.actor.Props; +import akka.japi.pf.ReceiveBuilder; +import io.numaproj.numaflow.sessionreduce.v1.Sessionreduce; +import io.numaproj.numaflow.sessionreducer.model.OutputStreamObserver; +import io.numaproj.numaflow.sessionreducer.model.SessionReducer; + +/** + * Session reducer actor invokes user defined functions to handle session reduce requests. + * Session reducer actor and session window has a one-to-one relationship, meaning + * a session reducer actor only works on its assigned single session window. + */ +class SessionReducerActor extends AbstractActor { + // the session window the actor is working on + private Sessionreduce.KeyedWindow keyedWindow; + private final SessionReducer sessionReducer; + private OutputStreamObserver outputStream; + // when set to true, it means this session is already closed. + private boolean isClosed = false; + + public SessionReducerActor( + Sessionreduce.KeyedWindow keyedWindow, + SessionReducer sessionReducer, + OutputStreamObserver outputStream) { + this.keyedWindow = keyedWindow; + this.sessionReducer = sessionReducer; + this.outputStream = outputStream; + } + + public static Props props( + Sessionreduce.KeyedWindow keyedWindow, + SessionReducer groupBy, + ActorRef outputActor) { + return Props.create( + SessionReducerActor.class, + keyedWindow, + groupBy, + new OutputStreamObserverImpl(outputActor, keyedWindow) + ); + } + + @Override + public Receive createReceive() { + return ReceiveBuilder + .create() + .match(Sessionreduce.KeyedWindow.class, this::updateKeyedWindow) + .match(HandlerDatum.class, this::invokeHandler) + .match(String.class, this::handleEOF) + .match(GetAccumulatorRequest.class, this::handleGetAccumulatorRequest) + .match(MergeAccumulatorRequest.class, this::handleMergeAccumulatorRequest) + .build(); + } + + // receiving a new keyed window, update the keyed window. + // this is for EXPAND operation. + private void updateKeyedWindow(Sessionreduce.KeyedWindow newKeyedWindow) { + // update the keyed window + this.keyedWindow = newKeyedWindow; + // update the output stream to use the new keyed window + OutputStreamObserverImpl newOutputStream = (OutputStreamObserverImpl) this.outputStream; + newOutputStream.setKeyedWindow(newKeyedWindow); + } + + // when receiving a message, process it. + // this is for OPEN/APPEND operation. + private void invokeHandler(HandlerDatum handlerDatum) { + this.sessionReducer.processMessage( + keyedWindow.getKeysList().toArray(new String[0]), + handlerDatum, + outputStream); + } + + // receiving an EOF signal, close the window. + // this is for CLOSE operation or for the close of gRPC input stream. + private void handleEOF(String EOF) { + if (this.isClosed) { + return; + } + // invoke handleEndOfStream to materialize the messages received so far. + this.sessionReducer.handleEndOfStream( + keyedWindow.getKeysList().toArray(new String[0]), + outputStream); + // construct an actor response and send it back to the supervisor actor, indicating the actor + // has finished processing all the messages for the corresponding keyed window. + getSender().tell(buildEOFResponse(), getSelf()); + this.isClosed = true; + } + + // receiving a GetAccumulatorRequest, return the accumulator of the window. + // this is for MERGE operation. + private void handleGetAccumulatorRequest(GetAccumulatorRequest getAccumulatorRequest) { + getSender().tell(buildMergeResponse( + this.sessionReducer.accumulator(), + getAccumulatorRequest.getMergeTaskId()) + , + getSelf()); + // after finishing handling a GetAccumulatorRequest, the session is considered closed. + this.isClosed = true; + } + + // receiving a MergeAccumulatorRequest, merge the accumulator. + // this is for MERGE operation. + private void handleMergeAccumulatorRequest(MergeAccumulatorRequest mergeAccumulatorRequest) { + this.sessionReducer.mergeAccumulator(mergeAccumulatorRequest.getAccumulator()); + } + + private ActorResponse buildEOFResponse() { + Sessionreduce.SessionReduceResponse.Builder responseBuilder = Sessionreduce.SessionReduceResponse.newBuilder(); + responseBuilder.setKeyedWindow(this.keyedWindow); + responseBuilder.setEOF(true); + return ActorResponse.builder() + .response(responseBuilder.build()) + .build(); + } + + private ActorResponse buildMergeResponse(byte[] accumulator, String mergeTaskId) { + Sessionreduce.SessionReduceResponse.Builder responseBuilder = Sessionreduce.SessionReduceResponse.newBuilder(); + responseBuilder.setKeyedWindow(this.keyedWindow); + return ActorResponse.builder() + .response(responseBuilder.build()) + .accumulator(accumulator) + .mergeTaskId(mergeTaskId) + .build(); + } +} diff --git a/src/main/java/io/numaproj/numaflow/sessionreducer/ShutdownActor.java b/src/main/java/io/numaproj/numaflow/sessionreducer/ShutdownActor.java new file mode 100644 index 00000000..dec19931 --- /dev/null +++ b/src/main/java/io/numaproj/numaflow/sessionreducer/ShutdownActor.java @@ -0,0 +1,65 @@ +package io.numaproj.numaflow.sessionreducer; + +import akka.actor.AbstractActor; +import akka.actor.AllDeadLetters; +import akka.actor.Props; +import akka.japi.pf.ReceiveBuilder; +import lombok.AllArgsConstructor; +import lombok.extern.slf4j.Slf4j; + +import java.util.Optional; +import java.util.concurrent.CompletableFuture; + +/** + * Shutdown actor, listens to exceptions and handles shutdown. + */ +@Slf4j +@AllArgsConstructor +class ShutdownActor extends AbstractActor { + private final CompletableFuture failureFuture; + + public static Props props( + CompletableFuture failureFuture) { + return Props.create(ShutdownActor.class, failureFuture); + } + + @Override + public void preRestart(Throwable reason, Optional message) { + failureFuture.completeExceptionally(reason); + } + + @Override + public Receive createReceive() { + return ReceiveBuilder + .create() + .match(Throwable.class, this::shutdown) + .match(String.class, this::completedSuccessfully) + .match(AllDeadLetters.class, this::handleDeadLetters) + .build(); + } + + /* + complete the future with exception so that the exception will be thrown + indicate that same to response observer. + */ + private void shutdown(Throwable throwable) { + log.debug("got a shut down exception"); + failureFuture.completeExceptionally(throwable); + } + + // if there are no exceptions, complete the future without exception. + private void completedSuccessfully(String eof) { + log.debug("completed successfully of shutdown executed"); + failureFuture.complete(null); + // if all the actors completed successfully, we can stop the shutdown actor. + getContext().getSystem().stop(getSelf()); + } + + // if we see dead letters, we need to stop the execution and exit + // to make sure no messages are lost + private void handleDeadLetters(AllDeadLetters deadLetter) { + log.debug("got a dead letter, stopping the execution"); + failureFuture.completeExceptionally(new Throwable("dead letters")); + getContext().getSystem().stop(getSelf()); + } +} diff --git a/src/main/java/io/numaproj/numaflow/sessionreducer/SupervisorActor.java b/src/main/java/io/numaproj/numaflow/sessionreducer/SupervisorActor.java new file mode 100644 index 00000000..0c7b7eb2 --- /dev/null +++ b/src/main/java/io/numaproj/numaflow/sessionreducer/SupervisorActor.java @@ -0,0 +1,403 @@ +package io.numaproj.numaflow.sessionreducer; + +import akka.actor.AbstractActor; +import akka.actor.ActorRef; +import akka.actor.ChildRestartStats; +import akka.actor.Props; +import akka.actor.SupervisorStrategy; +import akka.japi.pf.DeciderBuilder; +import akka.japi.pf.ReceiveBuilder; +import com.google.common.base.Preconditions; +import com.google.protobuf.Timestamp; +import io.numaproj.numaflow.sessionreduce.v1.Sessionreduce; +import io.numaproj.numaflow.sessionreducer.model.SessionReducer; +import io.numaproj.numaflow.sessionreducer.model.SessionReducerFactory; +import lombok.extern.slf4j.Slf4j; +import scala.PartialFunction; +import scala.collection.Iterable; + +import java.time.Instant; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +/** + * Supervisor actor distributes the messages to other actors and handles failures. + */ +@Slf4j +class SupervisorActor extends AbstractActor { + private final SessionReducerFactory sessionReducerFactory; + private final ActorRef shutdownActor; + private final ActorRef outputActor; + // actorMap maintains a map of active sessions. + // key is the unique id of a session, value is the reference to the actor working on the session. + private final Map actorsMap = new HashMap<>(); + // number of accumulators to be collected by current MERGE request. + private int numberOfPendingAccumulators; + // when set to true, isInputStreamClosed means the gRPC input stream has reached EOF. + private boolean isInputStreamClosed = false; + // mergeRequestSender is used to track the reference to the merge request sender, + // it's used to report back when the MERGE request is completed. + // the mergeRequest is sent using ask, which creates a temporary actor as sender behind the scene. + private ActorRef mergeRequestSender; + + public SupervisorActor( + SessionReducerFactory sessionReducerFactory, + ActorRef shutdownActor, + ActorRef outputActor) { + this.sessionReducerFactory = sessionReducerFactory; + this.shutdownActor = shutdownActor; + this.outputActor = outputActor; + } + + public static Props props( + SessionReducerFactory sessionReducerFactory, + ActorRef shutdownActor, + ActorRef outputActor) { + return Props.create( + SupervisorActor.class, + sessionReducerFactory, + shutdownActor, + outputActor); + } + + // if there is an uncaught exception stop in the supervisor actor, send a signal to shut down + @Override + public void preRestart(Throwable reason, Optional message) { + log.debug("supervisor pre restart was executed"); + shutdownActor.tell(reason, ActorRef.noSender()); + Service.sessionReduceActorSystem.stop(getSelf()); + } + + @Override + public SupervisorStrategy supervisorStrategy() { + return new ReduceSupervisorStrategy(); + } + + + @Override + public void postStop() { + log.debug("post stop of supervisor executed - {}", getSelf().toString()); + shutdownActor.tell(Constants.SUCCESS, ActorRef.noSender()); + } + + @Override + public Receive createReceive() { + return ReceiveBuilder + .create() + .match(String.class, this::handleEOF) + .match(Sessionreduce.SessionReduceRequest.class, this::handleReduceRequest) + .match(ActorResponse.class, this::handleActorResponse) + .build(); + } + + private void handleEOF(String EOF) { + this.isInputStreamClosed = true; + if (actorsMap.isEmpty()) { + this.outputActor.tell(EOF, getSelf()); + return; + } + for (Map.Entry entry : actorsMap.entrySet()) { + entry.getValue().tell(EOF, getSelf()); + } + } + + private void handleReduceRequest(Sessionreduce.SessionReduceRequest request) { + Sessionreduce.SessionReduceRequest.WindowOperation windowOperation = request.getOperation(); + switch (windowOperation.getEvent()) { + case OPEN: { + if (windowOperation.getKeyedWindowsCount() != 1) { + throw new RuntimeException( + "open operation error: expected exactly one window"); + } + String windowId = UniqueIdGenerator.getUniqueIdentifier(windowOperation.getKeyedWindows( + 0)); + if (this.actorsMap.containsKey(windowId)) { + throw new RuntimeException( + "received an OPEN request but the session reducer actor already exists"); + } + + ActorRequest createRequest = ActorRequest.builder() + .type(ActorRequestType.OPEN) + .keyedWindow(windowOperation.getKeyedWindows(0)) + .payload(request.hasPayload() ? request.getPayload():null) + .build(); + this.invokeActor(createRequest); + break; + } + case APPEND: { + if (windowOperation.getKeyedWindowsCount() != 1) { + throw new RuntimeException( + "append operation error: expected exactly one window"); + } + ActorRequest appendRequest = ActorRequest.builder() + .type(ActorRequestType.APPEND) + .keyedWindow(windowOperation.getKeyedWindows(0)) + .payload(request.hasPayload() ? request.getPayload():null) + .build(); + this.invokeActor(appendRequest); + break; + } + case CLOSE: { + windowOperation.getKeyedWindowsList().forEach( + keyedWindow -> { + ActorRequest closeRequest = ActorRequest.builder() + .type(ActorRequestType.CLOSE) + .keyedWindow(keyedWindow) + .build(); + this.invokeActor(closeRequest); + }); + break; + } + case EXPAND: { + if (windowOperation.getKeyedWindowsCount() != 2) { + throw new RuntimeException( + "expand operation error: expected exactly two windows"); + } + String currentId = UniqueIdGenerator.getUniqueIdentifier(windowOperation.getKeyedWindows( + 0)); + String newId = UniqueIdGenerator.getUniqueIdentifier(windowOperation.getKeyedWindows( + 1)); + if (!this.actorsMap.containsKey(currentId)) { + throw new RuntimeException( + "expand operation error: session not found for id: " + currentId); + } + // we divide the EXPAND request to two. One is to update the actor. + // The other is to send the payload to the updated actor. + // because in AKKA's actor model, message processing within a single actor is sequential, + // we ensure that the payload is handled using the updated keyed window. + + // 1. ask the session reducer actor to update its keyed window. + // update the map to use the new id to point to the actor. + ActorRequest expandRequest = ActorRequest.builder() + .type(ActorRequestType.EXPAND) + .keyedWindow(windowOperation.getKeyedWindows(0)) + .newKeyedWindow(windowOperation.getKeyedWindows(1)) + .build(); + this.invokeActor(expandRequest); + this.actorsMap.put(newId, this.actorsMap.get(currentId)); + this.actorsMap.remove(currentId); + + // 2. send the payload to the updated actor. + ActorRequest appendRequest = ActorRequest.builder() + .type(ActorRequestType.APPEND) + .keyedWindow(windowOperation.getKeyedWindows(1)) + .payload(request.hasPayload() ? request.getPayload():null) + .build(); + this.invokeActor(appendRequest); + break; + } + case MERGE: { + this.mergeRequestSender = getSender(); + Timestamp mergedStartTime = windowOperation.getKeyedWindows(0).getStart(); + Timestamp mergedEndTime = windowOperation.getKeyedWindows(0).getEnd(); + for (Sessionreduce.KeyedWindow window : windowOperation.getKeyedWindowsList()) { + String id = UniqueIdGenerator.getUniqueIdentifier(window); + if (!this.actorsMap.containsKey(id)) { + throw new RuntimeException( + "merge operation error: session not found for id: " + id); + } + // merged window will be the largest window which contains all the windows. + if (Instant + .ofEpochSecond( + window.getStart().getSeconds(), + window.getStart().getNanos()) + .isBefore(Instant.ofEpochSecond( + mergedStartTime.getSeconds(), + mergedStartTime.getNanos()))) { + mergedStartTime = window.getStart(); + } + if (Instant + .ofEpochSecond( + window.getEnd().getSeconds(), + window.getEnd().getNanos()) + .isAfter(Instant.ofEpochSecond( + mergedEndTime.getSeconds(), + mergedEndTime.getNanos()))) { + mergedEndTime = window.getEnd(); + } + } + + Sessionreduce.KeyedWindow mergedWindow = Sessionreduce.KeyedWindow.newBuilder(). + setStart(mergedStartTime) + .setEnd(mergedEndTime) + .addAllKeys(windowOperation.getKeyedWindows(0).getKeysList()) + .setSlot(windowOperation.getKeyedWindows(0).getSlot()).build(); + + String mergeTaskId = UniqueIdGenerator.getUniqueIdentifier(mergedWindow); + this.numberOfPendingAccumulators = windowOperation.getKeyedWindowsCount(); + for (Sessionreduce.KeyedWindow window : windowOperation.getKeyedWindowsList()) { + // tell the session reducer actor - "hey, you are about to be merged." + ActorRequest getAccumulatorRequest = ActorRequest.builder() + .type(ActorRequestType.GET_ACCUMULATOR) + .keyedWindow(window) + .mergeTaskId(mergeTaskId) + .build(); + this.invokeActor(getAccumulatorRequest); + } + // open a new session for the merged keyed window. + // it's possible that merged keyed window is the same as one of the existing windows, + // in this case, since we already send out the GET_ACCUMULATOR request, it's ok to replace + // the existing window with the new one. + // the accumulator of the old window will get merged to the new window eventually, + // when the supervisor receives the get accumulator response. + ActorRequest openRequest = ActorRequest.builder() + .type(ActorRequestType.OPEN) + .keyedWindow(mergedWindow) + .build(); + this.invokeActor(openRequest); + break; + } + default: + throw new RuntimeException( + "received an unsupported window operation: " + windowOperation.getEvent()); + } + } + + private void invokeActor(ActorRequest actorRequest) { + String uniqueId = UniqueIdGenerator.getUniqueIdentifier(actorRequest.getKeyedWindow()); + switch (actorRequest.getType()) { + case OPEN: { + SessionReducer sessionReducer = sessionReducerFactory.createSessionReducer(); + ActorRef actorRef = getContext() + .actorOf(SessionReducerActor.props( + actorRequest.getKeyedWindow(), + sessionReducer, + this.outputActor + )); + this.actorsMap.put(uniqueId, actorRef); + break; + } + case APPEND: { + if (!this.actorsMap.containsKey(uniqueId)) { + SessionReducer sessionReducer = sessionReducerFactory.createSessionReducer(); + ActorRef actorRef = getContext() + .actorOf(SessionReducerActor.props( + actorRequest.getKeyedWindow(), + sessionReducer, + this.outputActor)); + this.actorsMap.put(uniqueId, actorRef); + } + break; + } + case CLOSE: { + if (this.actorsMap.containsKey(uniqueId)) { + this.actorsMap.get(uniqueId).tell(Constants.EOF, getSelf()); + } + break; + } + case EXPAND: { + this.actorsMap.get(uniqueId).tell(actorRequest.getNewKeyedWindow(), getSelf()); + break; + } + case GET_ACCUMULATOR: { + this.actorsMap + .get(uniqueId) + .tell( + new GetAccumulatorRequest(actorRequest.getMergeTaskId()), + getSelf()); + break; + } + } + + if (actorRequest.getPayload() != null) { + HandlerDatum handlerDatum = constructHandlerDatum(actorRequest.getPayload()); + this.actorsMap.get(uniqueId).tell(handlerDatum, getSelf()); + } + } + + private void handleActorResponse(ActorResponse actorResponse) { + String responseWindowId = UniqueIdGenerator.getUniqueIdentifier(actorResponse + .getResponse() + .getKeyedWindow()); + + if (actorResponse.isEOFResponse()) { + // when the supervisor receives an EOF actor response, + // it means the corresponding session reducer actor has finished its job. + // we remove the entry from the actors map. + this.actorsMap.remove(responseWindowId); + if (this.actorsMap.isEmpty() && this.isInputStreamClosed) { + // the actor map is empty and the gRPC input stream has been closed, hence this is the very last response of the entire system. + // this is the only place to set last. + actorResponse.setLast(true); + this.outputActor.tell(actorResponse, getSelf()); + } else { + this.outputActor.tell(actorResponse, getSelf()); + } + } else { + // handle get accumulator response + String mergeTaskId = actorResponse.getMergeTaskId(); + if (!this.actorsMap.containsKey(mergeTaskId)) { + throw new RuntimeException( + "received an accumulator but the corresponding parent merge session doesn't exist."); + } + this.numberOfPendingAccumulators--; + if (!responseWindowId.equals(mergeTaskId)) { + // release the session that returns us the accumulator, indicating it has finished its lifecycle. + // the session is released without being explicitly closed because it has been merged and tracked by the newly merged session. + // we release a session by simply removing it from the actor map so that the corresponding actor + // gets de-referred and handled by Java GC. + this.actorsMap.remove(responseWindowId); + } + this.actorsMap.get(mergeTaskId).tell( + new MergeAccumulatorRequest( + actorResponse.getAccumulator()), getSelf()); + if (this.numberOfPendingAccumulators == 0) { + // tell the gRPC input stream that the merge request is completely processed. + this.mergeRequestSender.tell(new MergeDoneResponse(), getSelf()); + } + } + } + + private HandlerDatum constructHandlerDatum(Sessionreduce.SessionReduceRequest.Payload payload) { + return new HandlerDatum( + payload.getValue().toByteArray(), + Instant.ofEpochSecond( + payload.getWatermark().getSeconds(), + payload.getWatermark().getNanos()), + Instant.ofEpochSecond( + payload.getEventTime().getSeconds(), + payload.getEventTime().getNanos()) + ); + } + + /* + We need supervisor to handle failures, by default if there are any failures + actors will be restarted, but we want to escalate the exception and terminate + the system. + */ + private final class ReduceSupervisorStrategy extends SupervisorStrategy { + + @Override + public PartialFunction decider() { + return DeciderBuilder.match(Exception.class, e -> SupervisorStrategy.stop()).build(); + } + + @Override + public void handleChildTerminated( + akka.actor.ActorContext context, + ActorRef child, + Iterable children) { + + } + + @Override + public void processFailure( + akka.actor.ActorContext context, + boolean restart, + ActorRef child, + Throwable cause, + ChildRestartStats stats, + Iterable children) { + + Preconditions.checkArgument( + !restart, + "on failures, we will never restart our actors, we escalate"); + /* + tell the shutdown actor about the exception. + */ + log.debug("process failure of supervisor strategy executed - {}", getSelf().toString()); + shutdownActor.tell(cause, context.parent()); + } + } +} diff --git a/src/main/java/io/numaproj/numaflow/sessionreducer/UniqueIdGenerator.java b/src/main/java/io/numaproj/numaflow/sessionreducer/UniqueIdGenerator.java new file mode 100644 index 00000000..447fbf33 --- /dev/null +++ b/src/main/java/io/numaproj/numaflow/sessionreducer/UniqueIdGenerator.java @@ -0,0 +1,29 @@ +package io.numaproj.numaflow.sessionreducer; + +import com.google.protobuf.Timestamp; +import io.numaproj.numaflow.sessionreduce.v1.Sessionreduce; + +import java.time.Instant; + +/** + * UniqueIdGenerator is a utility class to generate a unique id for a keyed session window. + */ +public class UniqueIdGenerator { + private UniqueIdGenerator() { + throw new AssertionError("utility class cannot be instantiated"); + } + + public static String getUniqueIdentifier(Sessionreduce.KeyedWindow keyedWindow) { + long startMillis = convertToEpochMilli(keyedWindow.getStart()); + long endMillis = convertToEpochMilli(keyedWindow.getEnd()); + return String.format( + "%d:%d:%s", + startMillis, + endMillis, + String.join(Constants.DELIMITER, keyedWindow.getKeysList())); + } + + private static long convertToEpochMilli(Timestamp timestamp) { + return Instant.ofEpochSecond(timestamp.getSeconds(), timestamp.getNanos()).toEpochMilli(); + } +} diff --git a/src/main/java/io/numaproj/numaflow/sessionreducer/model/Datum.java b/src/main/java/io/numaproj/numaflow/sessionreducer/model/Datum.java new file mode 100644 index 00000000..99fddfad --- /dev/null +++ b/src/main/java/io/numaproj/numaflow/sessionreducer/model/Datum.java @@ -0,0 +1,29 @@ +package io.numaproj.numaflow.sessionreducer.model; + +import java.time.Instant; + +/** + * Datum contains methods to get the payload information. + */ +public interface Datum { + /** + * method to get the payload value + * + * @return returns the payload value in byte array + */ + byte[] getValue(); + + /** + * method to get the event time of the payload + * + * @return returns the event time of the payload + */ + Instant getEventTime(); + + /** + * method to get the watermark information + * + * @return returns the watermark + */ + Instant getWatermark(); +} diff --git a/src/main/java/io/numaproj/numaflow/sessionreducer/model/Message.java b/src/main/java/io/numaproj/numaflow/sessionreducer/model/Message.java new file mode 100644 index 00000000..524eb485 --- /dev/null +++ b/src/main/java/io/numaproj/numaflow/sessionreducer/model/Message.java @@ -0,0 +1,56 @@ +package io.numaproj.numaflow.sessionreducer.model; + +import lombok.Getter; + +/** + * Message is used to wrap the data returned by Session Reducer functions. + */ +@Getter +public class Message { + public static final String DROP = "U+005C__DROP__"; + private final String[] keys; + private final byte[] value; + private final String[] tags; + + + /** + * used to create Message with value, keys and tags(used for conditional forwarding) + * + * @param value message value + * @param keys message keys + * @param tags message tags which will be used for conditional forwarding + */ + public Message(byte[] value, String[] keys, String[] tags) { + this.keys = keys; + this.value = value; + this.tags = tags; + } + + /** + * used to create Message with value. + * + * @param value message value + */ + public Message(byte[] value) { + this(value, null, null); + } + + /** + * used to create Message with value and keys. + * + * @param value message value + * @param keys message keys + */ + public Message(byte[] value, String[] keys) { + this(value, keys, null); + } + + /** + * creates a Message which will be dropped + * + * @return returns the Message which will be dropped + */ + public static Message toDrop() { + return new Message(new byte[0], null, new String[]{DROP}); + } +} diff --git a/src/main/java/io/numaproj/numaflow/sessionreducer/model/OutputStreamObserver.java b/src/main/java/io/numaproj/numaflow/sessionreducer/model/OutputStreamObserver.java new file mode 100644 index 00000000..edcfc989 --- /dev/null +++ b/src/main/java/io/numaproj/numaflow/sessionreducer/model/OutputStreamObserver.java @@ -0,0 +1,13 @@ +package io.numaproj.numaflow.sessionreducer.model; + +/** + * OutputStreamObserver sends to the output stream, the messages generate by the session reducer. + */ +public interface OutputStreamObserver { + /** + * method will be used for sending messages to the output stream. + * + * @param message the message to be sent + */ + void send(Message message); +} diff --git a/src/main/java/io/numaproj/numaflow/sessionreducer/model/SessionReducer.java b/src/main/java/io/numaproj/numaflow/sessionreducer/model/SessionReducer.java new file mode 100644 index 00000000..ddfdc117 --- /dev/null +++ b/src/main/java/io/numaproj/numaflow/sessionreducer/model/SessionReducer.java @@ -0,0 +1,55 @@ +package io.numaproj.numaflow.sessionreducer.model; + +/** + * SessionReducer exposes methods for performing session reduce operations. + */ +public abstract class SessionReducer { + /** + * processMessage is invoked for each session reduce input message. + * It reads the input data from the datum and performs session reduce operations for the given keys. + * An output stream is provided for sending back the result to the session reduce output stream. + * + * @param keys message keys + * @param datum current message to be processed + * @param outputStreamObserver observer of the reduce result, which is used to send back session reduce responses + */ + public abstract void processMessage( + String[] keys, + Datum datum, + OutputStreamObserver outputStreamObserver); + + /** + * handleEndOfStream handles the closure of the session reduce input stream. + * This method is invoked when the input session reduce stream is closed. + * It provides the capability of constructing final responses based on the messages processed so far. + * + * @param keys message keys + * @param outputStreamObserver observer of the reduce result, which is used to send back reduce responses + */ + public abstract void handleEndOfStream( + String[] keys, + OutputStreamObserver outputStreamObserver); + + /** + * accumulator transforms the current state of the session into a byte array representation. + * This method is invoked when the session is to be merged with another session. + *

+ * e.g. For a session reducer which tracks the number of events in a session by keeping a count integer, + * accumulator() can represent the state of it by returning the byte array representation of the integer. + * + * @return the accumulator for the session reducer. + */ + public abstract byte[] accumulator(); + + /** + * mergeAccumulator merges the current session with another session. + *

+ * e.g. For session reducers which track the number of events in a session by keeping a count integer, + * and use the byte array representation of the integer to form the accumulator, + * mergeAccumulator() can parse the input accumulator to the corresponding count integer and add it + * to the current session count to complete a session merge operation. + * + * @param accumulator the accumulator of the session to be merged in. + */ + public abstract void mergeAccumulator(byte[] accumulator); +} diff --git a/src/main/java/io/numaproj/numaflow/sessionreducer/model/SessionReducerFactory.java b/src/main/java/io/numaproj/numaflow/sessionreducer/model/SessionReducerFactory.java new file mode 100644 index 00000000..b807aaf8 --- /dev/null +++ b/src/main/java/io/numaproj/numaflow/sessionreducer/model/SessionReducerFactory.java @@ -0,0 +1,13 @@ +package io.numaproj.numaflow.sessionreducer.model; + +/** + * ReduceStreamerFactory is the factory for SessionReducer. + */ +public abstract class SessionReducerFactory { + /** + * Helper function to create a session reducer. + * + * @return a concrete session reducer instance + */ + public abstract SessionReducerT createSessionReducer(); +} diff --git a/src/main/proto/sessionreduce/v1/sessionreduce.proto b/src/main/proto/sessionreduce/v1/sessionreduce.proto new file mode 100644 index 00000000..4bac0df4 --- /dev/null +++ b/src/main/proto/sessionreduce/v1/sessionreduce.proto @@ -0,0 +1,83 @@ +syntax = "proto3"; + +option java_package = "io.numaproj.numaflow.sessionreduce.v1"; + +import "google/protobuf/empty.proto"; +import "google/protobuf/timestamp.proto"; + +package sessionreduce.v1; + +service SessionReduce { + // SessionReduceFn applies a reduce function to a request stream. + rpc SessionReduceFn(stream SessionReduceRequest) returns (stream SessionReduceResponse); + + // IsReady is the heartbeat endpoint for gRPC. + rpc IsReady(google.protobuf.Empty) returns (ReadyResponse); +} + +// KeyedWindow represents a window with keys. +// since the client track the keys, we use keyed window. +message KeyedWindow { + google.protobuf.Timestamp start = 1; + google.protobuf.Timestamp end = 2; + string slot = 3; + repeated string keys = 4; +} + +/** + * SessionReduceRequest represents a request element. + */ +message SessionReduceRequest { + // WindowOperation represents a window operation. + // For Aligned window values can be one of OPEN, CLOSE, EXPAND, MERGE and APPEND. + message WindowOperation { + enum Event { + OPEN = 0; + CLOSE = 1; + EXPAND = 2; + MERGE = 3; + APPEND = 4; + } + + Event event = 1; + repeated KeyedWindow keyedWindows = 2; + } + + // Payload represents a payload element. + message Payload { + repeated string keys = 1; + bytes value = 2; + google.protobuf.Timestamp event_time = 3; + google.protobuf.Timestamp watermark = 4; + } + + Payload payload = 1; + WindowOperation operation = 2; +} + +/** + * SessionReduceResponse represents a response element. + */ +message SessionReduceResponse { + // Result represents a result element. It contains the result of the reduce function. + message Result { + repeated string keys = 1; + bytes value = 2; + repeated string tags = 3; + } + + Result result = 1; + + // keyedWindow represents a window to which the result belongs. + KeyedWindow keyedWindow = 2; + + // EOF represents the end of the response for a window. + bool EOF = 3; +} + +/** + * ReadyResponse is the health check result. + */ +message ReadyResponse { + bool ready = 1; +} diff --git a/src/test/java/io/numaproj/numaflow/sessionreducer/GRPCConfigTest.java b/src/test/java/io/numaproj/numaflow/sessionreducer/GRPCConfigTest.java new file mode 100644 index 00000000..9cf2da48 --- /dev/null +++ b/src/test/java/io/numaproj/numaflow/sessionreducer/GRPCConfigTest.java @@ -0,0 +1,38 @@ +package io.numaproj.numaflow.sessionreducer; + +import org.junit.Assert; +import org.junit.Test; + +public class GRPCConfigTest { + + @Test + public void testDefaultGrpcConfig() { + GRPCConfig grpcConfig = GRPCConfig.defaultGrpcConfig(); + Assert.assertNotNull(grpcConfig); + Assert.assertEquals( + Constants.DEFAULT_SERVER_INFO_FILE_PATH, + grpcConfig.getInfoFilePath()); + Assert.assertEquals( + Constants.DEFAULT_MESSAGE_SIZE, + grpcConfig.getMaxMessageSize()); + Assert.assertEquals( + Constants.DEFAULT_SOCKET_PATH, + grpcConfig.getSocketPath()); + } + + @Test + public void testNewBuilder() { + String socketPath = "test-socket-path"; + int maxMessageSize = 2000; + String infoFilePath = "test-info-file-path"; + GRPCConfig grpcConfig = GRPCConfig.newBuilder() + .socketPath(socketPath) + .maxMessageSize(maxMessageSize) + .infoFilePath(infoFilePath) + .build(); + Assert.assertNotNull(grpcConfig); + Assert.assertEquals(socketPath, grpcConfig.getSocketPath()); + Assert.assertEquals(maxMessageSize, grpcConfig.getMaxMessageSize()); + Assert.assertEquals(infoFilePath, grpcConfig.getInfoFilePath()); + } +} diff --git a/src/test/java/io/numaproj/numaflow/sessionreducer/ReduceOutputStreamObserver.java b/src/test/java/io/numaproj/numaflow/sessionreducer/ReduceOutputStreamObserver.java new file mode 100644 index 00000000..95f77da4 --- /dev/null +++ b/src/test/java/io/numaproj/numaflow/sessionreducer/ReduceOutputStreamObserver.java @@ -0,0 +1,70 @@ +package io.numaproj.numaflow.sessionreducer; + +import io.grpc.stub.StreamObserver; +import io.numaproj.numaflow.sessionreduce.v1.Sessionreduce; +import lombok.extern.slf4j.Slf4j; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.concurrent.atomic.AtomicReference; + +/** + * This is a dummy implementation of reduce output stream observer for testing purpose. + */ +@Slf4j +public class ReduceOutputStreamObserver implements StreamObserver { + public AtomicReference completed = new AtomicReference<>(false); + public AtomicReference> resultDatum = new AtomicReference<>( + new ArrayList<>()); + public Throwable t; + + @Override + public void onNext(Sessionreduce.SessionReduceResponse response) { + List receivedResponses = resultDatum.get(); + receivedResponses.add(response); + // sort the list for unit tests. + Collections.sort(receivedResponses, new Comparator() { + @Override + public int compare( + Sessionreduce.SessionReduceResponse o1, + Sessionreduce.SessionReduceResponse o2) { + // compare eof + if (o1.getEOF() && !o2.getEOF()) { + return 1; + } else if (!o1.getEOF() && o2.getEOF()) { + return -1; + } + + // compare keys + int keyCompare = String + .join("-", o1.getKeyedWindow().getKeysList()) + .compareTo(String + .join("-", o2.getKeyedWindow().getKeysList())); + if (keyCompare != 0) { + return keyCompare; + } + + // compare value + return o1 + .getResult() + .getValue() + .toStringUtf8() + .compareTo(o2.getResult().getValue().toStringUtf8()); + } + }); + resultDatum.set(receivedResponses); + } + + @Override + public void onError(Throwable throwable) { + t = throwable; + } + + @Override + public void onCompleted() { + log.info("on completed executed"); + this.completed.set(true); + } +} diff --git a/src/test/java/io/numaproj/numaflow/sessionreducer/ServerErrTest.java b/src/test/java/io/numaproj/numaflow/sessionreducer/ServerErrTest.java new file mode 100644 index 00000000..630d0ca1 --- /dev/null +++ b/src/test/java/io/numaproj/numaflow/sessionreducer/ServerErrTest.java @@ -0,0 +1,252 @@ +package io.numaproj.numaflow.sessionreducer; + +import com.google.protobuf.ByteString; +import com.google.protobuf.Timestamp; +import io.grpc.Context; +import io.grpc.Contexts; +import io.grpc.ManagedChannel; +import io.grpc.Metadata; +import io.grpc.ServerCall; +import io.grpc.ServerCallHandler; +import io.grpc.ServerInterceptor; +import io.grpc.inprocess.InProcessChannelBuilder; +import io.grpc.inprocess.InProcessServerBuilder; +import io.grpc.stub.StreamObserver; +import io.grpc.testing.GrpcCleanupRule; +import io.numaproj.numaflow.sessionreduce.v1.SessionReduceGrpc; +import io.numaproj.numaflow.sessionreduce.v1.Sessionreduce; +import io.numaproj.numaflow.sessionreducer.model.Datum; +import io.numaproj.numaflow.sessionreducer.model.OutputStreamObserver; +import io.numaproj.numaflow.sessionreducer.model.SessionReducer; +import io.numaproj.numaflow.sessionreducer.model.SessionReducerFactory; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; + +import java.util.List; +import java.util.concurrent.atomic.AtomicReference; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +public class ServerErrTest { + @Rule + public final GrpcCleanupRule grpcCleanup = new GrpcCleanupRule(); + private Server server; + private ManagedChannel inProcessChannel; + + @Before + public void setUp() throws Exception { + ServerInterceptor interceptor = new ServerInterceptor() { + @Override + public ServerCall.Listener interceptCall( + ServerCall call, + Metadata headers, + ServerCallHandler next) { + final var context = Context.current(); + return Contexts.interceptCall(context, call, headers, next); + } + }; + + String serverName = InProcessServerBuilder.generateName(); + + GRPCConfig grpcServerConfig = GRPCConfig.newBuilder() + .maxMessageSize(Constants.DEFAULT_MESSAGE_SIZE) + .socketPath(Constants.DEFAULT_SOCKET_PATH) + .infoFilePath("/tmp/numaflow-test-server-info)") + .build(); + + server = new Server( + new SessionReducerErrTestFactory(), + grpcServerConfig); + + server.setServerBuilder(InProcessServerBuilder.forName(serverName) + .intercept(interceptor) + .directExecutor()); + + server.start(); + + inProcessChannel = grpcCleanup.register(InProcessChannelBuilder + .forName(serverName) + .directExecutor() + .build()); + } + + @After + public void tearDown() throws Exception { + server.stop(); + } + + @Test + public void given_actorThrows_when_serverRuns_then_outputStreamContainsThrowable() { + // create an output stream observer + ReduceOutputStreamObserver outputStreamObserver = new ReduceOutputStreamObserver(); + // we need to maintain a reference to any exceptions thrown inside the thread, otherwise even if the assertion failed in the thread, + // the test can still succeed. + AtomicReference exceptionInThread = new AtomicReference<>(); + + Thread t = new Thread(() -> { + while (outputStreamObserver.t == null) { + try { + Thread.sleep(100); + } catch (InterruptedException e) { + exceptionInThread.set(e); + } + } + try { + // this test triggers a supervisor runtime exception by sending an OPEN request with 2 windows. + // we are expecting the error message below. + assertEquals( + "UNKNOWN: java.lang.RuntimeException: open operation error: expected exactly one window", + outputStreamObserver.t.getMessage()); + } catch (Throwable e) { + exceptionInThread.set(e); + } + }); + t.start(); + + StreamObserver inputStreamObserver = SessionReduceGrpc + .newStub(inProcessChannel) + .sessionReduceFn(outputStreamObserver); + + List testKey = List.of("test-key"); + // an open request with two windows is invalid + Sessionreduce.SessionReduceRequest openRequest = Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(testKey) + .setStart(Timestamp + .newBuilder().setSeconds(6000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(7000).build()) + .setSlot("test-slot").build(), + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(testKey) + .setStart(Timestamp + .newBuilder().setSeconds(8000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(9000).build()) + .setSlot("test-slot").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload + .newBuilder() + .addAllKeys(testKey) + .setValue(ByteString.copyFromUtf8(String.valueOf(1))) + .build()) + .build(); + inputStreamObserver.onNext(openRequest); + + try { + t.join(); + } catch (InterruptedException e) { + fail("Thread got interrupted before test assertion."); + } + // Fail the test if any exception caught in the thread + if (exceptionInThread.get() != null) { + fail("Assertion failed in the thread: " + exceptionInThread.get().getMessage()); + } + } + + @Test + public void given_sessionReducerThrows_when_serverRuns_then_outputStreamContainsThrowable() { + // create an output stream observer + ReduceOutputStreamObserver outputStreamObserver = new ReduceOutputStreamObserver(); + // we need to maintain a reference to any exceptions thrown inside the thread, otherwise even if the assertion failed in the thread, + // the test can still succeed. + AtomicReference exceptionInThread = new AtomicReference<>(); + + Thread t = new Thread(() -> { + while (outputStreamObserver.t == null) { + try { + Thread.sleep(100); + } catch (InterruptedException e) { + exceptionInThread.set(e); + } + } + try { + assertEquals( + "UNKNOWN: java.lang.RuntimeException: unknown exception", + outputStreamObserver.t.getMessage()); + } catch (Throwable e) { + exceptionInThread.set(e); + } + }); + t.start(); + + StreamObserver inputStreamObserver = SessionReduceGrpc + .newStub(inProcessChannel) + .sessionReduceFn(outputStreamObserver); + + List testKeys = List.of("reduce-key"); + for (int i = 1; i <= 10; i++) { + Sessionreduce.SessionReduceRequest reduceRequest = Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.APPEND_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(testKeys) + .setStart(Timestamp + .newBuilder().setSeconds(6000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(7000).build()) + .setSlot("test-slot").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload + .newBuilder() + .addAllKeys(testKeys) + .setValue(ByteString.copyFromUtf8(String.valueOf(i))) + .build()) + .build(); + inputStreamObserver.onNext(reduceRequest); + } + + inputStreamObserver.onCompleted(); + + try { + t.join(); + } catch (InterruptedException e) { + fail("Thread got interrupted before test assertion."); + } + // Fail the test if any exception caught in the thread + if (exceptionInThread.get() != null) { + fail("Assertion failed in the thread: " + exceptionInThread.get().getMessage()); + } + } + + public static class SessionReducerErrTestFactory extends SessionReducerFactory { + @Override + public TestSessionReducerHandler createSessionReducer() { + return new TestSessionReducerHandler(); + } + + public static class TestSessionReducerHandler extends SessionReducer { + @Override + public void processMessage( + String[] keys, + Datum datum, + OutputStreamObserver outputStream) { + throw new RuntimeException("unknown exception"); + } + + @Override + public void handleEndOfStream( + String[] keys, + OutputStreamObserver outputStreamObserver) { + + } + + @Override + public byte[] accumulator() { + return new byte[0]; + } + + @Override + public void mergeAccumulator(byte[] accumulator) { + + } + } + } +} diff --git a/src/test/java/io/numaproj/numaflow/sessionreducer/ServerTest.java b/src/test/java/io/numaproj/numaflow/sessionreducer/ServerTest.java new file mode 100644 index 00000000..93775e9a --- /dev/null +++ b/src/test/java/io/numaproj/numaflow/sessionreducer/ServerTest.java @@ -0,0 +1,2405 @@ +package io.numaproj.numaflow.sessionreducer; + +import com.google.protobuf.ByteString; +import com.google.protobuf.Timestamp; +import io.grpc.Context; +import io.grpc.Contexts; +import io.grpc.ManagedChannel; +import io.grpc.Metadata; +import io.grpc.ServerCall; +import io.grpc.ServerCallHandler; +import io.grpc.ServerInterceptor; +import io.grpc.inprocess.InProcessChannelBuilder; +import io.grpc.inprocess.InProcessServerBuilder; +import io.grpc.stub.StreamObserver; +import io.grpc.testing.GrpcCleanupRule; +import io.numaproj.numaflow.sessionreduce.v1.SessionReduceGrpc; +import io.numaproj.numaflow.sessionreduce.v1.Sessionreduce; +import io.numaproj.numaflow.sessionreducer.model.Datum; +import io.numaproj.numaflow.sessionreducer.model.Message; +import io.numaproj.numaflow.sessionreducer.model.OutputStreamObserver; +import io.numaproj.numaflow.sessionreducer.model.SessionReducer; +import io.numaproj.numaflow.sessionreducer.model.SessionReducerFactory; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; + +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.junit.Assert.assertEquals; + +public class ServerTest { + private final static String REDUCE_PROCESSED_KEY_SUFFIX = "-processed"; + @Rule + public final GrpcCleanupRule grpcCleanup = new GrpcCleanupRule(); + private Server server; + private ManagedChannel inProcessChannel; + + @Before + public void setUp() throws Exception { + ServerInterceptor interceptor = new ServerInterceptor() { + @Override + public ServerCall.Listener interceptCall( + ServerCall call, + Metadata headers, + ServerCallHandler next) { + final var context = Context.current(); + return Contexts.interceptCall(context, call, headers, next); + } + }; + + String serverName = InProcessServerBuilder.generateName(); + + GRPCConfig grpcServerConfig = GRPCConfig.newBuilder() + .maxMessageSize(Constants.DEFAULT_MESSAGE_SIZE) + .socketPath(Constants.DEFAULT_SOCKET_PATH) + .infoFilePath("/tmp/numaflow-test-server-info)") + .build(); + + server = new Server( + new SessionReducerTestFactory(), + grpcServerConfig); + + server.setServerBuilder(InProcessServerBuilder.forName(serverName) + .intercept(interceptor) + .directExecutor()); + + server.start(); + + inProcessChannel = grpcCleanup.register(InProcessChannelBuilder + .forName(serverName) + .directExecutor() + .build()); + } + + @After + public void tearDown() throws Exception { + server.stop(); + } + + @Test + public void open_append_close() { + // create an output stream observer + ReduceOutputStreamObserver outputStreamObserver = new ReduceOutputStreamObserver(); + StreamObserver inputStreamObserver = SessionReduceGrpc + .newStub(inProcessChannel) + .sessionReduceFn(outputStreamObserver); + + List requests = List.of( + // open a window for key client1, value 5. window start 60000, end 120000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(120000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client1")) + .setValue(ByteString.copyFromUtf8(String.valueOf(5))) + .build()) + .build(), + // append to key client, value 10. + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.APPEND_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(120000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client1")) + .setValue(ByteString.copyFromUtf8(String.valueOf(10))) + .build()) + .build(), + // append to key client, value 15. + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.APPEND_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(120000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client1")) + .setValue(ByteString.copyFromUtf8(String.valueOf(15))) + .build()) + .build(), + // append to key client without any value. + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.APPEND_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(120000).build()) + .setSlot("slot-0").build())) + .build()) + .build(), + // close the window + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.CLOSE_VALUE) + .addAllKeyedWindows(List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(120000) + .build()) + .setSlot("slot-0").build() + )) + .build()) + .build() + ); + + // send the test requests one by one to the input stream. + for (Sessionreduce.SessionReduceRequest request : requests) { + inputStreamObserver.onNext(request); + } + inputStreamObserver.onCompleted(); + + while (!outputStreamObserver.completed.get()) ; + List result = outputStreamObserver.resultDatum.get(); + assertEquals(2, result.size()); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(false) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(120000).build()) + .addAllKeys(List.of("client1")) + .setSlot("slot-0") + .build()) + .setResult(Sessionreduce.SessionReduceResponse.Result.newBuilder() + .addAllKeys(List.of("client1" + REDUCE_PROCESSED_KEY_SUFFIX)) + .setValue(ByteString.copyFromUtf8(String.valueOf(30)))) + .build(), + result.get(0)); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(true) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(120000).build()) + .addAllKeys(List.of("client1")) + .setSlot("slot-0") + .build()) + .build(), + result.get(1)); + } + + @Test + public void open_expand_close() { + // create an output stream observer + ReduceOutputStreamObserver outputStreamObserver = new ReduceOutputStreamObserver(); + StreamObserver inputStreamObserver = SessionReduceGrpc + .newStub(inProcessChannel) + .sessionReduceFn(outputStreamObserver); + + List requests = List.of( + // open a window for key client1, value 10. window start 60000, end 70000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(70000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client1")) + .setValue(ByteString.copyFromUtf8(String.valueOf(10))) + .build()) + .build(), + // open a window for key client2, value 20. window start 60000, end 70000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(70000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client2")) + .setValue(ByteString.copyFromUtf8(String.valueOf(20))) + .build()) + .build(), + // expand the window for key client1, value 10. expand [60000, 70000] to [60000, 75000] + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.EXPAND_VALUE) + .addAllKeyedWindows(List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(70000) + .build()) + .setSlot("slot-0").build(), + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(75000) + .build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client1")) + .setValue(ByteString.copyFromUtf8(String.valueOf(10))) + .build()) + .build(), + // expand the window for key client2, value 20. expand [60000, 70000] to [60000, 79000] + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.EXPAND_VALUE) + .addAllKeyedWindows(List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(70000) + .build()) + .setSlot("slot-0").build(), + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(79000) + .build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client2")) + .setValue(ByteString.copyFromUtf8(String.valueOf(20))) + .build()) + .build(), + // close both expanded windows + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.CLOSE_VALUE) + .addAllKeyedWindows(List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(75000) + .build()) + .setSlot("slot-0").build(), + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(79000) + .build()) + .setSlot("slot-0").build() + )) + .build()) + .build() + ); + + // send the test requests one by one to the input stream. + for (Sessionreduce.SessionReduceRequest request : requests) { + inputStreamObserver.onNext(request); + } + inputStreamObserver.onCompleted(); + + while (!outputStreamObserver.completed.get()) ; + List result = outputStreamObserver.resultDatum.get(); + + assertEquals(4, result.size()); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(false) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(75000).build()) + .addAllKeys(List.of("client1")) + .setSlot("slot-0") + .build()) + .setResult(Sessionreduce.SessionReduceResponse.Result.newBuilder() + .addAllKeys(List.of("client1" + REDUCE_PROCESSED_KEY_SUFFIX)) + .setValue(ByteString.copyFromUtf8(String.valueOf(20)))) + .build(), + result.get(0)); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(false) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(79000).build()) + .addAllKeys(List.of("client2")) + .setSlot("slot-0") + .build()) + .setResult(Sessionreduce.SessionReduceResponse.Result.newBuilder() + .addAllKeys(List.of("client2" + REDUCE_PROCESSED_KEY_SUFFIX)) + .setValue(ByteString.copyFromUtf8(String.valueOf(40)))) + .build(), + result.get(1)); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(true) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(75000).build()) + .addAllKeys(List.of("client1")) + .setSlot("slot-0") + .build()) + .build(), + result.get(2)); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(true) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(79000).build()) + .addAllKeys(List.of("client2")) + .setSlot("slot-0") + .build()) + .build(), + result.get(3)); + } + + @Test + public void open_merge_close() { + // create an output stream observer + ReduceOutputStreamObserver outputStreamObserver = new ReduceOutputStreamObserver(); + StreamObserver inputStreamObserver = SessionReduceGrpc + .newStub(inProcessChannel) + .sessionReduceFn(outputStreamObserver); + + List requests = List.of( + // open a window for key client1, value 10. window start 60000, end 70000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(70000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client1")) + .setValue(ByteString.copyFromUtf8(String.valueOf(10))) + .build()) + .build(), + // open a window for key client2, value 20. window start 60000, end 70000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(70000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client2")) + .setValue(ByteString.copyFromUtf8(String.valueOf(20))) + .build()) + .build(), + // open a window for key client1, value 10. window start 75000, end 85000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(75000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(85000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client1")) + .setValue(ByteString.copyFromUtf8(String.valueOf(10))) + .build()) + .build(), + // open a window for key client2, value 20. window start 78000, end 88000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(78000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(88000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client2")) + .setValue(ByteString.copyFromUtf8(String.valueOf(20))) + .build()) + .build(), + // merge two windows for key client1 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.MERGE_VALUE) + .addAllKeyedWindows(List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(70000) + .build()) + .setSlot("slot-0").build(), + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(75000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(85000) + .build()) + .setSlot("slot-0").build() + )) + .build()) + .build(), + // merge two windows for key client2 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.MERGE_VALUE) + .addAllKeyedWindows(List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(70000) + .build()) + .setSlot("slot-0").build(), + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(78000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(88000) + .build()) + .setSlot("slot-0").build() + )) + .build()) + .build(), + // close both merged windows + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.CLOSE_VALUE) + .addAllKeyedWindows(List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(85000) + .build()) + .setSlot("slot-0").build(), + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(88000) + .build()) + .setSlot("slot-0").build() + )) + .build()) + .build() + ); + + // send the test requests one by one to the input stream. + for (Sessionreduce.SessionReduceRequest request : requests) { + inputStreamObserver.onNext(request); + } + inputStreamObserver.onCompleted(); + + while (!outputStreamObserver.completed.get()) ; + List result = outputStreamObserver.resultDatum.get(); + + assertEquals(4, result.size()); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(false) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(85000).build()) + .addAllKeys(List.of("client1")) + .setSlot("slot-0") + .build()) + .setResult(Sessionreduce.SessionReduceResponse.Result.newBuilder() + .addAllKeys(List.of("client1" + REDUCE_PROCESSED_KEY_SUFFIX)) + .setValue(ByteString.copyFromUtf8(String.valueOf(20)))) + .build(), + result.get(0)); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(false) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(88000).build()) + .addAllKeys(List.of("client2")) + .setSlot("slot-0") + .build()) + .setResult(Sessionreduce.SessionReduceResponse.Result.newBuilder() + .addAllKeys(List.of("client2" + REDUCE_PROCESSED_KEY_SUFFIX)) + .setValue(ByteString.copyFromUtf8(String.valueOf(40)))) + .build(), + result.get(1)); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(true) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(85000).build()) + .addAllKeys(List.of("client1")) + .setSlot("slot-0") + .build()) + .build(), + result.get(2)); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(true) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(88000).build()) + .addAllKeys(List.of("client2")) + .setSlot("slot-0") + .build()) + .build(), + result.get(3)); + } + + @Test + public void open_expand_append_merge_close() throws InterruptedException { + // create an output stream observer + ReduceOutputStreamObserver outputStreamObserver = new ReduceOutputStreamObserver(); + StreamObserver inputStreamObserver = SessionReduceGrpc + .newStub(inProcessChannel) + .sessionReduceFn(outputStreamObserver); + + List requests = List.of( + // open a window for key client1, value 5. window start 60000, end 70000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(70000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client1")) + .setValue(ByteString.copyFromUtf8(String.valueOf(5))) + .build()) + .build(), + // open a window for key client2, value 10. window start 60000, end 70000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(70000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client2")) + .setValue(ByteString.copyFromUtf8(String.valueOf(10))) + .build()) + .build(), + // open a window for key client1, value 5. window start 75000, end 85000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(75000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(85000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client1")) + .setValue(ByteString.copyFromUtf8(String.valueOf(5))) + .build()) + .build(), + // open a window for key client2, value 10. window start 78000, end 88000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(78000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(88000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client2")) + .setValue(ByteString.copyFromUtf8(String.valueOf(10))) + .build()) + .build(), + // expand a window for key client1, value 5. expand from [75000, 85000] to [75000, 95000] + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.EXPAND_VALUE) + .addAllKeyedWindows( + List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder() + .setSeconds(75000) + .build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(85000) + .build()) + .setSlot("slot-0").build(), + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder() + .setSeconds(75000) + .build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(95000) + .build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client1")) + .setValue(ByteString.copyFromUtf8(String.valueOf(5))) + .build()) + .build(), + // expand a window for key client2, value 10. expand from [78000, 88000] to [78000, 98000] + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.EXPAND_VALUE) + .addAllKeyedWindows( + List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder() + .setSeconds(78000) + .build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(88000) + .build()) + .setSlot("slot-0").build(), + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder() + .setSeconds(78000) + .build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(98000) + .build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client2")) + .setValue(ByteString.copyFromUtf8(String.valueOf(10))) + .build()) + .build(), + // append to the key client1 with value 5 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.APPEND_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow + .newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(75000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(95000).build()) + .setSlot("slot-0") + .build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client1")) + .setValue(ByteString.copyFromUtf8(String.valueOf(5))) + .build()) + .build(), + // append to the key client2 with value 10 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.APPEND_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow + .newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(78000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(98000).build()) + .setSlot("slot-0") + .build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client2")) + .setValue(ByteString.copyFromUtf8(String.valueOf(10))) + .build()) + .build(), + // merge two windows for key client1 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.MERGE_VALUE) + .addAllKeyedWindows(List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(70000) + .build()) + .setSlot("slot-0").build(), + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(75000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(95000) + .build()) + .setSlot("slot-0").build() + )) + .build()) + .build(), + // merge two windows for key client2 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.MERGE_VALUE) + .addAllKeyedWindows(List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(70000) + .build()) + .setSlot("slot-0").build(), + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(78000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(98000) + .build()) + .setSlot("slot-0").build() + )) + .build()) + .build(), + // close both merged windows + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.CLOSE_VALUE) + .addAllKeyedWindows(List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(95000) + .build()) + .setSlot("slot-0").build(), + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(98000) + .build()) + .setSlot("slot-0").build() + )) + .build()) + .build() + ); + + // send the test requests one by one to the input stream. + for (Sessionreduce.SessionReduceRequest request : requests) { + inputStreamObserver.onNext(request); + } + // This sleep statement tests a case when there is no active windows and an EOF is received. + Thread.sleep(1000); + inputStreamObserver.onCompleted(); + + while (!outputStreamObserver.completed.get()) ; + List result = outputStreamObserver.resultDatum.get(); + + assertEquals(4, result.size()); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(false) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(95000).build()) + .addAllKeys(List.of("client1")) + .setSlot("slot-0") + .build()) + .setResult(Sessionreduce.SessionReduceResponse.Result.newBuilder() + .addAllKeys(List.of("client1" + REDUCE_PROCESSED_KEY_SUFFIX)) + .setValue(ByteString.copyFromUtf8(String.valueOf(20)))) + .build(), + result.get(0)); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(false) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(98000).build()) + .addAllKeys(List.of("client2")) + .setSlot("slot-0") + .build()) + .setResult(Sessionreduce.SessionReduceResponse.Result.newBuilder() + .addAllKeys(List.of("client2" + REDUCE_PROCESSED_KEY_SUFFIX)) + .setValue(ByteString.copyFromUtf8(String.valueOf(40)))) + .build(), + result.get(1)); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(true) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(95000).build()) + .addAllKeys(List.of("client1")) + .setSlot("slot-0") + .build()) + .build(), + result.get(2)); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(true) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(98000).build()) + .addAllKeys(List.of("client2")) + .setSlot("slot-0") + .build()) + .build(), + result.get(3)); + } + + @Test + public void open_merge_append_close() { + // create an output stream observer + ReduceOutputStreamObserver outputStreamObserver = new ReduceOutputStreamObserver(); + StreamObserver inputStreamObserver = SessionReduceGrpc + .newStub(inProcessChannel) + .sessionReduceFn(outputStreamObserver); + + List requests = List.of( + // open a window for key client1, value 10. window start 60000, end 70000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(70000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client1")) + .setValue(ByteString.copyFromUtf8(String.valueOf(10))) + .build()) + .build(), + // open a window for key client2, value 20. window start 60000, end 70000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(70000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client2")) + .setValue(ByteString.copyFromUtf8(String.valueOf(20))) + .build()) + .build(), + // open a window for key client1, value 10. window start 75000, end 85000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(75000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(85000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client1")) + .setValue(ByteString.copyFromUtf8(String.valueOf(10))) + .build()) + .build(), + // open a window for key client2, value 20. window start 78000, end 88000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(78000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(88000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client2")) + .setValue(ByteString.copyFromUtf8(String.valueOf(20))) + .build()) + .build(), + // merge two windows for key client1 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.MERGE_VALUE) + .addAllKeyedWindows(List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(70000) + .build()) + .setSlot("slot-0").build(), + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(75000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(85000) + .build()) + .setSlot("slot-0").build() + )) + .build()) + .build(), + // append to it value 10 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.APPEND_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(85000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client1")) + .setValue(ByteString.copyFromUtf8(String.valueOf(10))) + .build()) + .build(), + // merge two windows for key client2 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.MERGE_VALUE) + .addAllKeyedWindows(List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(70000) + .build()) + .setSlot("slot-0").build(), + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(78000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(88000) + .build()) + .setSlot("slot-0").build() + )) + .build()) + .build(), + // append to it value 10 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.APPEND_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(88000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client2")) + .setValue(ByteString.copyFromUtf8(String.valueOf(10))) + .build()) + .build(), + // close both windows + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.CLOSE_VALUE) + .addAllKeyedWindows(List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(85000) + .build()) + .setSlot("slot-0").build(), + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(88000) + .build()) + .setSlot("slot-0").build() + )) + .build()) + .build() + ); + + // send the test requests one by one to the input stream. + for (Sessionreduce.SessionReduceRequest request : requests) { + inputStreamObserver.onNext(request); + } + inputStreamObserver.onCompleted(); + + while (!outputStreamObserver.completed.get()) ; + List result = outputStreamObserver.resultDatum.get(); + + assertEquals(4, result.size()); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(false) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(85000).build()) + .addAllKeys(List.of("client1")) + .setSlot("slot-0") + .build()) + .setResult(Sessionreduce.SessionReduceResponse.Result.newBuilder() + .addAllKeys(List.of("client1" + REDUCE_PROCESSED_KEY_SUFFIX)) + .setValue(ByteString.copyFromUtf8(String.valueOf(30)))) + .build(), + result.get(0)); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(false) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(88000).build()) + .addAllKeys(List.of("client2")) + .setSlot("slot-0") + .build()) + .setResult(Sessionreduce.SessionReduceResponse.Result.newBuilder() + .addAllKeys(List.of("client2" + REDUCE_PROCESSED_KEY_SUFFIX)) + .setValue(ByteString.copyFromUtf8(String.valueOf(50)))) + .build(), + result.get(1)); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(true) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(85000).build()) + .addAllKeys(List.of("client1")) + .setSlot("slot-0") + .build()) + .build(), + result.get(2)); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(true) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(88000).build()) + .addAllKeys(List.of("client2")) + .setSlot("slot-0") + .build()) + .build(), + result.get(3)); + } + + @Test + public void open_merge_expand_close() { + // create an output stream observer + ReduceOutputStreamObserver outputStreamObserver = new ReduceOutputStreamObserver(); + StreamObserver inputStreamObserver = SessionReduceGrpc + .newStub(inProcessChannel) + .sessionReduceFn(outputStreamObserver); + + List requests = List.of( + // open a window for key client1, value 10. window start 60000, end 70000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(70000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client1")) + .setValue(ByteString.copyFromUtf8(String.valueOf(10))) + .build()) + .build(), + // open a window for key client2, value 20. window start 60000, end 70000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(70000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client2")) + .setValue(ByteString.copyFromUtf8(String.valueOf(20))) + .build()) + .build(), + // open a window for key client1, value 10. window start 75000, end 85000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(75000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(85000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client1")) + .setValue(ByteString.copyFromUtf8(String.valueOf(10))) + .build()) + .build(), + // open a window for key client2, value 20. window start 78000, end 88000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(78000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(88000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client2")) + .setValue(ByteString.copyFromUtf8(String.valueOf(20))) + .build()) + .build(), + // merge two windows for key client1 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.MERGE_VALUE) + .addAllKeyedWindows(List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(70000) + .build()) + .setSlot("slot-0").build(), + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(75000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(85000) + .build()) + .setSlot("slot-0").build() + )) + .build()) + .build(), + // merge two windows for key client2 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.MERGE_VALUE) + .addAllKeyedWindows(List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(70000) + .build()) + .setSlot("slot-0").build(), + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(78000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(88000) + .build()) + .setSlot("slot-0").build() + )) + .build()) + .build(), + // expand the window for key client1, value 10. expand [60000, 85000] to [60000, 95000] + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.EXPAND_VALUE) + .addAllKeyedWindows(List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(85000) + .build()) + .setSlot("slot-0").build(), + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(95000) + .build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client1")) + .setValue(ByteString.copyFromUtf8(String.valueOf(10))) + .build()) + .build(), + // expand the window for key client2, value 10. expand [60000, 88000] to [60000, 98000] + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.EXPAND_VALUE) + .addAllKeyedWindows(List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(88000) + .build()) + .setSlot("slot-0").build(), + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(98000) + .build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client2")) + .setValue(ByteString.copyFromUtf8(String.valueOf(10))) + .build()) + .build(), + // close both windows + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.CLOSE_VALUE) + .addAllKeyedWindows(List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(95000) + .build()) + .setSlot("slot-0").build(), + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(98000) + .build()) + .setSlot("slot-0").build() + )) + .build()) + .build() + ); + + // send the test requests one by one to the input stream. + for (Sessionreduce.SessionReduceRequest request : requests) { + inputStreamObserver.onNext(request); + } + inputStreamObserver.onCompleted(); + + while (!outputStreamObserver.completed.get()) ; + List result = outputStreamObserver.resultDatum.get(); + + assertEquals(4, result.size()); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(false) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(95000).build()) + .addAllKeys(List.of("client1")) + .setSlot("slot-0") + .build()) + .setResult(Sessionreduce.SessionReduceResponse.Result.newBuilder() + .addAllKeys(List.of("client1" + REDUCE_PROCESSED_KEY_SUFFIX)) + .setValue(ByteString.copyFromUtf8(String.valueOf(30)))) + .build(), + result.get(0)); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(false) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(98000).build()) + .addAllKeys(List.of("client2")) + .setSlot("slot-0") + .build()) + .setResult(Sessionreduce.SessionReduceResponse.Result.newBuilder() + .addAllKeys(List.of("client2" + REDUCE_PROCESSED_KEY_SUFFIX)) + .setValue(ByteString.copyFromUtf8(String.valueOf(50)))) + .build(), + result.get(1)); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(true) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(95000).build()) + .addAllKeys(List.of("client1")) + .setSlot("slot-0") + .build()) + .build(), + result.get(2)); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(true) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(98000).build()) + .addAllKeys(List.of("client2")) + .setSlot("slot-0") + .build()) + .build(), + result.get(3)); + } + + @Test + public void open_merge_merge_close() { + // create an output stream observer + ReduceOutputStreamObserver outputStreamObserver = new ReduceOutputStreamObserver(); + StreamObserver inputStreamObserver = SessionReduceGrpc + .newStub(inProcessChannel) + .sessionReduceFn(outputStreamObserver); + + List requests = List.of( + // open a window for key client1, value 10. window start 60000, end 70000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(70000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client1")) + .setValue(ByteString.copyFromUtf8(String.valueOf(10))) + .build()) + .build(), + // open a window for key client2, value 20. window start 60000, end 70000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(70000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client2")) + .setValue(ByteString.copyFromUtf8(String.valueOf(20))) + .build()) + .build(), + // open a window for key client1, value 10. window start 75000, end 85000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(75000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(85000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client1")) + .setValue(ByteString.copyFromUtf8(String.valueOf(10))) + .build()) + .build(), + // open a window for key client2, value 20. window start 78000, end 88000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(78000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(88000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client2")) + .setValue(ByteString.copyFromUtf8(String.valueOf(20))) + .build()) + .build(), + // merge two windows for key client1 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.MERGE_VALUE) + .addAllKeyedWindows(List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(70000) + .build()) + .setSlot("slot-0").build(), + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(75000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(85000) + .build()) + .setSlot("slot-0").build() + )) + .build()) + .build(), + // merge two windows for key client2 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.MERGE_VALUE) + .addAllKeyedWindows(List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(70000) + .build()) + .setSlot("slot-0").build(), + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(78000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(88000) + .build()) + .setSlot("slot-0").build() + )) + .build()) + .build(), + // open a window for key client1, value 10. window start 50000, end 80000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(50000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(80000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client1")) + .setValue(ByteString.copyFromUtf8(String.valueOf(10))) + .build()) + .build(), + // open a window for key client2, value 10. window start 50000, end 80000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(50000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(80000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client2")) + .setValue(ByteString.copyFromUtf8(String.valueOf(10))) + .build()) + .build(), + // merge two windows for key client1 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.MERGE_VALUE) + .addAllKeyedWindows(List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(85000) + .build()) + .setSlot("slot-0").build(), + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(50000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(80000) + .build()) + .setSlot("slot-0").build() + )) + .build()) + .build(), + // merge two windows for key client2 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.MERGE_VALUE) + .addAllKeyedWindows(List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(88000) + .build()) + .setSlot("slot-0").build(), + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(50000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(80000) + .build()) + .setSlot("slot-0").build() + )) + .build()) + .build(), + // close both merged windows + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.CLOSE_VALUE) + .addAllKeyedWindows(List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(50000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(85000) + .build()) + .setSlot("slot-0").build(), + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(50000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(88000) + .build()) + .setSlot("slot-0").build() + )) + .build()) + .build() + ); + + // send the test requests one by one to the input stream. + for (Sessionreduce.SessionReduceRequest request : requests) { + inputStreamObserver.onNext(request); + } + inputStreamObserver.onCompleted(); + + while (!outputStreamObserver.completed.get()) ; + List result = outputStreamObserver.resultDatum.get(); + + assertEquals(4, result.size()); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(false) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(50000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(85000).build()) + .addAllKeys(List.of("client1")) + .setSlot("slot-0") + .build()) + .setResult(Sessionreduce.SessionReduceResponse.Result.newBuilder() + .addAllKeys(List.of("client1" + REDUCE_PROCESSED_KEY_SUFFIX)) + .setValue(ByteString.copyFromUtf8(String.valueOf(30)))) + .build(), + result.get(0)); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(false) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(50000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(88000).build()) + .addAllKeys(List.of("client2")) + .setSlot("slot-0") + .build()) + .setResult(Sessionreduce.SessionReduceResponse.Result.newBuilder() + .addAllKeys(List.of("client2" + REDUCE_PROCESSED_KEY_SUFFIX)) + .setValue(ByteString.copyFromUtf8(String.valueOf(50)))) + .build(), + result.get(1)); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(true) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(50000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(85000).build()) + .addAllKeys(List.of("client1")) + .setSlot("slot-0") + .build()) + .build(), + result.get(2)); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(true) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(50000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(88000).build()) + .addAllKeys(List.of("client2")) + .setSlot("slot-0") + .build()) + .build(), + result.get(3)); + } + + + // till now, we have completed the Java version of the unit tests in Go SDK package: https://github.com/numaproj/numaflow-go/blob/main/pkg/sessionreducer/service_test.go + // below are more tests that are NOT in the numaflow-go unit tests. + @Test + public void open_merge_close_mergeIntoAnExistingWindow() { + // create an output stream observer + ReduceOutputStreamObserver outputStreamObserver = new ReduceOutputStreamObserver(); + StreamObserver inputStreamObserver = SessionReduceGrpc + .newStub(inProcessChannel) + .sessionReduceFn(outputStreamObserver); + + List requests = List.of( + // open a window for key client1, value 10. window start 60000, end 70000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(70000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client1")) + .setValue(ByteString.copyFromUtf8(String.valueOf(10))) + .build()) + .build(), + // open a window for key client1, value 20. window start 61000, end 69000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(61000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(69000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client1")) + .setValue(ByteString.copyFromUtf8(String.valueOf(20))) + .build()) + .build(), + // open a window for key client1, value 1. window start 62000, end 69500 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(62000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(69500).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client1")) + .setValue(ByteString.copyFromUtf8(String.valueOf(1))) + .build()) + .build(), + // merge the windows for key client1 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.MERGE_VALUE) + .addAllKeyedWindows(List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(70000) + .build()) + .setSlot("slot-0").build(), + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(61000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(69000) + .build()) + .setSlot("slot-0").build(), + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(62000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(69500) + .build()) + .setSlot("slot-0").build() + )) + .build()) + .build(), + // close the merged window + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.CLOSE_VALUE) + .addAllKeyedWindows(List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(70000) + .build()) + .setSlot("slot-0").build() + )) + .build()) + .build() + ); + + // send the test requests one by one to the input stream. + for (Sessionreduce.SessionReduceRequest request : requests) { + inputStreamObserver.onNext(request); + } + inputStreamObserver.onCompleted(); + + while (!outputStreamObserver.completed.get()) ; + List result = outputStreamObserver.resultDatum.get(); + + assertEquals(2, result.size()); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(false) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(70000).build()) + .addAllKeys(List.of("client1")) + .setSlot("slot-0") + .build()) + .setResult(Sessionreduce.SessionReduceResponse.Result.newBuilder() + .addAllKeys(List.of("client1" + REDUCE_PROCESSED_KEY_SUFFIX)) + .setValue(ByteString.copyFromUtf8(String.valueOf(31)))) + .build(), + result.get(0)); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(true) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(70000).build()) + .addAllKeys(List.of("client1")) + .setSlot("slot-0") + .build()) + .build(), + result.get(1)); + } + + @Test + public void open_close_open_open_eof() { + // create an output stream observer + ReduceOutputStreamObserver outputStreamObserver = new ReduceOutputStreamObserver(); + StreamObserver inputStreamObserver = SessionReduceGrpc + .newStub(inProcessChannel) + .sessionReduceFn(outputStreamObserver); + + List requests = List.of( + // open a window for key client1, value 10. window start 60000, end 70000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(70000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client1")) + .setValue(ByteString.copyFromUtf8(String.valueOf(10))) + .build()) + .build(), + // close the window for client1 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.CLOSE_VALUE) + .addAllKeyedWindows(List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(70000) + .build()) + .setSlot("slot-0").build() + )) + .build()) + .build(), + // open a window for key client2, value 20. window start 61000, end 69000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client2")) + .setStart(Timestamp + .newBuilder().setSeconds(61000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(69000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client2")) + .setValue(ByteString.copyFromUtf8(String.valueOf(20))) + .build()) + .build(), + // open a window for key client3, value 30. window start 62000, end 68000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client3")) + .setStart(Timestamp + .newBuilder().setSeconds(62000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(68000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client3")) + .setValue(ByteString.copyFromUtf8(String.valueOf(30))) + .build()) + .build() + + ); + + // send the test requests one by one to the input stream. + for (Sessionreduce.SessionReduceRequest request : requests) { + inputStreamObserver.onNext(request); + } + inputStreamObserver.onCompleted(); + + while (!outputStreamObserver.completed.get()) ; + List result = outputStreamObserver.resultDatum.get(); + + assertEquals(6, result.size()); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(false) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(70000).build()) + .addAllKeys(List.of("client1")) + .setSlot("slot-0") + .build()) + .setResult(Sessionreduce.SessionReduceResponse.Result.newBuilder() + .addAllKeys(List.of("client1" + REDUCE_PROCESSED_KEY_SUFFIX)) + .setValue(ByteString.copyFromUtf8(String.valueOf(10)))) + .build(), + result.get(0)); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(false) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(61000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(69000).build()) + .addAllKeys(List.of("client2")) + .setSlot("slot-0") + .build()) + .setResult(Sessionreduce.SessionReduceResponse.Result.newBuilder() + .addAllKeys(List.of("client2" + REDUCE_PROCESSED_KEY_SUFFIX)) + .setValue(ByteString.copyFromUtf8(String.valueOf(20)))) + .build(), + result.get(1)); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(false) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(62000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(68000).build()) + .addAllKeys(List.of("client3")) + .setSlot("slot-0") + .build()) + .setResult(Sessionreduce.SessionReduceResponse.Result.newBuilder() + .addAllKeys(List.of("client3" + REDUCE_PROCESSED_KEY_SUFFIX)) + .setValue(ByteString.copyFromUtf8(String.valueOf(30)))) + .build(), + result.get(2)); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(true) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(70000).build()) + .addAllKeys(List.of("client1")) + .setSlot("slot-0") + .build()) + .build(), + result.get(3)); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(true) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(61000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(69000).build()) + .addAllKeys(List.of("client2")) + .setSlot("slot-0") + .build()) + .build(), + result.get(4)); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(true) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(62000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(68000).build()) + .addAllKeys(List.of("client3")) + .setSlot("slot-0") + .build()) + .build(), + result.get(5)); + } + + @Test + public void append_udf_send_to_output_close() { + // create an output stream observer + ReduceOutputStreamObserver outputStreamObserver = new ReduceOutputStreamObserver(); + StreamObserver inputStreamObserver = SessionReduceGrpc + .newStub(inProcessChannel) + .sessionReduceFn(outputStreamObserver); + + List requests = List.of( + // open a window for key client1, value 10. window start 60000, end 70000 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.APPEND_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(70000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client1")) + .setValue(ByteString.copyFromUtf8(String.valueOf(10))) + .build()) + .build(), + // append to it value 20 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.APPEND_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(70000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client1")) + .setValue(ByteString.copyFromUtf8(String.valueOf(20))) + .build()) + .build(), + // append to it value 30 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.APPEND_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(70000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client1")) + .setValue(ByteString.copyFromUtf8(String.valueOf(30))) + .build()) + .build(), + // append to it value 40 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.APPEND_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(70000).build()) + .setSlot("slot-0").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(List.of("client1")) + .setValue(ByteString.copyFromUtf8(String.valueOf(40))) + .build()) + .build(), + // close the window for client1 + Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.CLOSE_VALUE) + .addAllKeyedWindows(List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(List.of("client1")) + .setStart(Timestamp + .newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp + .newBuilder() + .setSeconds(70000) + .build()) + .setSlot("slot-0").build() + )) + .build()) + .build() + ); + + // send the test requests one by one to the input stream. + for (Sessionreduce.SessionReduceRequest request : requests) { + inputStreamObserver.onNext(request); + } + inputStreamObserver.onCompleted(); + + while (!outputStreamObserver.completed.get()) ; + List result = outputStreamObserver.resultDatum.get(); + + assertEquals(3, result.size()); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(false) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(70000).build()) + .addAllKeys(List.of("client1")) + .setSlot("slot-0") + .build()) + .setResult(Sessionreduce.SessionReduceResponse.Result.newBuilder() + .addAllKeys(List.of("client1" + REDUCE_PROCESSED_KEY_SUFFIX)) + .setValue(ByteString.copyFromUtf8(String.valueOf(40)))) + .build(), + result.get(0)); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(false) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(70000).build()) + .addAllKeys(List.of("client1")) + .setSlot("slot-0") + .build()) + .setResult(Sessionreduce.SessionReduceResponse.Result.newBuilder() + .addAllKeys(List.of("client1" + REDUCE_PROCESSED_KEY_SUFFIX)) + .setValue(ByteString.copyFromUtf8(String.valueOf(60)))) + .build(), + result.get(1)); + assertEquals( + Sessionreduce.SessionReduceResponse.newBuilder() + .setEOF(true) + .setKeyedWindow(Sessionreduce.KeyedWindow.newBuilder() + .setStart(Timestamp.newBuilder().setSeconds(60000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(70000).build()) + .addAllKeys(List.of("client1")) + .setSlot("slot-0") + .build()) + .build(), + result.get(2)); + } + + + public static class SessionReducerTestFactory extends SessionReducerFactory { + @Override + public TestSessionReducerHandler createSessionReducer() { + return new TestSessionReducerHandler(); + } + + public static class TestSessionReducerHandler extends SessionReducer { + private final AtomicInteger sum = new AtomicInteger(0); + + @Override + public void processMessage( + String[] keys, + Datum datum, + OutputStreamObserver outputStreamObserver) { + sum.addAndGet(Integer.parseInt(new String(datum.getValue()))); + if (sum.get() > 50) { + String[] updatedKeys = Arrays + .stream(keys) + .map(c -> c + REDUCE_PROCESSED_KEY_SUFFIX) + .toArray(String[]::new); + Message message = new Message( + String.valueOf(sum.get()).getBytes(), + updatedKeys); + outputStreamObserver.send(message); + // reset sum + sum.set(0); + } + } + + @Override + public void handleEndOfStream( + String[] keys, + OutputStreamObserver outputStreamObserver) { + String[] updatedKeys = Arrays + .stream(keys) + .map(c -> c + REDUCE_PROCESSED_KEY_SUFFIX) + .toArray(String[]::new); + Message message = new Message(String.valueOf(sum.get()).getBytes(), updatedKeys); + outputStreamObserver.send(message); + } + + @Override + public byte[] accumulator() { + return String.valueOf(sum.get()).getBytes(); + } + + @Override + public void mergeAccumulator(byte[] accumulator) { + int value = Integer.parseInt(new String(accumulator)); + sum.addAndGet(value); + } + } + } +} diff --git a/src/test/java/io/numaproj/numaflow/sessionreducer/ShutdownActorTest.java b/src/test/java/io/numaproj/numaflow/sessionreducer/ShutdownActorTest.java new file mode 100644 index 00000000..82385396 --- /dev/null +++ b/src/test/java/io/numaproj/numaflow/sessionreducer/ShutdownActorTest.java @@ -0,0 +1,206 @@ +package io.numaproj.numaflow.sessionreducer; + +import akka.actor.ActorRef; +import akka.actor.ActorSystem; +import akka.actor.AllDeadLetters; +import akka.actor.DeadLetter; +import com.google.protobuf.ByteString; +import com.google.protobuf.Timestamp; +import io.numaproj.numaflow.sessionreduce.v1.Sessionreduce; +import io.numaproj.numaflow.sessionreducer.model.Datum; +import io.numaproj.numaflow.sessionreducer.model.Message; +import io.numaproj.numaflow.sessionreducer.model.OutputStreamObserver; +import io.numaproj.numaflow.sessionreducer.model.SessionReducer; +import io.numaproj.numaflow.sessionreducer.model.SessionReducerFactory; +import org.junit.Test; + +import java.util.List; +import java.util.concurrent.CompletableFuture; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +public class ShutdownActorTest { + @Test + public void given_shutdownActor_when_supervisorActorThrows_thenFutureCompletesWithExpectedException() { + final ActorSystem actorSystem = ActorSystem.create("test-system-1"); + CompletableFuture completableFuture = new CompletableFuture<>(); + + List keys = List.of("reduceKey"); + ActorRef shutdownActor = actorSystem + .actorOf(ShutdownActor + .props(completableFuture)); + + ReduceOutputStreamObserver reduceOutputStreamObserver = new ReduceOutputStreamObserver(); + + ActorRef outputActor = actorSystem.actorOf(OutputActor + .props(reduceOutputStreamObserver)); + + ActorRef supervisorActor = actorSystem + .actorOf(SupervisorActor + .props( + new TestExceptionFactory(), + shutdownActor, + outputActor)); + + Sessionreduce.SessionReduceRequest openRequest = Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.OPEN_VALUE) + .addAllKeyedWindows(List.of( + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(keys) + .setStart(Timestamp + .newBuilder().setSeconds(6000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(7000).build()) + .setSlot("test-slot").build(), + Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(keys) + .setStart(Timestamp + .newBuilder().setSeconds(7000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(8000).build()) + .setSlot("test-slot").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(keys) + .setValue(ByteString.copyFromUtf8(String.valueOf(1))) + .build()) + .build(); + + supervisorActor.tell(openRequest, ActorRef.noSender()); + + try { + completableFuture.get(); + fail("Expected the future to complete with exception"); + } catch (Exception e) { + assertEquals( + e.getMessage(), + "java.lang.RuntimeException: open operation error: expected exactly one window"); + } + } + + @Test + public void given_shutdownActor_when_udfThrows_thenFutureCompletesWithExpectedException() { + final ActorSystem actorSystem = ActorSystem.create("test-system-1"); + CompletableFuture completableFuture = new CompletableFuture<>(); + + List keys = List.of("reduceKey"); + ActorRef shutdownActor = actorSystem + .actorOf(ShutdownActor + .props(completableFuture)); + + ReduceOutputStreamObserver reduceOutputStreamObserver = new ReduceOutputStreamObserver(); + + ActorRef outputActor = actorSystem.actorOf(OutputActor + .props(reduceOutputStreamObserver)); + + ActorRef supervisorActor = actorSystem + .actorOf(SupervisorActor + .props( + new TestExceptionFactory(), + shutdownActor, + outputActor)); + + Sessionreduce.SessionReduceRequest request = Sessionreduce.SessionReduceRequest + .newBuilder() + .setOperation(Sessionreduce.SessionReduceRequest.WindowOperation + .newBuilder() + .setEventValue(Sessionreduce.SessionReduceRequest.WindowOperation.Event.APPEND_VALUE) + .addAllKeyedWindows(List.of(Sessionreduce.KeyedWindow.newBuilder() + .addAllKeys(keys) + .setStart(Timestamp + .newBuilder().setSeconds(6000).build()) + .setEnd(Timestamp.newBuilder().setSeconds(7000).build()) + .setSlot("test-slot").build())) + .build()) + .setPayload(Sessionreduce.SessionReduceRequest.Payload.newBuilder() + .addAllKeys(keys) + .setValue(ByteString.copyFromUtf8(String.valueOf(1))) + .build()) + .build(); + + supervisorActor.tell(request, ActorRef.noSender()); + + try { + completableFuture.get(); + fail("Expected the future to complete with exception"); + } catch (Exception e) { + assertEquals(e.getMessage(), "java.lang.RuntimeException: UDF Failure"); + } + } + + @Test + public void given_shutdownActor_when_deadLetterReceived_thenFutureCompletesWithExpectedException() { + final ActorSystem actorSystem = ActorSystem.create("test-system-2"); + CompletableFuture completableFuture = new CompletableFuture<>(); + + ActorRef shutdownActor = actorSystem + .actorOf(ShutdownActor + .props(completableFuture)); + + actorSystem.eventStream().subscribe(shutdownActor, AllDeadLetters.class); + + ReduceOutputStreamObserver reduceOutputStreamObserver = new ReduceOutputStreamObserver(); + + ActorRef outputActor = actorSystem.actorOf(OutputActor + .props(reduceOutputStreamObserver)); + + ActorRef supervisorActor = actorSystem + .actorOf(SupervisorActor + .props( + new TestExceptionFactory(), + shutdownActor, + outputActor)); + + DeadLetter deadLetter = new DeadLetter("dead-letter", shutdownActor, supervisorActor); + supervisorActor.tell(deadLetter, ActorRef.noSender()); + + try { + completableFuture.get(); + fail("Expected the future to complete with exception"); + } catch (Exception e) { + assertEquals(e.getMessage(), "java.lang.Throwable: dead letters"); + } + } + + + public static class TestExceptionFactory extends SessionReducerFactory { + + @Override + public TestException createSessionReducer() { + return new TestException(); + } + + public static class TestException extends SessionReducer { + + int count = 0; + + @Override + public void processMessage( + String[] keys, + Datum datum, + OutputStreamObserver outputStreamObserver) { + count += 1; + throw new RuntimeException("UDF Failure"); + } + + @Override + public void handleEndOfStream( + String[] keys, + OutputStreamObserver outputStreamObserver) { + outputStreamObserver.send(new Message(String.valueOf(count).getBytes())); + } + + @Override + public byte[] accumulator() { + return new byte[0]; + } + + @Override + public void mergeAccumulator(byte[] accumulator) { + + } + } + } +}