From 0d87b8f7603401a646d3ee5a1204077d4207ed27 Mon Sep 17 00:00:00 2001 From: Sidhant Kohli Date: Sat, 24 May 2025 13:15:46 -0700 Subject: [PATCH 01/30] chore: release version v0.9.2 (#228) Signed-off-by: Sidhant Kohli --- pyproject.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pyproject.toml b/pyproject.toml index 309a300c..e602898d 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -1,6 +1,6 @@ [tool.poetry] name = "pynumaflow" -version = "0.9.1" +version = "0.9.2" description = "Provides the interfaces of writing Python User Defined Functions and Sinks for NumaFlow." authors = ["NumaFlow Developers"] readme = "README.md" From 3ed02e671cf6fee55e19ff53ad716a012482456a Mon Sep 17 00:00:00 2001 From: Sidhant Kohli Date: Tue, 24 Jun 2025 15:12:26 -0700 Subject: [PATCH 02/30] feat: add async source transformer (#230) --- .../async_event_time_filter/Dockerfile | 55 ++++ .../async_event_time_filter/Makefile | 22 ++ .../async_event_time_filter/entry.sh | 4 + .../async_event_time_filter/example.py | 48 ++++ .../async_event_time_filter/pyproject.toml | 15 + pynumaflow/sourcetransformer/__init__.py | 2 + pynumaflow/sourcetransformer/_dtypes.py | 7 + pynumaflow/sourcetransformer/async_server.py | 157 ++++++++++ .../servicer/_async_servicer.py | 142 +++++++++ tests/sourcetransform/test_async.py | 272 ++++++++++++++++++ 10 files changed, 724 insertions(+) create mode 100644 examples/sourcetransform/async_event_time_filter/Dockerfile create mode 100644 examples/sourcetransform/async_event_time_filter/Makefile create mode 100644 examples/sourcetransform/async_event_time_filter/entry.sh create mode 100644 examples/sourcetransform/async_event_time_filter/example.py create mode 100644 examples/sourcetransform/async_event_time_filter/pyproject.toml create mode 100644 pynumaflow/sourcetransformer/async_server.py create mode 100644 pynumaflow/sourcetransformer/servicer/_async_servicer.py create mode 100644 tests/sourcetransform/test_async.py diff --git a/examples/sourcetransform/async_event_time_filter/Dockerfile b/examples/sourcetransform/async_event_time_filter/Dockerfile new file mode 100644 index 00000000..78f24d83 --- /dev/null +++ b/examples/sourcetransform/async_event_time_filter/Dockerfile @@ -0,0 +1,55 @@ +#################################################################################################### +# builder: install needed dependencies +#################################################################################################### + +FROM python:3.10-slim-bullseye AS builder + +ENV PYTHONFAULTHANDLER=1 \ + PYTHONUNBUFFERED=1 \ + PYTHONHASHSEED=random \ + PIP_NO_CACHE_DIR=on \ + PIP_DISABLE_PIP_VERSION_CHECK=on \ + PIP_DEFAULT_TIMEOUT=100 \ + POETRY_VERSION=1.2.2 \ + POETRY_HOME="/opt/poetry" \ + POETRY_VIRTUALENVS_IN_PROJECT=true \ + POETRY_NO_INTERACTION=1 \ + PYSETUP_PATH="/opt/pysetup" + +ENV EXAMPLE_PATH="$PYSETUP_PATH/examples/sourcetransform/async_event_time_filter" +ENV VENV_PATH="$EXAMPLE_PATH/.venv" +ENV PATH="$POETRY_HOME/bin:$VENV_PATH/bin:$PATH" + +RUN apt-get update \ + && apt-get install --no-install-recommends -y \ + curl \ + wget \ + # deps for building python deps + build-essential \ + && apt-get install -y git \ + && apt-get clean && rm -rf /var/lib/apt/lists/* \ + \ + # install dumb-init + && wget -O /dumb-init https://github.com/Yelp/dumb-init/releases/download/v1.2.5/dumb-init_1.2.5_x86_64 \ + && chmod +x /dumb-init \ + && curl -sSL https://install.python-poetry.org | python3 - + +#################################################################################################### +# udf: used for running the udf vertices +#################################################################################################### +FROM builder AS udf + +WORKDIR $PYSETUP_PATH +COPY ./ ./ + +WORKDIR $EXAMPLE_PATH +RUN poetry lock +RUN poetry install --no-cache --no-root && \ + rm -rf ~/.cache/pypoetry/ + +RUN chmod +x entry.sh + +ENTRYPOINT ["/dumb-init", "--"] +CMD ["sh", "-c", "$EXAMPLE_PATH/entry.sh"] + +EXPOSE 5000 diff --git a/examples/sourcetransform/async_event_time_filter/Makefile b/examples/sourcetransform/async_event_time_filter/Makefile new file mode 100644 index 00000000..5ad2dd38 --- /dev/null +++ b/examples/sourcetransform/async_event_time_filter/Makefile @@ -0,0 +1,22 @@ +TAG ?= stable +PUSH ?= false +IMAGE_REGISTRY = quay.io/numaio/numaflow-python/async-mapt-event-time-filter:${TAG} +DOCKER_FILE_PATH = examples/sourcetransform/async_event_time_filter/Dockerfile + +.PHONY: update +update: + poetry update -vv + +.PHONY: image-push +image-push: update + cd ../../../ && docker buildx build \ + -f ${DOCKER_FILE_PATH} \ + -t ${IMAGE_REGISTRY} \ + --platform linux/amd64,linux/arm64 . --push + +.PHONY: image +image: update + cd ../../../ && docker build \ + -f ${DOCKER_FILE_PATH} \ + -t ${IMAGE_REGISTRY} . + @if [ "$(PUSH)" = "true" ]; then docker push ${IMAGE_REGISTRY}:${TAG}; fi diff --git a/examples/sourcetransform/async_event_time_filter/entry.sh b/examples/sourcetransform/async_event_time_filter/entry.sh new file mode 100644 index 00000000..073b05e3 --- /dev/null +++ b/examples/sourcetransform/async_event_time_filter/entry.sh @@ -0,0 +1,4 @@ +#!/bin/sh +set -eux + +python example.py diff --git a/examples/sourcetransform/async_event_time_filter/example.py b/examples/sourcetransform/async_event_time_filter/example.py new file mode 100644 index 00000000..659b7e24 --- /dev/null +++ b/examples/sourcetransform/async_event_time_filter/example.py @@ -0,0 +1,48 @@ +import datetime +import logging + +from pynumaflow.sourcetransformer import Messages, Message, Datum +from pynumaflow.sourcetransformer import SourceTransformAsyncServer + +""" +This is a simple User Defined Function example which receives a message, applies the following +data transformation, and returns the message. +If the message event time is before year 2022, drop the message with event time unchanged. +If it's within year 2022, update the tag to "within_year_2022" and +update the message event time to Jan 1st 2022. +Otherwise, (exclusively after year 2022), update the tag to "after_year_2022" and update the +message event time to Jan 1st 2023. +""" + +january_first_2022 = datetime.datetime.fromtimestamp(1640995200) +january_first_2023 = datetime.datetime.fromtimestamp(1672531200) + + +async def my_handler(keys: list[str], datum: Datum) -> Messages: + val = datum.value + event_time = datum.event_time + messages = Messages() + + if event_time < january_first_2022: + logging.info("Got event time:%s, it is before 2022, so dropping", event_time) + messages.append(Message.to_drop(event_time)) + elif event_time < january_first_2023: + logging.info( + "Got event time:%s, it is within year 2022, so forwarding to within_year_2022", + event_time, + ) + messages.append( + Message(value=val, event_time=january_first_2022, tags=["within_year_2022"]) + ) + else: + logging.info( + "Got event time:%s, it is after year 2022, so forwarding to after_year_2022", event_time + ) + messages.append(Message(value=val, event_time=january_first_2023, tags=["after_year_2022"])) + + return messages + + +if __name__ == "__main__": + grpc_server = SourceTransformAsyncServer(my_handler) + grpc_server.start() diff --git a/examples/sourcetransform/async_event_time_filter/pyproject.toml b/examples/sourcetransform/async_event_time_filter/pyproject.toml new file mode 100644 index 00000000..7c5bf2b5 --- /dev/null +++ b/examples/sourcetransform/async_event_time_filter/pyproject.toml @@ -0,0 +1,15 @@ +[tool.poetry] +name = "async-mapt-event-time-filter" +version = "0.2.4" +description = "" +authors = ["Numaflow developers"] +readme = "README.md" +packages = [{include = "mapt_event_time_filter"}] + +[tool.poetry.dependencies] +python = ">=3.9, <3.12" +pynumaflow = { path = "../../../"} + +[build-system] +requires = ["poetry-core"] +build-backend = "poetry.core.masonry.api" diff --git a/pynumaflow/sourcetransformer/__init__.py b/pynumaflow/sourcetransformer/__init__.py index 69f8018c..8eee3786 100644 --- a/pynumaflow/sourcetransformer/__init__.py +++ b/pynumaflow/sourcetransformer/__init__.py @@ -7,6 +7,7 @@ ) from pynumaflow.sourcetransformer.multiproc_server import SourceTransformMultiProcServer from pynumaflow.sourcetransformer.server import SourceTransformServer +from pynumaflow.sourcetransformer.async_server import SourceTransformAsyncServer __all__ = [ "Message", @@ -16,4 +17,5 @@ "SourceTransformServer", "SourceTransformer", "SourceTransformMultiProcServer", + "SourceTransformAsyncServer", ] diff --git a/pynumaflow/sourcetransformer/_dtypes.py b/pynumaflow/sourcetransformer/_dtypes.py index 13b2bce1..bc0ec7b5 100644 --- a/pynumaflow/sourcetransformer/_dtypes.py +++ b/pynumaflow/sourcetransformer/_dtypes.py @@ -3,6 +3,7 @@ from dataclasses import dataclass from datetime import datetime from typing import TypeVar, Callable, Union, Optional +from collections.abc import Awaitable from warnings import warn from pynumaflow._constants import DROP @@ -210,3 +211,9 @@ def handler(self, keys: list[str], datum: Datum) -> Messages: # SourceTransformCallable is the type of the handler function for the # Source Transformer UDFunction. SourceTransformCallable = Union[SourceTransformHandler, SourceTransformer] + + +# SourceTransformAsyncCallable is a callable which can be used as a handler +# for the Asynchronous Transformer UDF +SourceTransformHandlerAsyncHandlerCallable = Callable[[list[str], Datum], Awaitable[Messages]] +SourceTransformAsyncCallable = Union[SourceTransformer, SourceTransformHandlerAsyncHandlerCallable] diff --git a/pynumaflow/sourcetransformer/async_server.py b/pynumaflow/sourcetransformer/async_server.py new file mode 100644 index 00000000..0dc8add9 --- /dev/null +++ b/pynumaflow/sourcetransformer/async_server.py @@ -0,0 +1,157 @@ +import aiorun +import grpc + +from pynumaflow._constants import ( + NUM_THREADS_DEFAULT, + MAX_MESSAGE_SIZE, + MAX_NUM_THREADS, + SOURCE_TRANSFORMER_SOCK_PATH, + SOURCE_TRANSFORMER_SERVER_INFO_FILE_PATH, +) +from pynumaflow.info.types import ( + ServerInfo, + MINIMUM_NUMAFLOW_VERSION, + ContainerType, +) +from pynumaflow.proto.sourcetransformer import transform_pb2_grpc +from pynumaflow.shared.server import ( + NumaflowServer, + start_async_server, +) +from pynumaflow.sourcetransformer._dtypes import SourceTransformAsyncCallable +from pynumaflow.sourcetransformer.servicer._async_servicer import SourceTransformAsyncServicer + + +class SourceTransformAsyncServer(NumaflowServer): + """ + Create a new grpc Source Transformer Server instance. + A new servicer instance is created and attached to the server. + The server instance is returned. + Args: + source_transform_instance: The source transformer instance to be used for + Source Transformer UDF + sock_path: The UNIX socket path to be used for the server + max_message_size: The max message size in bytes the server can receive and send + max_threads: The max number of threads to be spawned; + defaults to 4 and max capped at 16 + + Example Invocation: + + import datetime + import logging + + from pynumaflow.sourcetransformer import Messages, Message, Datum, SourceTransformServer + # This is a simple User Defined Function example which receives a message, + # applies the following + # data transformation, and returns the message. + # If the message event time is before year 2022, drop the message with event time unchanged. + # If it's within year 2022, update the tag to "within_year_2022" and + # update the message event time to Jan 1st 2022. + # Otherwise, (exclusively after year 2022), update the tag to + # "after_year_2022" and update the + # message event time to Jan 1st 2023. + + january_first_2022 = datetime.datetime.fromtimestamp(1640995200) + january_first_2023 = datetime.datetime.fromtimestamp(1672531200) + + + async def my_handler(keys: list[str], datum: Datum) -> Messages: + val = datum.value + event_time = datum.event_time + messages = Messages() + + if event_time < january_first_2022: + logging.info("Got event time:%s, it is before 2022, so dropping", event_time) + messages.append(Message.to_drop(event_time)) + elif event_time < january_first_2023: + logging.info( + "Got event time:%s, it is within year 2022, so forwarding to within_year_2022", + event_time, + ) + messages.append( + Message(value=val, event_time=january_first_2022, + tags=["within_year_2022"]) + ) + else: + logging.info( + "Got event time:%s, it is after year 2022, so forwarding to + after_year_2022", event_time + ) + messages.append(Message(value=val, event_time=january_first_2023, + tags=["after_year_2022"])) + + return messages + + + if __name__ == "__main__": + grpc_server = SourceTransformAsyncServer(my_handler) + grpc_server.start() + """ + + def __init__( + self, + source_transform_instance: SourceTransformAsyncCallable, + sock_path=SOURCE_TRANSFORMER_SOCK_PATH, + max_message_size=MAX_MESSAGE_SIZE, + max_threads=NUM_THREADS_DEFAULT, + server_info_file=SOURCE_TRANSFORMER_SERVER_INFO_FILE_PATH, + ): + """ + Create a new grpc Asynchronous Map Server instance. + A new servicer instance is created and attached to the server. + The server instance is returned. + Args: + mapper_instance: The mapper instance to be used for Map UDF + sock_path: The UNIX socket path to be used for the server + max_message_size: The max message size in bytes the server can receive and send + max_threads: The max number of threads to be spawned; + defaults to 4 and max capped at 16 + """ + self.sock_path = f"unix://{sock_path}" + self.max_threads = min(max_threads, MAX_NUM_THREADS) + self.max_message_size = max_message_size + self.server_info_file = server_info_file + + self.source_transform_instance = source_transform_instance + + self._server_options = [ + ("grpc.max_send_message_length", self.max_message_size), + ("grpc.max_receive_message_length", self.max_message_size), + ] + self.servicer = SourceTransformAsyncServicer(handler=source_transform_instance) + + def start(self) -> None: + """ + Starter function for the Async server class, need a separate caller + so that all the async coroutines can be started from a single context + """ + aiorun.run(self.aexec(), use_uvloop=True) + + async def aexec(self) -> None: + """ + Starts the Async gRPC server on the given UNIX socket with + given max threads. + """ + + # As the server is async, we need to create a new server instance in the + # same thread as the event loop so that all the async calls are made in the + # same context + + server_new = grpc.aio.server(options=self._server_options) + server_new.add_insecure_port(self.sock_path) + transform_pb2_grpc.add_SourceTransformServicer_to_server(self.servicer, server_new) + + serv_info = ServerInfo.get_default_server_info() + serv_info.minimum_numaflow_version = MINIMUM_NUMAFLOW_VERSION[ + ContainerType.Sourcetransformer + ] + + # Start the async server + await start_async_server( + server_async=server_new, + sock_path=self.sock_path, + max_threads=self.max_threads, + cleanup_coroutines=list(), + server_info_file=self.server_info_file, + server_info=serv_info, + ) diff --git a/pynumaflow/sourcetransformer/servicer/_async_servicer.py b/pynumaflow/sourcetransformer/servicer/_async_servicer.py new file mode 100644 index 00000000..b2e70799 --- /dev/null +++ b/pynumaflow/sourcetransformer/servicer/_async_servicer.py @@ -0,0 +1,142 @@ +import asyncio +from collections.abc import AsyncIterable + +from google.protobuf import empty_pb2 as _empty_pb2 +from google.protobuf import timestamp_pb2 as _timestamp_pb2 + +from pynumaflow._constants import _LOGGER, STREAM_EOF, ERR_UDF_EXCEPTION_STRING +from pynumaflow.proto.sourcetransformer import transform_pb2, transform_pb2_grpc +from pynumaflow.shared.asynciter import NonBlockingIterator +from pynumaflow.shared.server import handle_async_error +from pynumaflow.sourcetransformer import Datum +from pynumaflow.sourcetransformer._dtypes import SourceTransformAsyncCallable +from pynumaflow.types import NumaflowServicerContext + + +class SourceTransformAsyncServicer(transform_pb2_grpc.SourceTransformServicer): + """ + This class is used to create a new grpc SourceTransform Async Servicer instance. + It implements the SourceTransformServicer interface from the proto + transform_pb2_grpc.py file. + Provides the functionality for the required rpc methods. + """ + + def __init__( + self, + handler: SourceTransformAsyncCallable, + ): + self.background_tasks = set() + self.__transform_handler: SourceTransformAsyncCallable = handler + + async def SourceTransformFn( + self, + request_iterator: AsyncIterable[transform_pb2.SourceTransformRequest], + context: NumaflowServicerContext, + ) -> AsyncIterable[transform_pb2.SourceTransformResponse]: + """ + Applies a transform function to a SourceTransformRequest stream + The pascal case function name comes from the proto transform_pb2_grpc.py file. + """ + try: + # The first message to be received should be a valid handshake + req = await request_iterator.__anext__() + # check if it is a valid handshake req + if not (req.handshake and req.handshake.sot): + raise Exception("SourceTransformFn: expected handshake message") + yield transform_pb2.SourceTransformResponse( + handshake=transform_pb2.Handshake(sot=True), + ) + + # result queue to stream messages from the user code back to the client + global_result_queue = NonBlockingIterator() + + # reader task to process the input task and invoke the required tasks + producer = asyncio.create_task( + self._process_inputs(request_iterator, global_result_queue) + ) + + # keep reading on result queue and send messages back + consumer = global_result_queue.read_iterator() + async for msg in consumer: + # If the message is an exception, we raise the exception + if isinstance(msg, BaseException): + await handle_async_error(context, msg, ERR_UDF_EXCEPTION_STRING) + return + # Send window response back to the client + else: + yield msg + # wait for the producer task to complete + await producer + except BaseException as e: + _LOGGER.critical("SourceTransformFnError, re-raising the error", exc_info=True) + await handle_async_error(context, e, ERR_UDF_EXCEPTION_STRING) + return + + async def _process_inputs( + self, + request_iterator: AsyncIterable[transform_pb2.SourceTransformRequest], + result_queue: NonBlockingIterator, + ): + """ + Utility function for processing incoming SourceTransformRequest + """ + try: + # for each incoming request, create a background task to execute the + # UDF code + async for req in request_iterator: + msg_task = asyncio.create_task(self._invoke_transform(req, result_queue)) + # save a reference to a set to store active tasks + self.background_tasks.add(msg_task) + msg_task.add_done_callback(self.background_tasks.discard) + + # Wait for all tasks to complete concurrently + await asyncio.gather(*self.background_tasks) + + # send an EOF to result queue to indicate that all tasks have completed + await result_queue.put(STREAM_EOF) + + except BaseException: + _LOGGER.critical("SourceTransformFnError Error, re-raising the error", exc_info=True) + + async def _invoke_transform( + self, request: transform_pb2.SourceTransformRequest, result_queue: NonBlockingIterator + ): + """ + Invokes the user defined function. + """ + try: + datum = Datum( + keys=list(request.request.keys), + value=request.request.value, + event_time=request.request.event_time.ToDatetime(), + watermark=request.request.watermark.ToDatetime(), + headers=dict(request.request.headers), + ) + msgs = await self.__transform_handler(list(request.request.keys), datum) + results = [] + for msg in msgs: + event_time_timestamp = _timestamp_pb2.Timestamp() + event_time_timestamp.FromDatetime(dt=msg.event_time) + results.append( + transform_pb2.SourceTransformResponse.Result( + keys=list(msg.keys), + value=msg.value, + tags=msg.tags, + event_time=event_time_timestamp, + ) + ) + await result_queue.put( + transform_pb2.SourceTransformResponse(results=results, id=request.request.id) + ) + except BaseException as err: + _LOGGER.critical("SourceTransformFnError handler error", exc_info=True) + await result_queue.put(err) + + async def IsReady( + self, request: _empty_pb2.Empty, context: NumaflowServicerContext + ) -> transform_pb2.ReadyResponse: + """ + IsReady is the heartbeat endpoint for gRPC. + The pascal case function name comes from the proto transform_pb2_grpc.py file. + """ + return transform_pb2.ReadyResponse(ready=True) diff --git a/tests/sourcetransform/test_async.py b/tests/sourcetransform/test_async.py new file mode 100644 index 00000000..05f7f29d --- /dev/null +++ b/tests/sourcetransform/test_async.py @@ -0,0 +1,272 @@ +import asyncio +import logging +import threading +import unittest +from unittest.mock import patch +from google.protobuf import timestamp_pb2 as _timestamp_pb2 + +import grpc +from google.protobuf import empty_pb2 as _empty_pb2 +from grpc.aio._server import Server + +from pynumaflow import setup_logging +from pynumaflow._constants import MAX_MESSAGE_SIZE +from pynumaflow.proto.sourcetransformer import transform_pb2_grpc +from pynumaflow.sourcetransformer import Datum, Messages, Message, SourceTransformer +from pynumaflow.sourcetransformer.async_server import SourceTransformAsyncServer +from tests.sourcetransform.utils import get_test_datums +from tests.testing_utils import ( + mock_terminate_on_stop, + mock_new_event_time, +) + +LOGGER = setup_logging(__name__) + +# if set to true, transform handler will raise a `ValueError` exception. +raise_error_from_st = False + + +class SimpleAsyncSourceTrn(SourceTransformer): + async def handler(self, keys: list[str], datum: Datum) -> Messages: + if raise_error_from_st: + raise ValueError("Exception thrown from transform") + val = datum.value + msg = "payload:{} event_time:{} ".format( + val.decode("utf-8"), + datum.event_time, + ) + val = bytes(msg, encoding="utf-8") + messages = Messages() + messages.append(Message(val, mock_new_event_time(), keys=keys)) + return messages + + +def request_generator(req): + yield from req + + +_s: Server = None +_channel = grpc.insecure_channel("unix:///tmp/async_st.sock") +_loop = None + + +def startup_callable(loop): + asyncio.set_event_loop(loop) + loop.run_forever() + + +def new_async_st(): + handle = SimpleAsyncSourceTrn() + server = SourceTransformAsyncServer(source_transform_instance=handle) + udfs = server.servicer + return udfs + + +async def start_server(udfs): + _server_options = [ + ("grpc.max_send_message_length", MAX_MESSAGE_SIZE), + ("grpc.max_receive_message_length", MAX_MESSAGE_SIZE), + ] + server = grpc.aio.server(options=_server_options) + transform_pb2_grpc.add_SourceTransformServicer_to_server(udfs, server) + listen_addr = "unix:///tmp/async_st.sock" + server.add_insecure_port(listen_addr) + logging.info("Starting server on %s", listen_addr) + global _s + _s = server + await server.start() + await server.wait_for_termination() + + +# We are mocking the terminate function from the psutil to not exit the program during testing +@patch("psutil.Process.kill", mock_terminate_on_stop) +class TestAsyncTransformer(unittest.TestCase): + @classmethod + def setUpClass(cls) -> None: + global _loop + loop = asyncio.new_event_loop() + _loop = loop + _thread = threading.Thread(target=startup_callable, args=(loop,), daemon=True) + _thread.start() + udfs = new_async_st() + asyncio.run_coroutine_threadsafe(start_server(udfs), loop=loop) + while True: + try: + with grpc.insecure_channel("unix:///tmp/async_st.sock") as channel: + f = grpc.channel_ready_future(channel) + f.result(timeout=10) + if f.done(): + break + except grpc.FutureTimeoutError as e: + LOGGER.error("error trying to connect to grpc server") + LOGGER.error(e) + + @classmethod + def tearDownClass(cls) -> None: + try: + _loop.stop() + LOGGER.info("stopped the event loop") + except Exception as e: + LOGGER.error(e) + + def test_run_server(self) -> None: + with grpc.insecure_channel("unix:///tmp/async_st.sock") as channel: + stub = transform_pb2_grpc.SourceTransformStub(channel) + request = get_test_datums() + generator_response = None + try: + generator_response = stub.SourceTransformFn( + request_iterator=request_generator(request) + ) + except grpc.RpcError as e: + logging.error(e) + + responses = [] + # capture the output from the ReadFn generator and assert. + for r in generator_response: + responses.append(r) + + # 1 handshake + 3 data responses + self.assertEqual(4, len(responses)) + + self.assertTrue(responses[0].handshake.sot) + + idx = 1 + while idx < len(responses): + _id = "test-id-" + str(idx) + self.assertEqual(_id, responses[idx].id) + self.assertEqual( + bytes( + "payload:test_mock_message " "event_time:2022-09-12 16:00:00 ", + encoding="utf-8", + ), + responses[idx].results[0].value, + ) + self.assertEqual(1, len(responses[idx].results)) + idx += 1 + + LOGGER.info("Successfully validated the server") + + def test_async_source_transformer(self) -> None: + stub = transform_pb2_grpc.SourceTransformStub(_channel) + request = get_test_datums() + generator_response = None + try: + generator_response = stub.SourceTransformFn(request_iterator=request_generator(request)) + except grpc.RpcError as e: + logging.error(e) + + responses = [] + # capture the output from the ReadFn generator and assert. + for r in generator_response: + responses.append(r) + + # 1 handshake + 3 data responses + self.assertEqual(4, len(responses)) + + self.assertTrue(responses[0].handshake.sot) + + idx = 1 + while idx < len(responses): + _id = "test-id-" + str(idx) + self.assertEqual(_id, responses[idx].id) + self.assertEqual( + bytes( + "payload:test_mock_message " "event_time:2022-09-12 16:00:00 ", + encoding="utf-8", + ), + responses[idx].results[0].value, + ) + self.assertEqual(1, len(responses[idx].results)) + idx += 1 + + # Verify new event time gets assigned. + updated_event_time_timestamp = _timestamp_pb2.Timestamp() + updated_event_time_timestamp.FromDatetime(dt=mock_new_event_time()) + self.assertEqual( + updated_event_time_timestamp, + responses[1].results[0].event_time, + ) + # self.assertEqual(code, grpc.StatusCode.OK) + + def test_async_source_transformer_grpc_error_no_handshake(self) -> None: + stub = transform_pb2_grpc.SourceTransformStub(_channel) + request = get_test_datums(handshake=False) + grpc_exception = None + + responses = [] + try: + generator_response = stub.SourceTransformFn(request_iterator=request_generator(request)) + # capture the output from the ReadFn generator and assert. + for r in generator_response: + responses.append(r) + except grpc.RpcError as e: + logging.error(e) + grpc_exception = e + self.assertTrue("SourceTransformFn: expected handshake message" in e.__str__()) + + self.assertEqual(0, len(responses)) + self.assertIsNotNone(grpc_exception) + + def test_async_source_transformer_grpc_error(self) -> None: + stub = transform_pb2_grpc.SourceTransformStub(_channel) + request = get_test_datums() + grpc_exception = None + + responses = [] + try: + global raise_error_from_st + raise_error_from_st = True + generator_response = stub.SourceTransformFn(request_iterator=request_generator(request)) + # capture the output from the ReadFn generator and assert. + for r in generator_response: + responses.append(r) + except grpc.RpcError as e: + logging.error(e) + grpc_exception = e + self.assertEqual(grpc.StatusCode.INTERNAL, e.code()) + self.assertTrue("Exception thrown from transform" in e.__str__()) + finally: + raise_error_from_st = False + # 1 handshake + self.assertEqual(1, len(responses)) + self.assertIsNotNone(grpc_exception) + + def test_is_ready(self) -> None: + with grpc.insecure_channel("unix:///tmp/async_st.sock") as channel: + stub = transform_pb2_grpc.SourceTransformStub(channel) + + request = _empty_pb2.Empty() + response = None + try: + response = stub.IsReady(request=request) + except grpc.RpcError as e: + logging.error(e) + + self.assertTrue(response.ready) + + def test_invalid_input(self): + with self.assertRaises(TypeError): + SourceTransformAsyncServer() + + def __stub(self): + return transform_pb2_grpc.SourceTransformStub(_channel) + + def test_max_threads(self): + handle = SimpleAsyncSourceTrn() + # max cap at 16 + server = SourceTransformAsyncServer(source_transform_instance=handle, max_threads=32) + self.assertEqual(server.max_threads, 16) + + # use argument provided + server = SourceTransformAsyncServer(source_transform_instance=handle, max_threads=5) + self.assertEqual(server.max_threads, 5) + + # defaults to 4 + server = SourceTransformAsyncServer(source_transform_instance=handle) + self.assertEqual(server.max_threads, 4) + + +if __name__ == "__main__": + logging.basicConfig(level=logging.DEBUG) + unittest.main() From 42f9fbde13e6c23eaee1b6bc01d6bc6487bf5c7f Mon Sep 17 00:00:00 2001 From: Sidhant Kohli Date: Wed, 25 Jun 2025 13:59:52 -0700 Subject: [PATCH 03/30] chore: prerelease for 0.10.0a0 (#231) Signed-off-by: Sidhant Kohli --- pyproject.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pyproject.toml b/pyproject.toml index e602898d..74e1c586 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -1,6 +1,6 @@ [tool.poetry] name = "pynumaflow" -version = "0.9.2" +version = "0.10.0a0" description = "Provides the interfaces of writing Python User Defined Functions and Sinks for NumaFlow." authors = ["NumaFlow Developers"] readme = "README.md" From 9a390a54e4e320ba958d80e68fc0b95224deee45 Mon Sep 17 00:00:00 2001 From: Aayush Sapkota Date: Tue, 8 Jul 2025 13:49:27 -0400 Subject: [PATCH 04/30] chore: optimize example docker files using multi-stage builds (#232) Signed-off-by: sapkota-aayush --- docs/DOCKER_OPTIMIZATION.md | 229 ++++++++++++++++++ examples/batchmap/flatmap/Dockerfile | 72 +++--- examples/map/even_odd/Dockerfile | 73 +++--- examples/map/even_odd/Makefile.optimized | 52 ++++ examples/map/flatmap/Dockerfile | 72 +++--- examples/map/forward_message/Dockerfile | 72 +++--- examples/map/multiproc_map/Dockerfile | 72 +++--- examples/mapstream/flatmap_stream/Dockerfile | 72 +++--- examples/reduce/asyncio_reduce/Dockerfile | 79 +++--- examples/reduce/asyncio_reduce/pyproject.toml | 2 +- examples/reduce/batchmap/flatmap/Dockerfile | 55 +++++ examples/reduce/counter/Dockerfile | 72 +++--- examples/reducestream/counter/Dockerfile | 72 +++--- examples/reducestream/sum/Dockerfile | 72 +++--- .../sideinput/simple_sideinput/Dockerfile | 72 +++--- .../sideinput/simple_sideinput/udf/Dockerfile | 72 +++--- examples/sink/async_log/Dockerfile | 72 +++--- examples/sink/log/Dockerfile | 72 +++--- examples/source/simple_source/Dockerfile | 72 +++--- .../async_event_time_filter/Dockerfile | 72 +++--- .../event_time_filter/Dockerfile | 72 +++--- poetry.lock | 24 +- pynumaflow/proto/mapper/map_pb2.pyi | 3 + pynumaflow/proto/reducer/reduce_pb2.pyi | 3 + pynumaflow/proto/sinker/sink_pb2.pyi | 3 + pynumaflow/proto/sourcer/source_pb2.pyi | 7 + .../proto/sourcetransformer/transform_pb2.pyi | 3 + pyproject.toml | 2 - 28 files changed, 985 insertions(+), 630 deletions(-) create mode 100644 docs/DOCKER_OPTIMIZATION.md create mode 100644 examples/map/even_odd/Makefile.optimized create mode 100644 examples/reduce/batchmap/flatmap/Dockerfile diff --git a/docs/DOCKER_OPTIMIZATION.md b/docs/DOCKER_OPTIMIZATION.md new file mode 100644 index 00000000..cc36410f --- /dev/null +++ b/docs/DOCKER_OPTIMIZATION.md @@ -0,0 +1,229 @@ +# Docker Build Optimization for NumaFlow Python UDFs + +## Overview + +This document outlines the optimization strategies to reduce Docker build times for NumaFlow Python UDFs from 2+ minutes to under 30 seconds for subsequent builds. + +## Current Issues + +1. **Redundant dependency installation**: Each UDF rebuilds the entire pynumaflow package +2. **No layer caching**: Dependencies are reinstalled every time +3. **Copying entire project**: The `COPY ./ ./` copies everything, including unnecessary files +4. **No shared base layers**: Each UDF builds its own base environment + +## Optimization Strategy: Three-Stage Approach + +As suggested by @kohlisid, we implement a three-stage build approach: + +### Stage 1: Base Layer +- Common Python environment and tools +- System dependencies (curl, wget, build-essential, git) +- Poetry installation +- dumb-init binary + +### Stage 2: Environment Setup +- pynumaflow package installation +- Shared virtual environment creation +- This layer is cached unless `pyproject.toml` or `poetry.lock` changes + +### Stage 3: Builder +- UDF-specific code and dependencies +- Reuses the pynumaflow installation from Stage 2 +- Minimal additional dependencies + +## Implementation Options + +### Option 1: Optimized Multi-Stage Build (Recommended) + +**File**: `examples/map/even_odd/Dockerfile.optimized` + +**Benefits**: +- Better layer caching +- Reduced build time by ~60-70% +- No external dependencies + +**Usage**: +```bash +cd examples/map/even_odd +make -f Makefile.optimized image +``` + +### Option 2: Shared Base Image (Fastest) + +**Files**: +- `Dockerfile.base` (shared base image) +- `examples/map/even_odd/Dockerfile.shared-base` (UDF-specific) + +**Benefits**: +- Maximum caching efficiency +- Build time reduced by ~80-90% for subsequent builds +- Perfect for CI/CD pipelines + +**Usage**: +```bash +# Build base image once +docker build -f Dockerfile.base -t numaflow-python-base . + +# Build UDF images (very fast) +cd examples/map/even_odd +make -f Makefile.optimized image-fast +``` + +## Performance Comparison + +| Approach | First Build | Subsequent Builds | Cache Efficiency | +|----------|-------------|-------------------|------------------| +| Current | ~2-3 minutes | ~2-3 minutes | Poor | +| Optimized Multi-Stage | ~2-3 minutes | ~45-60 seconds | Good | +| Shared Base Image | ~2-3 minutes | ~15-30 seconds | Excellent | + +## Implementation Steps + +### 1. Build Shared Base Image (One-time setup) + +```bash +# From project root +docker build -f Dockerfile.base -t numaflow-python-base . +``` + +### 2. Update UDF Dockerfiles + +Replace the current Dockerfile with the optimized version: + +```bash +# For each UDF directory +cp Dockerfile.optimized Dockerfile +# or +cp Dockerfile.shared-base Dockerfile +``` + +### 3. Update Makefiles + +Use the optimized Makefile: + +```bash +# For each UDF directory +cp Makefile.optimized Makefile +``` + +### 4. CI/CD Integration + +For CI/CD pipelines, add the base image build step: + +```yaml +# Example GitHub Actions step +- name: Build base image + run: docker build -f Dockerfile.base -t numaflow-python-base . + +- name: Build UDF images + run: | + cd examples/map/even_odd + make image-fast +``` + +## Advanced Optimizations + +### 1. Dependency Caching + +The optimized Dockerfiles implement smart dependency caching: +- `pyproject.toml` and `poetry.lock` are copied first +- pynumaflow installation is cached separately +- UDF-specific dependencies are installed last + +### 2. Layer Optimization + +- Minimal system dependencies in runtime image +- Separate build and runtime stages +- Efficient file copying with specific paths + +### 3. Build Context Optimization + +- Copy only necessary files +- Use `.dockerignore` to exclude unnecessary files +- Minimize build context size + +## Migration Guide + +### For Existing UDFs + +1. **Backup current Dockerfile**: + ```bash + cp Dockerfile Dockerfile.backup + ``` + +2. **Choose optimization approach**: + - For single UDF: Use `Dockerfile.optimized` + - For multiple UDFs: Use `Dockerfile.shared-base` + +3. **Update Makefile**: + ```bash + cp Makefile.optimized Makefile + ``` + +4. **Test the build**: + ```bash + make image + # or + make image-fast + ``` + +### For New UDFs + +1. **Use the optimized template**: + ```bash + cp examples/map/even_odd/Dockerfile.optimized your-udf/Dockerfile + cp examples/map/even_odd/Makefile.optimized your-udf/Makefile + ``` + +2. **Update paths in Dockerfile**: + - Change `EXAMPLE_PATH` to your UDF path + - Update `COPY` commands accordingly + +## Troubleshooting + +### Common Issues + +1. **Base image not found**: + ```bash + docker build -f Dockerfile.base -t numaflow-python-base . + ``` + +2. **Permission issues**: + ```bash + chmod +x entry.sh + ``` + +3. **Poetry cache issues**: + ```bash + poetry cache clear --all pypi + ``` + +### Performance Monitoring + +Monitor build times: +```bash +time make image +time make image-fast +``` + +## Future Enhancements + +1. **Registry-based base images**: Push base image to registry for team sharing +2. **BuildKit optimizations**: Enable BuildKit for parallel layer building +3. **Multi-platform builds**: Optimize for ARM64 and AMD64 +4. **Dependency analysis**: Automate dependency optimization + +## Contributing + +When adding new UDFs or modifying existing ones: + +1. Use the optimized Dockerfile templates +2. Follow the three-stage approach +3. Test build times before and after changes +4. Update this documentation if needed + +## References + +- [Docker Multi-Stage Builds](https://docs.docker.com/develop/dev-best-practices/multistage-build/) +- [Docker Layer Caching](https://docs.docker.com/develop/dev-best-practices/dockerfile_best-practices/#leverage-build-cache) +- [Poetry Docker Best Practices](https://python-poetry.org/docs/configuration/#virtualenvsin-project) \ No newline at end of file diff --git a/examples/batchmap/flatmap/Dockerfile b/examples/batchmap/flatmap/Dockerfile index 20f1a820..99319c4a 100644 --- a/examples/batchmap/flatmap/Dockerfile +++ b/examples/batchmap/flatmap/Dockerfile @@ -1,52 +1,52 @@ #################################################################################################### -# builder: install needed dependencies +# Stage 1: Base Builder - installs core dependencies using poetry #################################################################################################### +FROM python:3.10-slim-bullseye AS base-builder -FROM python:3.10-slim-bullseye AS builder +ENV PYSETUP_PATH="/opt/pysetup" +WORKDIR $PYSETUP_PATH + +# Copy only core dependency files first for better caching +COPY pyproject.toml poetry.lock README.md ./ +COPY pynumaflow/ ./pynumaflow/ +RUN apt-get update && apt-get install --no-install-recommends -y \ + curl wget build-essential git \ + && apt-get clean && rm -rf /var/lib/apt/lists/* \ + && pip install poetry \ + && poetry install --no-root --no-interaction + +#################################################################################################### +# Stage 2: UDF Builder - adds UDF code and installs UDF-specific deps +#################################################################################################### +FROM base-builder AS udf-builder + +ENV EXAMPLE_PATH="/opt/pysetup/examples/batchmap/flatmap" +ENV POETRY_VIRTUALENVS_IN_PROJECT=true + +WORKDIR $EXAMPLE_PATH +COPY examples/batchmap/flatmap/ ./ +RUN poetry install --no-root --no-interaction -ENV PYTHONFAULTHANDLER=1 \ - PYTHONUNBUFFERED=1 \ - PYTHONHASHSEED=random \ - PIP_NO_CACHE_DIR=on \ - PIP_DISABLE_PIP_VERSION_CHECK=on \ - PIP_DEFAULT_TIMEOUT=100 \ - POETRY_VERSION=1.2.2 \ - POETRY_HOME="/opt/poetry" \ - POETRY_VIRTUALENVS_IN_PROJECT=true \ - POETRY_NO_INTERACTION=1 \ - PYSETUP_PATH="/opt/pysetup" +#################################################################################################### +# Stage 3: UDF Runtime - clean container with only needed stuff +#################################################################################################### +FROM python:3.10-slim-bullseye AS udf +ENV PYSETUP_PATH="/opt/pysetup" ENV EXAMPLE_PATH="$PYSETUP_PATH/examples/batchmap/flatmap" ENV VENV_PATH="$EXAMPLE_PATH/.venv" -ENV PATH="$POETRY_HOME/bin:$VENV_PATH/bin:$PATH" - -RUN apt-get update \ - && apt-get install --no-install-recommends -y \ - curl \ - wget \ - # deps for building python deps - build-essential \ - && apt-get install -y git \ +ENV PATH="$VENV_PATH/bin:$PATH" + +RUN apt-get update && apt-get install --no-install-recommends -y wget \ && apt-get clean && rm -rf /var/lib/apt/lists/* \ - \ - # install dumb-init && wget -O /dumb-init https://github.com/Yelp/dumb-init/releases/download/v1.2.5/dumb-init_1.2.5_x86_64 \ - && chmod +x /dumb-init \ - && curl -sSL https://install.python-poetry.org | python3 - - -#################################################################################################### -# udf: used for running the udf vertices -#################################################################################################### -FROM builder AS udf + && chmod +x /dumb-init WORKDIR $PYSETUP_PATH -COPY ./ ./ +COPY --from=udf-builder $VENV_PATH $VENV_PATH +COPY --from=udf-builder $EXAMPLE_PATH $EXAMPLE_PATH WORKDIR $EXAMPLE_PATH -RUN poetry lock -RUN poetry install --no-cache --no-root && \ - rm -rf ~/.cache/pypoetry/ - RUN chmod +x entry.sh ENTRYPOINT ["/dumb-init", "--"] diff --git a/examples/map/even_odd/Dockerfile b/examples/map/even_odd/Dockerfile index a2da2f81..1bf155ca 100644 --- a/examples/map/even_odd/Dockerfile +++ b/examples/map/even_odd/Dockerfile @@ -1,52 +1,53 @@ #################################################################################################### -# builder: install needed dependencies +# Stage 1: Base Builder - installs core dependencies using poetry #################################################################################################### +FROM python:3.10-slim-bullseye AS base-builder -FROM python:3.10-slim-bullseye AS builder +ENV PYSETUP_PATH="/opt/pysetup" +WORKDIR $PYSETUP_PATH + +# Copy only core dependency files first for better caching +COPY pyproject.toml poetry.lock README.md ./ +COPY pynumaflow/ ./pynumaflow/ +RUN echo "Simulating long build step..." && sleep 20 +RUN apt-get update && apt-get install --no-install-recommends -y \ + curl wget build-essential git \ + && apt-get clean && rm -rf /var/lib/apt/lists/* \ + && pip install poetry \ + && poetry install --no-root --no-interaction + +#################################################################################################### +# Stage 2: UDF Builder - adds UDF code and installs UDF-specific deps +#################################################################################################### +FROM base-builder AS udf-builder + +ENV EXAMPLE_PATH="/opt/pysetup/examples/map/even_odd" +ENV POETRY_VIRTUALENVS_IN_PROJECT=true + +WORKDIR $EXAMPLE_PATH +COPY examples/map/even_odd/ ./ +RUN poetry install --no-root --no-interaction -ENV PYTHONFAULTHANDLER=1 \ - PYTHONUNBUFFERED=1 \ - PYTHONHASHSEED=random \ - PIP_NO_CACHE_DIR=on \ - PIP_DISABLE_PIP_VERSION_CHECK=on \ - PIP_DEFAULT_TIMEOUT=100 \ - POETRY_VERSION=1.2.2 \ - POETRY_HOME="/opt/poetry" \ - POETRY_VIRTUALENVS_IN_PROJECT=true \ - POETRY_NO_INTERACTION=1 \ - PYSETUP_PATH="/opt/pysetup" +#################################################################################################### +# Stage 3: UDF Runtime - clean container with only needed stuff +#################################################################################################### +FROM python:3.10-slim-bullseye AS udf +ENV PYSETUP_PATH="/opt/pysetup" ENV EXAMPLE_PATH="$PYSETUP_PATH/examples/map/even_odd" ENV VENV_PATH="$EXAMPLE_PATH/.venv" -ENV PATH="$POETRY_HOME/bin:$VENV_PATH/bin:$PATH" - -RUN apt-get update \ - && apt-get install --no-install-recommends -y \ - curl \ - wget \ - # deps for building python deps - build-essential \ - && apt-get install -y git \ +ENV PATH="$VENV_PATH/bin:$PATH" + +RUN apt-get update && apt-get install --no-install-recommends -y wget \ && apt-get clean && rm -rf /var/lib/apt/lists/* \ - \ - # install dumb-init && wget -O /dumb-init https://github.com/Yelp/dumb-init/releases/download/v1.2.5/dumb-init_1.2.5_x86_64 \ - && chmod +x /dumb-init \ - && curl -sSL https://install.python-poetry.org | python3 - - -#################################################################################################### -# udf: used for running the udf vertices -#################################################################################################### -FROM builder AS udf + && chmod +x /dumb-init WORKDIR $PYSETUP_PATH -COPY ./ ./ +COPY --from=udf-builder $VENV_PATH $VENV_PATH +COPY --from=udf-builder $EXAMPLE_PATH $EXAMPLE_PATH WORKDIR $EXAMPLE_PATH -RUN poetry lock -RUN poetry install --no-cache --no-root && \ - rm -rf ~/.cache/pypoetry/ - RUN chmod +x entry.sh ENTRYPOINT ["/dumb-init", "--"] diff --git a/examples/map/even_odd/Makefile.optimized b/examples/map/even_odd/Makefile.optimized new file mode 100644 index 00000000..3e33cc03 --- /dev/null +++ b/examples/map/even_odd/Makefile.optimized @@ -0,0 +1,52 @@ +TAG ?= stable +PUSH ?= false +IMAGE_REGISTRY = quay.io/numaio/numaflow-python/even-odd:${TAG} +DOCKER_FILE_PATH = examples/map/even_odd/Dockerfile.optimized +BASE_IMAGE_NAME = numaflow-python-base + +.PHONY: base-image +base-image: + @echo "Building shared base image..." + docker build -f Dockerfile.base -t ${BASE_IMAGE_NAME} . + +.PHONY: update +update: + poetry update -vv + +.PHONY: image-push +image-push: base-image update + cd ../../../ && docker buildx build \ + -f ${DOCKER_FILE_PATH} \ + -t ${IMAGE_REGISTRY} \ + --platform linux/amd64,linux/arm64 . --push + +.PHONY: image +image: base-image update + cd ../../../ && docker build \ + -f ${DOCKER_FILE_PATH} \ + -t ${IMAGE_REGISTRY} . + @if [ "$(PUSH)" = "true" ]; then docker push ${IMAGE_REGISTRY}; fi + +.PHONY: image-fast +image-fast: update + @echo "Building with shared base image (fastest option)..." + cd ../../../ && docker build \ + -f examples/map/even_odd/Dockerfile.shared-base \ + -t ${IMAGE_REGISTRY} . + @if [ "$(PUSH)" = "true" ]; then docker push ${IMAGE_REGISTRY}; fi + +.PHONY: clean +clean: + docker rmi ${BASE_IMAGE_NAME} 2>/dev/null || true + docker rmi ${IMAGE_REGISTRY} 2>/dev/null || true + +.PHONY: help +help: + @echo "Available targets:" + @echo " base-image - Build the shared base image with pynumaflow" + @echo " image - Build UDF image with optimized multi-stage build" + @echo " image-fast - Build UDF image using shared base (fastest)" + @echo " image-push - Build and push multi-platform image" + @echo " update - Update poetry dependencies" + @echo " clean - Remove built images" + @echo " help - Show this help message" \ No newline at end of file diff --git a/examples/map/flatmap/Dockerfile b/examples/map/flatmap/Dockerfile index d2ce662f..22d744c0 100644 --- a/examples/map/flatmap/Dockerfile +++ b/examples/map/flatmap/Dockerfile @@ -1,52 +1,52 @@ #################################################################################################### -# builder: install needed dependencies +# Stage 1: Base Builder - installs core dependencies using poetry #################################################################################################### +FROM python:3.10-slim-bullseye AS base-builder -FROM python:3.10-slim-bullseye AS builder +ENV PYSETUP_PATH="/opt/pysetup" +WORKDIR $PYSETUP_PATH + +# Copy only core dependency files first for better caching +COPY pyproject.toml poetry.lock README.md ./ +COPY pynumaflow/ ./pynumaflow/ +RUN apt-get update && apt-get install --no-install-recommends -y \ + curl wget build-essential git \ + && apt-get clean && rm -rf /var/lib/apt/lists/* \ + && pip install poetry \ + && poetry install --no-root --no-interaction + +#################################################################################################### +# Stage 2: UDF Builder - adds UDF code and installs UDF-specific deps +#################################################################################################### +FROM base-builder AS udf-builder + +ENV EXAMPLE_PATH="/opt/pysetup/examples/map/flatmap" +ENV POETRY_VIRTUALENVS_IN_PROJECT=true + +WORKDIR $EXAMPLE_PATH +COPY examples/map/flatmap/ ./ +RUN poetry install --no-root --no-interaction -ENV PYTHONFAULTHANDLER=1 \ - PYTHONUNBUFFERED=1 \ - PYTHONHASHSEED=random \ - PIP_NO_CACHE_DIR=on \ - PIP_DISABLE_PIP_VERSION_CHECK=on \ - PIP_DEFAULT_TIMEOUT=100 \ - POETRY_VERSION=1.2.2 \ - POETRY_HOME="/opt/poetry" \ - POETRY_VIRTUALENVS_IN_PROJECT=true \ - POETRY_NO_INTERACTION=1 \ - PYSETUP_PATH="/opt/pysetup" +#################################################################################################### +# Stage 3: UDF Runtime - clean container with only needed stuff +#################################################################################################### +FROM python:3.10-slim-bullseye AS udf +ENV PYSETUP_PATH="/opt/pysetup" ENV EXAMPLE_PATH="$PYSETUP_PATH/examples/map/flatmap" ENV VENV_PATH="$EXAMPLE_PATH/.venv" -ENV PATH="$POETRY_HOME/bin:$VENV_PATH/bin:$PATH" - -RUN apt-get update \ - && apt-get install --no-install-recommends -y \ - curl \ - wget \ - # deps for building python deps - build-essential \ - && apt-get install -y git \ +ENV PATH="$VENV_PATH/bin:$PATH" + +RUN apt-get update && apt-get install --no-install-recommends -y wget \ && apt-get clean && rm -rf /var/lib/apt/lists/* \ - \ - # install dumb-init && wget -O /dumb-init https://github.com/Yelp/dumb-init/releases/download/v1.2.5/dumb-init_1.2.5_x86_64 \ - && chmod +x /dumb-init \ - && curl -sSL https://install.python-poetry.org | python3 - - -#################################################################################################### -# udf: used for running the udf vertices -#################################################################################################### -FROM builder AS udf + && chmod +x /dumb-init WORKDIR $PYSETUP_PATH -COPY ./ ./ +COPY --from=udf-builder $VENV_PATH $VENV_PATH +COPY --from=udf-builder $EXAMPLE_PATH $EXAMPLE_PATH WORKDIR $EXAMPLE_PATH -RUN poetry lock -RUN poetry install --no-cache --no-root && \ - rm -rf ~/.cache/pypoetry/ - RUN chmod +x entry.sh ENTRYPOINT ["/dumb-init", "--"] diff --git a/examples/map/forward_message/Dockerfile b/examples/map/forward_message/Dockerfile index 84b4bdff..464fc1fc 100644 --- a/examples/map/forward_message/Dockerfile +++ b/examples/map/forward_message/Dockerfile @@ -1,52 +1,52 @@ #################################################################################################### -# builder: install needed dependencies +# Stage 1: Base Builder - installs core dependencies using poetry #################################################################################################### +FROM python:3.10-slim-bullseye AS base-builder -FROM python:3.10-slim-bullseye AS builder +ENV PYSETUP_PATH="/opt/pysetup" +WORKDIR $PYSETUP_PATH + +# Copy only core dependency files first for better caching +COPY pyproject.toml poetry.lock README.md ./ +COPY pynumaflow/ ./pynumaflow/ +RUN apt-get update && apt-get install --no-install-recommends -y \ + curl wget build-essential git \ + && apt-get clean && rm -rf /var/lib/apt/lists/* \ + && pip install poetry \ + && poetry install --no-root --no-interaction + +#################################################################################################### +# Stage 2: UDF Builder - adds UDF code and installs UDF-specific deps +#################################################################################################### +FROM base-builder AS udf-builder + +ENV EXAMPLE_PATH="/opt/pysetup/examples/map/forward_message" +ENV POETRY_VIRTUALENVS_IN_PROJECT=true + +WORKDIR $EXAMPLE_PATH +COPY examples/map/forward_message/ ./ +RUN poetry install --no-root --no-interaction -ENV PYTHONFAULTHANDLER=1 \ - PYTHONUNBUFFERED=1 \ - PYTHONHASHSEED=random \ - PIP_NO_CACHE_DIR=on \ - PIP_DISABLE_PIP_VERSION_CHECK=on \ - PIP_DEFAULT_TIMEOUT=100 \ - POETRY_VERSION=1.2.2 \ - POETRY_HOME="/opt/poetry" \ - POETRY_VIRTUALENVS_IN_PROJECT=true \ - POETRY_NO_INTERACTION=1 \ - PYSETUP_PATH="/opt/pysetup" +#################################################################################################### +# Stage 3: UDF Runtime - clean container with only needed stuff +#################################################################################################### +FROM python:3.10-slim-bullseye AS udf +ENV PYSETUP_PATH="/opt/pysetup" ENV EXAMPLE_PATH="$PYSETUP_PATH/examples/map/forward_message" ENV VENV_PATH="$EXAMPLE_PATH/.venv" -ENV PATH="$POETRY_HOME/bin:$VENV_PATH/bin:$PATH" - -RUN apt-get update \ - && apt-get install --no-install-recommends -y \ - curl \ - wget \ - # deps for building python deps - build-essential \ - && apt-get install -y git \ +ENV PATH="$VENV_PATH/bin:$PATH" + +RUN apt-get update && apt-get install --no-install-recommends -y wget \ && apt-get clean && rm -rf /var/lib/apt/lists/* \ - \ - # install dumb-init && wget -O /dumb-init https://github.com/Yelp/dumb-init/releases/download/v1.2.5/dumb-init_1.2.5_x86_64 \ - && chmod +x /dumb-init \ - && curl -sSL https://install.python-poetry.org | python3 - - -#################################################################################################### -# udf: used for running the udf vertices -#################################################################################################### -FROM builder AS udf + && chmod +x /dumb-init WORKDIR $PYSETUP_PATH -COPY ./ ./ +COPY --from=udf-builder $VENV_PATH $VENV_PATH +COPY --from=udf-builder $EXAMPLE_PATH $EXAMPLE_PATH WORKDIR $EXAMPLE_PATH -RUN poetry lock -RUN poetry install --no-cache --no-root && \ - rm -rf ~/.cache/pypoetry/ - RUN chmod +x entry.sh ENTRYPOINT ["/dumb-init", "--"] diff --git a/examples/map/multiproc_map/Dockerfile b/examples/map/multiproc_map/Dockerfile index 0928c03a..3c6e8205 100644 --- a/examples/map/multiproc_map/Dockerfile +++ b/examples/map/multiproc_map/Dockerfile @@ -1,52 +1,52 @@ #################################################################################################### -# builder: install needed dependencies +# Stage 1: Base Builder - installs core dependencies using poetry #################################################################################################### +FROM python:3.10-slim-bullseye AS base-builder -FROM python:3.10-slim-bullseye AS builder +ENV PYSETUP_PATH="/opt/pysetup" +WORKDIR $PYSETUP_PATH + +# Copy only core dependency files first for better caching +COPY pyproject.toml poetry.lock README.md ./ +COPY pynumaflow/ ./pynumaflow/ +RUN apt-get update && apt-get install --no-install-recommends -y \ + curl wget build-essential git \ + && apt-get clean && rm -rf /var/lib/apt/lists/* \ + && pip install poetry \ + && poetry install --no-root --no-interaction + +#################################################################################################### +# Stage 2: UDF Builder - adds UDF code and installs UDF-specific deps +#################################################################################################### +FROM base-builder AS udf-builder + +ENV EXAMPLE_PATH="/opt/pysetup/examples/map/multiproc_map" +ENV POETRY_VIRTUALENVS_IN_PROJECT=true + +WORKDIR $EXAMPLE_PATH +COPY examples/map/multiproc_map/ ./ +RUN poetry install --no-root --no-interaction -ENV PYTHONFAULTHANDLER=1 \ - PYTHONUNBUFFERED=1 \ - PYTHONHASHSEED=random \ - PIP_NO_CACHE_DIR=on \ - PIP_DISABLE_PIP_VERSION_CHECK=on \ - PIP_DEFAULT_TIMEOUT=100 \ - POETRY_VERSION=1.2.2 \ - POETRY_HOME="/opt/poetry" \ - POETRY_VIRTUALENVS_IN_PROJECT=true \ - POETRY_NO_INTERACTION=1 \ - PYSETUP_PATH="/opt/pysetup" +#################################################################################################### +# Stage 3: UDF Runtime - clean container with only needed stuff +#################################################################################################### +FROM python:3.10-slim-bullseye AS udf +ENV PYSETUP_PATH="/opt/pysetup" ENV EXAMPLE_PATH="$PYSETUP_PATH/examples/map/multiproc_map" ENV VENV_PATH="$EXAMPLE_PATH/.venv" -ENV PATH="$POETRY_HOME/bin:$VENV_PATH/bin:$PATH" - -RUN apt-get update \ - && apt-get install --no-install-recommends -y \ - curl \ - wget \ - # deps for building python deps - build-essential \ - && apt-get install -y git \ +ENV PATH="$VENV_PATH/bin:$PATH" + +RUN apt-get update && apt-get install --no-install-recommends -y wget \ && apt-get clean && rm -rf /var/lib/apt/lists/* \ - \ - # install dumb-init && wget -O /dumb-init https://github.com/Yelp/dumb-init/releases/download/v1.2.5/dumb-init_1.2.5_x86_64 \ - && chmod +x /dumb-init \ - && curl -sSL https://install.python-poetry.org | python3 - - -#################################################################################################### -# udf: used for running the udf vertices -#################################################################################################### -FROM builder AS udf + && chmod +x /dumb-init WORKDIR $PYSETUP_PATH -COPY ./ ./ +COPY --from=udf-builder $VENV_PATH $VENV_PATH +COPY --from=udf-builder $EXAMPLE_PATH $EXAMPLE_PATH WORKDIR $EXAMPLE_PATH -RUN poetry lock -RUN poetry install --no-cache --no-root && \ - rm -rf ~/.cache/pypoetry/ - RUN chmod +x entry.sh ENTRYPOINT ["/dumb-init", "--"] diff --git a/examples/mapstream/flatmap_stream/Dockerfile b/examples/mapstream/flatmap_stream/Dockerfile index a7397526..e56d7fb5 100644 --- a/examples/mapstream/flatmap_stream/Dockerfile +++ b/examples/mapstream/flatmap_stream/Dockerfile @@ -1,52 +1,52 @@ #################################################################################################### -# builder: install needed dependencies +# Stage 1: Base Builder - installs core dependencies using poetry #################################################################################################### +FROM python:3.10-slim-bullseye AS base-builder -FROM python:3.10-slim-bullseye AS builder +ENV PYSETUP_PATH="/opt/pysetup" +WORKDIR $PYSETUP_PATH + +# Copy only core dependency files first for better caching +COPY pyproject.toml poetry.lock README.md ./ +COPY pynumaflow/ ./pynumaflow/ +RUN apt-get update && apt-get install --no-install-recommends -y \ + curl wget build-essential git \ + && apt-get clean && rm -rf /var/lib/apt/lists/* \ + && pip install poetry \ + && poetry install --no-root --no-interaction + +#################################################################################################### +# Stage 2: UDF Builder - adds UDF code and installs UDF-specific deps +#################################################################################################### +FROM base-builder AS udf-builder + +ENV EXAMPLE_PATH="/opt/pysetup/examples/mapstream/flatmap_stream" +ENV POETRY_VIRTUALENVS_IN_PROJECT=true + +WORKDIR $EXAMPLE_PATH +COPY examples/mapstream/flatmap_stream/ ./ +RUN poetry install --no-root --no-interaction -ENV PYTHONFAULTHANDLER=1 \ - PYTHONUNBUFFERED=1 \ - PYTHONHASHSEED=random \ - PIP_NO_CACHE_DIR=on \ - PIP_DISABLE_PIP_VERSION_CHECK=on \ - PIP_DEFAULT_TIMEOUT=100 \ - POETRY_VERSION=1.2.2 \ - POETRY_HOME="/opt/poetry" \ - POETRY_VIRTUALENVS_IN_PROJECT=true \ - POETRY_NO_INTERACTION=1 \ - PYSETUP_PATH="/opt/pysetup" +#################################################################################################### +# Stage 3: UDF Runtime - clean container with only needed stuff +#################################################################################################### +FROM python:3.10-slim-bullseye AS udf +ENV PYSETUP_PATH="/opt/pysetup" ENV EXAMPLE_PATH="$PYSETUP_PATH/examples/mapstream/flatmap_stream" ENV VENV_PATH="$EXAMPLE_PATH/.venv" -ENV PATH="$POETRY_HOME/bin:$VENV_PATH/bin:$PATH" - -RUN apt-get update \ - && apt-get install --no-install-recommends -y \ - curl \ - wget \ - # deps for building python deps - build-essential \ - && apt-get install -y git \ +ENV PATH="$VENV_PATH/bin:$PATH" + +RUN apt-get update && apt-get install --no-install-recommends -y wget \ && apt-get clean && rm -rf /var/lib/apt/lists/* \ - \ - # install dumb-init && wget -O /dumb-init https://github.com/Yelp/dumb-init/releases/download/v1.2.5/dumb-init_1.2.5_x86_64 \ - && chmod +x /dumb-init \ - && curl -sSL https://install.python-poetry.org | python3 - - -#################################################################################################### -# udf: used for running the udf vertices -#################################################################################################### -FROM builder AS udf + && chmod +x /dumb-init WORKDIR $PYSETUP_PATH -COPY ./ ./ +COPY --from=udf-builder $VENV_PATH $VENV_PATH +COPY --from=udf-builder $EXAMPLE_PATH $EXAMPLE_PATH WORKDIR $EXAMPLE_PATH -RUN poetry lock -RUN poetry install --no-cache --no-root && \ - rm -rf ~/.cache/pypoetry/ - RUN chmod +x entry.sh ENTRYPOINT ["/dumb-init", "--"] diff --git a/examples/reduce/asyncio_reduce/Dockerfile b/examples/reduce/asyncio_reduce/Dockerfile index 32cb8500..e74b6036 100644 --- a/examples/reduce/asyncio_reduce/Dockerfile +++ b/examples/reduce/asyncio_reduce/Dockerfile @@ -1,54 +1,55 @@ #################################################################################################### -# builder: install needed dependencies +# Stage 1: Base Builder - installs core dependencies using poetry #################################################################################################### +FROM python:3.10-slim-bullseye AS base-builder -FROM python:3.10-slim-bullseye AS builder - -ENV PYTHONFAULTHANDLER=1 \ - PYTHONUNBUFFERED=1 \ - PYTHONHASHSEED=random \ - PIP_NO_CACHE_DIR=on \ - PIP_DISABLE_PIP_VERSION_CHECK=on \ - PIP_DEFAULT_TIMEOUT=100 \ - POETRY_VERSION=1.2.2 \ - POETRY_HOME="/opt/poetry" \ - POETRY_VIRTUALENVS_IN_PROJECT=true \ - POETRY_NO_INTERACTION=1 \ - PYSETUP_PATH="/opt/pysetup" \ - VENV_PATH="/opt/pysetup/.venv" - -ENV PATH="$POETRY_HOME/bin:$VENV_PATH/bin:$PATH" - -RUN apt-get update \ - && apt-get install --no-install-recommends -y \ - curl \ - wget \ - # deps for building python deps - build-essential \ - && apt-get install -y git \ +ENV PYSETUP_PATH="/opt/pysetup" +WORKDIR $PYSETUP_PATH + +# Copy only core dependency files first for better caching +COPY pyproject.toml poetry.lock README.md ./ +COPY pynumaflow/ ./pynumaflow/ +RUN apt-get update && apt-get install --no-install-recommends -y \ + curl wget build-essential git \ && apt-get clean && rm -rf /var/lib/apt/lists/* \ - \ - # install dumb-init - && wget -O /dumb-init https://github.com/Yelp/dumb-init/releases/download/v1.2.5/dumb-init_1.2.5_x86_64 \ - && chmod +x /dumb-init \ - && curl -sSL https://install.python-poetry.org | python3 - + && pip install poetry \ + && poetry install --no-root --no-interaction #################################################################################################### -# udf: used for running the udf vertices +# Stage 2: UDF Builder - adds UDF code and installs UDF-specific deps #################################################################################################### -FROM builder AS udf +FROM base-builder AS udf-builder -WORKDIR $PYSETUP_PATH -COPY pyproject.toml ./ -RUN poetry install --no-cache --no-root && \ - rm -rf ~/.cache/pypoetry/ +ENV EXAMPLE_PATH="/opt/pysetup/examples/reduce/asyncio_reduce" +ENV POETRY_VIRTUALENVS_IN_PROJECT=true -ADD . /app -WORKDIR /app +WORKDIR $EXAMPLE_PATH +COPY examples/reduce/asyncio_reduce/ ./ +RUN poetry install --no-root --no-interaction + +#################################################################################################### +# Stage 3: UDF Runtime - clean container with only needed stuff +#################################################################################################### +FROM python:3.10-slim-bullseye AS udf + +ENV PYSETUP_PATH="/opt/pysetup" +ENV EXAMPLE_PATH="$PYSETUP_PATH/examples/reduce/asyncio_reduce" +ENV VENV_PATH="$EXAMPLE_PATH/.venv" +ENV PATH="$VENV_PATH/bin:$PATH" + +RUN apt-get update && apt-get install --no-install-recommends -y wget \ + && apt-get clean && rm -rf /var/lib/apt/lists/* \ + && wget -O /dumb-init https://github.com/Yelp/dumb-init/releases/download/v1.2.5/dumb-init_1.2.5_x86_64 \ + && chmod +x /dumb-init + +WORKDIR $PYSETUP_PATH +COPY --from=udf-builder $VENV_PATH $VENV_PATH +COPY --from=udf-builder $EXAMPLE_PATH $EXAMPLE_PATH +WORKDIR $EXAMPLE_PATH RUN chmod +x entry.sh ENTRYPOINT ["/dumb-init", "--"] -CMD ["/app/entry.sh"] +CMD ["sh", "-c", "$EXAMPLE_PATH/entry.sh"] EXPOSE 5000 diff --git a/examples/reduce/asyncio_reduce/pyproject.toml b/examples/reduce/asyncio_reduce/pyproject.toml index 31cce969..cac90449 100644 --- a/examples/reduce/asyncio_reduce/pyproject.toml +++ b/examples/reduce/asyncio_reduce/pyproject.toml @@ -7,7 +7,7 @@ authors = ["Numaflow developers"] [tool.poetry.dependencies] python = "~3.10" pynumaflow = "~0.6.0" -aiorun = "^2022.11.1" +aiorun = ">=2023.7,<2024.0" aiohttp = "~3.8.4" asyncio = "~3.4.3" diff --git a/examples/reduce/batchmap/flatmap/Dockerfile b/examples/reduce/batchmap/flatmap/Dockerfile new file mode 100644 index 00000000..a70d0d90 --- /dev/null +++ b/examples/reduce/batchmap/flatmap/Dockerfile @@ -0,0 +1,55 @@ +#################################################################################################### +# Stage 1: Base Builder - installs core dependencies using poetry +#################################################################################################### +FROM python:3.10-slim-bullseye AS base-builder + +ENV PYSETUP_PATH="/opt/pysetup" +WORKDIR $PYSETUP_PATH + +# Copy only core dependency files first for better caching +COPY pyproject.toml poetry.lock README.md ./ +COPY pynumaflow/ ./pynumaflow/ +RUN apt-get update && apt-get install --no-install-recommends -y \ + curl wget build-essential git \ + && apt-get clean && rm -rf /var/lib/apt/lists/* \ + && pip install poetry \ + && poetry install --no-root --no-interaction + +#################################################################################################### +# Stage 2: UDF Builder - adds UDF code and installs UDF-specific deps +#################################################################################################### +FROM base-builder AS udf-builder + +ENV EXAMPLE_PATH="/opt/pysetup/examples/reduce/batchmap/flatmap" +ENV POETRY_VIRTUALENVS_IN_PROJECT=true + +WORKDIR $EXAMPLE_PATH +COPY examples/reduce/batchmap/flatmap/ ./ +RUN poetry install --no-root --no-interaction + +#################################################################################################### +# Stage 3: UDF Runtime - clean container with only needed stuff +#################################################################################################### +FROM python:3.10-slim-bullseye AS udf + +ENV PYSETUP_PATH="/opt/pysetup" +ENV EXAMPLE_PATH="$PYSETUP_PATH/examples/reduce/batchmap/flatmap" +ENV VENV_PATH="$EXAMPLE_PATH/.venv" +ENV PATH="$VENV_PATH/bin:$PATH" + +RUN apt-get update && apt-get install --no-install-recommends -y wget \ + && apt-get clean && rm -rf /var/lib/apt/lists/* \ + && wget -O /dumb-init https://github.com/Yelp/dumb-init/releases/download/v1.2.5/dumb-init_1.2.5_x86_64 \ + && chmod +x /dumb-init + +WORKDIR $PYSETUP_PATH +COPY --from=udf-builder $VENV_PATH $VENV_PATH +COPY --from=udf-builder $EXAMPLE_PATH $EXAMPLE_PATH + +WORKDIR $EXAMPLE_PATH +RUN chmod +x entry.sh + +ENTRYPOINT ["/dumb-init", "--"] +CMD ["sh", "-c", "$EXAMPLE_PATH/entry.sh"] + +EXPOSE 5000 \ No newline at end of file diff --git a/examples/reduce/counter/Dockerfile b/examples/reduce/counter/Dockerfile index a617b3fa..f25a9c46 100644 --- a/examples/reduce/counter/Dockerfile +++ b/examples/reduce/counter/Dockerfile @@ -1,52 +1,52 @@ #################################################################################################### -# builder: install needed dependencies +# Stage 1: Base Builder - installs core dependencies using poetry #################################################################################################### +FROM python:3.10-slim-bullseye AS base-builder -FROM python:3.10-slim-bullseye AS builder +ENV PYSETUP_PATH="/opt/pysetup" +WORKDIR $PYSETUP_PATH + +# Copy only core dependency files first for better caching +COPY pyproject.toml poetry.lock README.md ./ +COPY pynumaflow/ ./pynumaflow/ +RUN apt-get update && apt-get install --no-install-recommends -y \ + curl wget build-essential git \ + && apt-get clean && rm -rf /var/lib/apt/lists/* \ + && pip install poetry \ + && poetry install --no-root --no-interaction + +#################################################################################################### +# Stage 2: UDF Builder - adds UDF code and installs UDF-specific deps +#################################################################################################### +FROM base-builder AS udf-builder + +ENV EXAMPLE_PATH="/opt/pysetup/examples/reduce/counter" +ENV POETRY_VIRTUALENVS_IN_PROJECT=true + +WORKDIR $EXAMPLE_PATH +COPY examples/reduce/counter/ ./ +RUN poetry install --no-root --no-interaction -ENV PYTHONFAULTHANDLER=1 \ - PYTHONUNBUFFERED=1 \ - PYTHONHASHSEED=random \ - PIP_NO_CACHE_DIR=on \ - PIP_DISABLE_PIP_VERSION_CHECK=on \ - PIP_DEFAULT_TIMEOUT=100 \ - POETRY_VERSION=1.2.2 \ - POETRY_HOME="/opt/poetry" \ - POETRY_VIRTUALENVS_IN_PROJECT=true \ - POETRY_NO_INTERACTION=1 \ - PYSETUP_PATH="/opt/pysetup" +#################################################################################################### +# Stage 3: UDF Runtime - clean container with only needed stuff +#################################################################################################### +FROM python:3.10-slim-bullseye AS udf +ENV PYSETUP_PATH="/opt/pysetup" ENV EXAMPLE_PATH="$PYSETUP_PATH/examples/reduce/counter" ENV VENV_PATH="$EXAMPLE_PATH/.venv" -ENV PATH="$POETRY_HOME/bin:$VENV_PATH/bin:$PATH" - -RUN apt-get update \ - && apt-get install --no-install-recommends -y \ - curl \ - wget \ - # deps for building python deps - build-essential \ - && apt-get install -y git \ +ENV PATH="$VENV_PATH/bin:$PATH" + +RUN apt-get update && apt-get install --no-install-recommends -y wget \ && apt-get clean && rm -rf /var/lib/apt/lists/* \ - \ - # install dumb-init && wget -O /dumb-init https://github.com/Yelp/dumb-init/releases/download/v1.2.5/dumb-init_1.2.5_x86_64 \ - && chmod +x /dumb-init \ - && curl -sSL https://install.python-poetry.org | python3 - - -#################################################################################################### -# udf: used for running the udf vertices -#################################################################################################### -FROM builder AS udf + && chmod +x /dumb-init WORKDIR $PYSETUP_PATH -COPY ./ ./ +COPY --from=udf-builder $VENV_PATH $VENV_PATH +COPY --from=udf-builder $EXAMPLE_PATH $EXAMPLE_PATH WORKDIR $EXAMPLE_PATH -RUN poetry lock -RUN poetry install --no-cache --no-root && \ - rm -rf ~/.cache/pypoetry/ - RUN chmod +x entry.sh ENTRYPOINT ["/dumb-init", "--"] diff --git a/examples/reducestream/counter/Dockerfile b/examples/reducestream/counter/Dockerfile index de1756fd..f26543d7 100644 --- a/examples/reducestream/counter/Dockerfile +++ b/examples/reducestream/counter/Dockerfile @@ -1,52 +1,52 @@ #################################################################################################### -# builder: install needed dependencies +# Stage 1: Base Builder - installs core dependencies using poetry #################################################################################################### +FROM python:3.10-slim-bullseye AS base-builder -FROM python:3.10-slim-bullseye AS builder +ENV PYSETUP_PATH="/opt/pysetup" +WORKDIR $PYSETUP_PATH + +# Copy only core dependency files first for better caching +COPY pyproject.toml poetry.lock README.md ./ +COPY pynumaflow/ ./pynumaflow/ +RUN apt-get update && apt-get install --no-install-recommends -y \ + curl wget build-essential git \ + && apt-get clean && rm -rf /var/lib/apt/lists/* \ + && pip install poetry \ + && poetry install --no-root --no-interaction + +#################################################################################################### +# Stage 2: UDF Builder - adds UDF code and installs UDF-specific deps +#################################################################################################### +FROM base-builder AS udf-builder + +ENV EXAMPLE_PATH="/opt/pysetup/examples/reducestream/counter" +ENV POETRY_VIRTUALENVS_IN_PROJECT=true + +WORKDIR $EXAMPLE_PATH +COPY examples/reducestream/counter/ ./ +RUN poetry install --no-root --no-interaction -ENV PYTHONFAULTHANDLER=1 \ - PYTHONUNBUFFERED=1 \ - PYTHONHASHSEED=random \ - PIP_NO_CACHE_DIR=on \ - PIP_DISABLE_PIP_VERSION_CHECK=on \ - PIP_DEFAULT_TIMEOUT=100 \ - POETRY_VERSION=1.2.2 \ - POETRY_HOME="/opt/poetry" \ - POETRY_VIRTUALENVS_IN_PROJECT=true \ - POETRY_NO_INTERACTION=1 \ - PYSETUP_PATH="/opt/pysetup" +#################################################################################################### +# Stage 3: UDF Runtime - clean container with only needed stuff +#################################################################################################### +FROM python:3.10-slim-bullseye AS udf +ENV PYSETUP_PATH="/opt/pysetup" ENV EXAMPLE_PATH="$PYSETUP_PATH/examples/reducestream/counter" ENV VENV_PATH="$EXAMPLE_PATH/.venv" -ENV PATH="$POETRY_HOME/bin:$VENV_PATH/bin:$PATH" - -RUN apt-get update \ - && apt-get install --no-install-recommends -y \ - curl \ - wget \ - # deps for building python deps - build-essential \ - && apt-get install -y git \ +ENV PATH="$VENV_PATH/bin:$PATH" + +RUN apt-get update && apt-get install --no-install-recommends -y wget \ && apt-get clean && rm -rf /var/lib/apt/lists/* \ - \ - # install dumb-init && wget -O /dumb-init https://github.com/Yelp/dumb-init/releases/download/v1.2.5/dumb-init_1.2.5_x86_64 \ - && chmod +x /dumb-init \ - && curl -sSL https://install.python-poetry.org | python3 - - -#################################################################################################### -# udf: used for running the udf vertices -#################################################################################################### -FROM builder AS udf + && chmod +x /dumb-init WORKDIR $PYSETUP_PATH -COPY ./ ./ +COPY --from=udf-builder $VENV_PATH $VENV_PATH +COPY --from=udf-builder $EXAMPLE_PATH $EXAMPLE_PATH WORKDIR $EXAMPLE_PATH -RUN poetry lock -RUN poetry install --no-cache --no-root && \ - rm -rf ~/.cache/pypoetry/ - RUN chmod +x entry.sh ENTRYPOINT ["/dumb-init", "--"] diff --git a/examples/reducestream/sum/Dockerfile b/examples/reducestream/sum/Dockerfile index 1f715387..4b372b78 100644 --- a/examples/reducestream/sum/Dockerfile +++ b/examples/reducestream/sum/Dockerfile @@ -1,52 +1,52 @@ #################################################################################################### -# builder: install needed dependencies +# Stage 1: Base Builder - installs core dependencies using poetry #################################################################################################### +FROM python:3.10-slim-bullseye AS base-builder -FROM python:3.10-slim-bullseye AS builder +ENV PYSETUP_PATH="/opt/pysetup" +WORKDIR $PYSETUP_PATH + +# Copy only core dependency files first for better caching +COPY pyproject.toml poetry.lock README.md ./ +COPY pynumaflow/ ./pynumaflow/ +RUN apt-get update && apt-get install --no-install-recommends -y \ + curl wget build-essential git \ + && apt-get clean && rm -rf /var/lib/apt/lists/* \ + && pip install poetry \ + && poetry install --no-root --no-interaction + +#################################################################################################### +# Stage 2: UDF Builder - adds UDF code and installs UDF-specific deps +#################################################################################################### +FROM base-builder AS udf-builder + +ENV EXAMPLE_PATH="/opt/pysetup/examples/reducestream/sum" +ENV POETRY_VIRTUALENVS_IN_PROJECT=true + +WORKDIR $EXAMPLE_PATH +COPY examples/reducestream/sum/ ./ +RUN poetry install --no-root --no-interaction -ENV PYTHONFAULTHANDLER=1 \ - PYTHONUNBUFFERED=1 \ - PYTHONHASHSEED=random \ - PIP_NO_CACHE_DIR=on \ - PIP_DISABLE_PIP_VERSION_CHECK=on \ - PIP_DEFAULT_TIMEOUT=100 \ - POETRY_VERSION=1.2.2 \ - POETRY_HOME="/opt/poetry" \ - POETRY_VIRTUALENVS_IN_PROJECT=true \ - POETRY_NO_INTERACTION=1 \ - PYSETUP_PATH="/opt/pysetup" +#################################################################################################### +# Stage 3: UDF Runtime - clean container with only needed stuff +#################################################################################################### +FROM python:3.10-slim-bullseye AS udf +ENV PYSETUP_PATH="/opt/pysetup" ENV EXAMPLE_PATH="$PYSETUP_PATH/examples/reducestream/sum" ENV VENV_PATH="$EXAMPLE_PATH/.venv" -ENV PATH="$POETRY_HOME/bin:$VENV_PATH/bin:$PATH" - -RUN apt-get update \ - && apt-get install --no-install-recommends -y \ - curl \ - wget \ - # deps for building python deps - build-essential \ - && apt-get install -y git \ +ENV PATH="$VENV_PATH/bin:$PATH" + +RUN apt-get update && apt-get install --no-install-recommends -y wget \ && apt-get clean && rm -rf /var/lib/apt/lists/* \ - \ - # install dumb-init && wget -O /dumb-init https://github.com/Yelp/dumb-init/releases/download/v1.2.5/dumb-init_1.2.5_x86_64 \ - && chmod +x /dumb-init \ - && curl -sSL https://install.python-poetry.org | python3 - - -#################################################################################################### -# udf: used for running the udf vertices -#################################################################################################### -FROM builder AS udf + && chmod +x /dumb-init WORKDIR $PYSETUP_PATH -COPY ./ ./ +COPY --from=udf-builder $VENV_PATH $VENV_PATH +COPY --from=udf-builder $EXAMPLE_PATH $EXAMPLE_PATH WORKDIR $EXAMPLE_PATH -RUN poetry lock -RUN poetry install --no-cache --no-root && \ - rm -rf ~/.cache/pypoetry/ - RUN chmod +x entry.sh ENTRYPOINT ["/dumb-init", "--"] diff --git a/examples/sideinput/simple_sideinput/Dockerfile b/examples/sideinput/simple_sideinput/Dockerfile index ab3e3355..47085100 100644 --- a/examples/sideinput/simple_sideinput/Dockerfile +++ b/examples/sideinput/simple_sideinput/Dockerfile @@ -1,52 +1,52 @@ #################################################################################################### -# builder: install needed dependencies +# Stage 1: Base Builder - installs core dependencies using poetry #################################################################################################### +FROM python:3.10-slim-bullseye AS base-builder -FROM python:3.10-slim-bullseye AS builder +ENV PYSETUP_PATH="/opt/pysetup" +WORKDIR $PYSETUP_PATH + +# Copy only core dependency files first for better caching +COPY pyproject.toml poetry.lock README.md ./ +COPY pynumaflow/ ./pynumaflow/ +RUN apt-get update && apt-get install --no-install-recommends -y \ + curl wget build-essential git \ + && apt-get clean && rm -rf /var/lib/apt/lists/* \ + && pip install poetry \ + && poetry install --no-root --no-interaction + +#################################################################################################### +# Stage 2: UDF Builder - adds UDF code and installs UDF-specific deps +#################################################################################################### +FROM base-builder AS udf-builder + +ENV EXAMPLE_PATH="/opt/pysetup/examples/sideinput/simple_sideinput" +ENV POETRY_VIRTUALENVS_IN_PROJECT=true + +WORKDIR $EXAMPLE_PATH +COPY examples/sideinput/simple_sideinput/ ./ +RUN poetry install --no-root --no-interaction -ENV PYTHONFAULTHANDLER=1 \ - PYTHONUNBUFFERED=1 \ - PYTHONHASHSEED=random \ - PIP_NO_CACHE_DIR=on \ - PIP_DISABLE_PIP_VERSION_CHECK=on \ - PIP_DEFAULT_TIMEOUT=100 \ - POETRY_VERSION=1.2.2 \ - POETRY_HOME="/opt/poetry" \ - POETRY_VIRTUALENVS_IN_PROJECT=true \ - POETRY_NO_INTERACTION=1 \ - PYSETUP_PATH="/opt/pysetup" +#################################################################################################### +# Stage 3: UDF Runtime - clean container with only needed stuff +#################################################################################################### +FROM python:3.10-slim-bullseye AS udf +ENV PYSETUP_PATH="/opt/pysetup" ENV EXAMPLE_PATH="$PYSETUP_PATH/examples/sideinput/simple_sideinput" ENV VENV_PATH="$EXAMPLE_PATH/.venv" -ENV PATH="$POETRY_HOME/bin:$VENV_PATH/bin:$PATH" - -RUN apt-get update \ - && apt-get install --no-install-recommends -y \ - curl \ - wget \ - # deps for building python deps - build-essential \ - && apt-get install -y git \ +ENV PATH="$VENV_PATH/bin:$PATH" + +RUN apt-get update && apt-get install --no-install-recommends -y wget \ && apt-get clean && rm -rf /var/lib/apt/lists/* \ - \ - # install dumb-init && wget -O /dumb-init https://github.com/Yelp/dumb-init/releases/download/v1.2.5/dumb-init_1.2.5_x86_64 \ - && chmod +x /dumb-init \ - && curl -sSL https://install.python-poetry.org | python3 - - -#################################################################################################### -# udf: used for running the udf vertices -#################################################################################################### -FROM builder AS udf + && chmod +x /dumb-init WORKDIR $PYSETUP_PATH -COPY ./ ./ +COPY --from=udf-builder $VENV_PATH $VENV_PATH +COPY --from=udf-builder $EXAMPLE_PATH $EXAMPLE_PATH WORKDIR $EXAMPLE_PATH -RUN poetry lock -RUN poetry install --no-cache --no-root && \ - rm -rf ~/.cache/pypoetry/ - RUN chmod +x entry.sh ENTRYPOINT ["/dumb-init", "--"] diff --git a/examples/sideinput/simple_sideinput/udf/Dockerfile b/examples/sideinput/simple_sideinput/udf/Dockerfile index 3cbd912a..50cc8578 100644 --- a/examples/sideinput/simple_sideinput/udf/Dockerfile +++ b/examples/sideinput/simple_sideinput/udf/Dockerfile @@ -1,52 +1,52 @@ #################################################################################################### -# builder: install needed dependencies +# Stage 1: Base Builder - installs core dependencies using poetry #################################################################################################### +FROM python:3.10-slim-bullseye AS base-builder -FROM python:3.10-slim-bullseye AS builder +ENV PYSETUP_PATH="/opt/pysetup" +WORKDIR $PYSETUP_PATH + +# Copy only core dependency files first for better caching +COPY pyproject.toml poetry.lock README.md ./ +COPY pynumaflow/ ./pynumaflow/ +RUN apt-get update && apt-get install --no-install-recommends -y \ + curl wget build-essential git \ + && apt-get clean && rm -rf /var/lib/apt/lists/* \ + && pip install poetry \ + && poetry install --no-root --no-interaction + +#################################################################################################### +# Stage 2: UDF Builder - adds UDF code and installs UDF-specific deps +#################################################################################################### +FROM base-builder AS udf-builder + +ENV EXAMPLE_PATH="/opt/pysetup/examples/sideinput/simple_sideinput/udf" +ENV POETRY_VIRTUALENVS_IN_PROJECT=true + +WORKDIR $EXAMPLE_PATH +COPY examples/sideinput/simple_sideinput/udf/ ./ +RUN poetry install --no-root --no-interaction -ENV PYTHONFAULTHANDLER=1 \ - PYTHONUNBUFFERED=1 \ - PYTHONHASHSEED=random \ - PIP_NO_CACHE_DIR=on \ - PIP_DISABLE_PIP_VERSION_CHECK=on \ - PIP_DEFAULT_TIMEOUT=100 \ - POETRY_VERSION=1.2.2 \ - POETRY_HOME="/opt/poetry" \ - POETRY_VIRTUALENVS_IN_PROJECT=true \ - POETRY_NO_INTERACTION=1 \ - PYSETUP_PATH="/opt/pysetup" +#################################################################################################### +# Stage 3: UDF Runtime - clean container with only needed stuff +#################################################################################################### +FROM python:3.10-slim-bullseye AS udf +ENV PYSETUP_PATH="/opt/pysetup" ENV EXAMPLE_PATH="$PYSETUP_PATH/examples/sideinput/simple_sideinput/udf" ENV VENV_PATH="$EXAMPLE_PATH/.venv" -ENV PATH="$POETRY_HOME/bin:$VENV_PATH/bin:$PATH" - -RUN apt-get update \ - && apt-get install --no-install-recommends -y \ - curl \ - wget \ - # deps for building python deps - build-essential \ - && apt-get install -y git \ +ENV PATH="$VENV_PATH/bin:$PATH" + +RUN apt-get update && apt-get install --no-install-recommends -y wget \ && apt-get clean && rm -rf /var/lib/apt/lists/* \ - \ - # install dumb-init && wget -O /dumb-init https://github.com/Yelp/dumb-init/releases/download/v1.2.5/dumb-init_1.2.5_x86_64 \ - && chmod +x /dumb-init \ - && curl -sSL https://install.python-poetry.org | python3 - - -#################################################################################################### -# udf: used for running the udf vertices -#################################################################################################### -FROM builder AS udf + && chmod +x /dumb-init WORKDIR $PYSETUP_PATH -COPY ./ ./ +COPY --from=udf-builder $VENV_PATH $VENV_PATH +COPY --from=udf-builder $EXAMPLE_PATH $EXAMPLE_PATH WORKDIR $EXAMPLE_PATH -RUN poetry lock -RUN poetry install --no-cache --no-root && \ - rm -rf ~/.cache/pypoetry/ - RUN chmod +x entry.sh ENTRYPOINT ["/dumb-init", "--"] diff --git a/examples/sink/async_log/Dockerfile b/examples/sink/async_log/Dockerfile index 3739ba70..4448c3a8 100644 --- a/examples/sink/async_log/Dockerfile +++ b/examples/sink/async_log/Dockerfile @@ -1,52 +1,52 @@ #################################################################################################### -# builder: install needed dependencies +# Stage 1: Base Builder - installs core dependencies using poetry #################################################################################################### +FROM python:3.10-slim-bullseye AS base-builder -FROM python:3.10-slim-bullseye AS builder +ENV PYSETUP_PATH="/opt/pysetup" +WORKDIR $PYSETUP_PATH + +# Copy only core dependency files first for better caching +COPY pyproject.toml poetry.lock README.md ./ +COPY pynumaflow/ ./pynumaflow/ +RUN apt-get update && apt-get install --no-install-recommends -y \ + curl wget build-essential git \ + && apt-get clean && rm -rf /var/lib/apt/lists/* \ + && pip install poetry \ + && poetry install --no-root --no-interaction + +#################################################################################################### +# Stage 2: UDF Builder - adds UDF code and installs UDF-specific deps +#################################################################################################### +FROM base-builder AS udf-builder + +ENV EXAMPLE_PATH="/opt/pysetup/examples/sink/async_log" +ENV POETRY_VIRTUALENVS_IN_PROJECT=true + +WORKDIR $EXAMPLE_PATH +COPY examples/sink/async_log/ ./ +RUN poetry install --no-root --no-interaction -ENV PYTHONFAULTHANDLER=1 \ - PYTHONUNBUFFERED=1 \ - PYTHONHASHSEED=random \ - PIP_NO_CACHE_DIR=on \ - PIP_DISABLE_PIP_VERSION_CHECK=on \ - PIP_DEFAULT_TIMEOUT=100 \ - POETRY_VERSION=1.2.2 \ - POETRY_HOME="/opt/poetry" \ - POETRY_VIRTUALENVS_IN_PROJECT=true \ - POETRY_NO_INTERACTION=1 \ - PYSETUP_PATH="/opt/pysetup" +#################################################################################################### +# Stage 3: UDF Runtime - clean container with only needed stuff +#################################################################################################### +FROM python:3.10-slim-bullseye AS udf +ENV PYSETUP_PATH="/opt/pysetup" ENV EXAMPLE_PATH="$PYSETUP_PATH/examples/sink/async_log" ENV VENV_PATH="$EXAMPLE_PATH/.venv" -ENV PATH="$POETRY_HOME/bin:$VENV_PATH/bin:$PATH" - -RUN apt-get update \ - && apt-get install --no-install-recommends -y \ - curl \ - wget \ - # deps for building python deps - build-essential \ - && apt-get install -y git \ +ENV PATH="$VENV_PATH/bin:$PATH" + +RUN apt-get update && apt-get install --no-install-recommends -y wget \ && apt-get clean && rm -rf /var/lib/apt/lists/* \ - \ - # install dumb-init && wget -O /dumb-init https://github.com/Yelp/dumb-init/releases/download/v1.2.5/dumb-init_1.2.5_x86_64 \ - && chmod +x /dumb-init \ - && curl -sSL https://install.python-poetry.org | python3 - - -#################################################################################################### -# udsink: used for running the udsink vertices -#################################################################################################### -FROM builder AS udsink + && chmod +x /dumb-init WORKDIR $PYSETUP_PATH -COPY ./ ./ +COPY --from=udf-builder $VENV_PATH $VENV_PATH +COPY --from=udf-builder $EXAMPLE_PATH $EXAMPLE_PATH WORKDIR $EXAMPLE_PATH -RUN poetry lock -RUN poetry install --no-cache --no-root && \ - rm -rf ~/.cache/pypoetry/ - RUN chmod +x entry.sh ENTRYPOINT ["/dumb-init", "--"] diff --git a/examples/sink/log/Dockerfile b/examples/sink/log/Dockerfile index 2b2a12aa..0c927395 100644 --- a/examples/sink/log/Dockerfile +++ b/examples/sink/log/Dockerfile @@ -1,52 +1,52 @@ #################################################################################################### -# builder: install needed dependencies +# Stage 1: Base Builder - installs core dependencies using poetry #################################################################################################### +FROM python:3.10-slim-bullseye AS base-builder -FROM python:3.10-slim-bullseye AS builder +ENV PYSETUP_PATH="/opt/pysetup" +WORKDIR $PYSETUP_PATH + +# Copy only core dependency files first for better caching +COPY pyproject.toml poetry.lock README.md ./ +COPY pynumaflow/ ./pynumaflow/ +RUN apt-get update && apt-get install --no-install-recommends -y \ + curl wget build-essential git \ + && apt-get clean && rm -rf /var/lib/apt/lists/* \ + && pip install poetry \ + && poetry install --no-root --no-interaction + +#################################################################################################### +# Stage 2: UDF Builder - adds UDF code and installs UDF-specific deps +#################################################################################################### +FROM base-builder AS udf-builder + +ENV EXAMPLE_PATH="/opt/pysetup/examples/sink/log" +ENV POETRY_VIRTUALENVS_IN_PROJECT=true + +WORKDIR $EXAMPLE_PATH +COPY examples/sink/log/ ./ +RUN poetry install --no-root --no-interaction -ENV PYTHONFAULTHANDLER=1 \ - PYTHONUNBUFFERED=1 \ - PYTHONHASHSEED=random \ - PIP_NO_CACHE_DIR=on \ - PIP_DISABLE_PIP_VERSION_CHECK=on \ - PIP_DEFAULT_TIMEOUT=100 \ - POETRY_VERSION=1.2.2 \ - POETRY_HOME="/opt/poetry" \ - POETRY_VIRTUALENVS_IN_PROJECT=true \ - POETRY_NO_INTERACTION=1 \ - PYSETUP_PATH="/opt/pysetup" +#################################################################################################### +# Stage 3: UDF Runtime - clean container with only needed stuff +#################################################################################################### +FROM python:3.10-slim-bullseye AS udf +ENV PYSETUP_PATH="/opt/pysetup" ENV EXAMPLE_PATH="$PYSETUP_PATH/examples/sink/log" ENV VENV_PATH="$EXAMPLE_PATH/.venv" -ENV PATH="$POETRY_HOME/bin:$VENV_PATH/bin:$PATH" - -RUN apt-get update \ - && apt-get install --no-install-recommends -y \ - curl \ - wget \ - # deps for building python deps - build-essential \ - && apt-get install -y git \ +ENV PATH="$VENV_PATH/bin:$PATH" + +RUN apt-get update && apt-get install --no-install-recommends -y wget \ && apt-get clean && rm -rf /var/lib/apt/lists/* \ - \ - # install dumb-init && wget -O /dumb-init https://github.com/Yelp/dumb-init/releases/download/v1.2.5/dumb-init_1.2.5_x86_64 \ - && chmod +x /dumb-init \ - && curl -sSL https://install.python-poetry.org | python3 - - -#################################################################################################### -# udsink: used for running the udsink vertices -#################################################################################################### -FROM builder AS udsink + && chmod +x /dumb-init WORKDIR $PYSETUP_PATH -COPY ./ ./ +COPY --from=udf-builder $VENV_PATH $VENV_PATH +COPY --from=udf-builder $EXAMPLE_PATH $EXAMPLE_PATH WORKDIR $EXAMPLE_PATH -RUN poetry lock -RUN poetry install --no-cache --no-root && \ - rm -rf ~/.cache/pypoetry/ - RUN chmod +x entry.sh ENTRYPOINT ["/dumb-init", "--"] diff --git a/examples/source/simple_source/Dockerfile b/examples/source/simple_source/Dockerfile index d07c719f..ca33fee3 100644 --- a/examples/source/simple_source/Dockerfile +++ b/examples/source/simple_source/Dockerfile @@ -1,52 +1,52 @@ #################################################################################################### -# builder: install needed dependencies +# Stage 1: Base Builder - installs core dependencies using poetry #################################################################################################### +FROM python:3.10-slim-bullseye AS base-builder -FROM python:3.10-slim-bullseye AS builder +ENV PYSETUP_PATH="/opt/pysetup" +WORKDIR $PYSETUP_PATH + +# Copy only core dependency files first for better caching +COPY pyproject.toml poetry.lock README.md ./ +COPY pynumaflow/ ./pynumaflow/ +RUN apt-get update && apt-get install --no-install-recommends -y \ + curl wget build-essential git \ + && apt-get clean && rm -rf /var/lib/apt/lists/* \ + && pip install poetry \ + && poetry install --no-root --no-interaction + +#################################################################################################### +# Stage 2: UDF Builder - adds UDF code and installs UDF-specific deps +#################################################################################################### +FROM base-builder AS udf-builder + +ENV EXAMPLE_PATH="/opt/pysetup/examples/source/simple_source" +ENV POETRY_VIRTUALENVS_IN_PROJECT=true + +WORKDIR $EXAMPLE_PATH +COPY examples/source/simple_source/ ./ +RUN poetry install --no-root --no-interaction -ENV PYTHONFAULTHANDLER=1 \ - PYTHONUNBUFFERED=1 \ - PYTHONHASHSEED=random \ - PIP_NO_CACHE_DIR=on \ - PIP_DISABLE_PIP_VERSION_CHECK=on \ - PIP_DEFAULT_TIMEOUT=100 \ - POETRY_VERSION=1.2.2 \ - POETRY_HOME="/opt/poetry" \ - POETRY_VIRTUALENVS_IN_PROJECT=true \ - POETRY_NO_INTERACTION=1 \ - PYSETUP_PATH="/opt/pysetup" +#################################################################################################### +# Stage 3: UDF Runtime - clean container with only needed stuff +#################################################################################################### +FROM python:3.10-slim-bullseye AS udf +ENV PYSETUP_PATH="/opt/pysetup" ENV EXAMPLE_PATH="$PYSETUP_PATH/examples/source/simple_source" ENV VENV_PATH="$EXAMPLE_PATH/.venv" -ENV PATH="$POETRY_HOME/bin:$VENV_PATH/bin:$PATH" - -RUN apt-get update \ - && apt-get install --no-install-recommends -y \ - curl \ - wget \ - # deps for building python deps - build-essential \ - && apt-get install -y git \ +ENV PATH="$VENV_PATH/bin:$PATH" + +RUN apt-get update && apt-get install --no-install-recommends -y wget \ && apt-get clean && rm -rf /var/lib/apt/lists/* \ - \ - # install dumb-init && wget -O /dumb-init https://github.com/Yelp/dumb-init/releases/download/v1.2.5/dumb-init_1.2.5_x86_64 \ - && chmod +x /dumb-init \ - && curl -sSL https://install.python-poetry.org | python3 - - -#################################################################################################### -# udf: used for running the udf vertices -#################################################################################################### -FROM builder AS udf + && chmod +x /dumb-init WORKDIR $PYSETUP_PATH -COPY ./ ./ +COPY --from=udf-builder $VENV_PATH $VENV_PATH +COPY --from=udf-builder $EXAMPLE_PATH $EXAMPLE_PATH WORKDIR $EXAMPLE_PATH -RUN poetry lock -RUN poetry install --no-cache --no-root && \ - rm -rf ~/.cache/pypoetry/ - RUN chmod +x entry.sh ENTRYPOINT ["/dumb-init", "--"] diff --git a/examples/sourcetransform/async_event_time_filter/Dockerfile b/examples/sourcetransform/async_event_time_filter/Dockerfile index 78f24d83..26e66415 100644 --- a/examples/sourcetransform/async_event_time_filter/Dockerfile +++ b/examples/sourcetransform/async_event_time_filter/Dockerfile @@ -1,52 +1,52 @@ #################################################################################################### -# builder: install needed dependencies +# Stage 1: Base Builder - installs core dependencies using poetry #################################################################################################### +FROM python:3.10-slim-bullseye AS base-builder -FROM python:3.10-slim-bullseye AS builder +ENV PYSETUP_PATH="/opt/pysetup" +WORKDIR $PYSETUP_PATH + +# Copy only core dependency files first for better caching +COPY pyproject.toml poetry.lock README.md ./ +COPY pynumaflow/ ./pynumaflow/ +RUN apt-get update && apt-get install --no-install-recommends -y \ + curl wget build-essential git \ + && apt-get clean && rm -rf /var/lib/apt/lists/* \ + && pip install poetry \ + && poetry install --no-root --no-interaction + +#################################################################################################### +# Stage 2: UDF Builder - adds UDF code and installs UDF-specific deps +#################################################################################################### +FROM base-builder AS udf-builder + +ENV EXAMPLE_PATH="/opt/pysetup/examples/sourcetransform/async_event_time_filter" +ENV POETRY_VIRTUALENVS_IN_PROJECT=true + +WORKDIR $EXAMPLE_PATH +COPY examples/sourcetransform/async_event_time_filter/ ./ +RUN poetry install --no-root --no-interaction -ENV PYTHONFAULTHANDLER=1 \ - PYTHONUNBUFFERED=1 \ - PYTHONHASHSEED=random \ - PIP_NO_CACHE_DIR=on \ - PIP_DISABLE_PIP_VERSION_CHECK=on \ - PIP_DEFAULT_TIMEOUT=100 \ - POETRY_VERSION=1.2.2 \ - POETRY_HOME="/opt/poetry" \ - POETRY_VIRTUALENVS_IN_PROJECT=true \ - POETRY_NO_INTERACTION=1 \ - PYSETUP_PATH="/opt/pysetup" +#################################################################################################### +# Stage 3: UDF Runtime - clean container with only needed stuff +#################################################################################################### +FROM python:3.10-slim-bullseye AS udf +ENV PYSETUP_PATH="/opt/pysetup" ENV EXAMPLE_PATH="$PYSETUP_PATH/examples/sourcetransform/async_event_time_filter" ENV VENV_PATH="$EXAMPLE_PATH/.venv" -ENV PATH="$POETRY_HOME/bin:$VENV_PATH/bin:$PATH" - -RUN apt-get update \ - && apt-get install --no-install-recommends -y \ - curl \ - wget \ - # deps for building python deps - build-essential \ - && apt-get install -y git \ +ENV PATH="$VENV_PATH/bin:$PATH" + +RUN apt-get update && apt-get install --no-install-recommends -y wget \ && apt-get clean && rm -rf /var/lib/apt/lists/* \ - \ - # install dumb-init && wget -O /dumb-init https://github.com/Yelp/dumb-init/releases/download/v1.2.5/dumb-init_1.2.5_x86_64 \ - && chmod +x /dumb-init \ - && curl -sSL https://install.python-poetry.org | python3 - - -#################################################################################################### -# udf: used for running the udf vertices -#################################################################################################### -FROM builder AS udf + && chmod +x /dumb-init WORKDIR $PYSETUP_PATH -COPY ./ ./ +COPY --from=udf-builder $VENV_PATH $VENV_PATH +COPY --from=udf-builder $EXAMPLE_PATH $EXAMPLE_PATH WORKDIR $EXAMPLE_PATH -RUN poetry lock -RUN poetry install --no-cache --no-root && \ - rm -rf ~/.cache/pypoetry/ - RUN chmod +x entry.sh ENTRYPOINT ["/dumb-init", "--"] diff --git a/examples/sourcetransform/event_time_filter/Dockerfile b/examples/sourcetransform/event_time_filter/Dockerfile index 3ed3480b..9e702ecf 100644 --- a/examples/sourcetransform/event_time_filter/Dockerfile +++ b/examples/sourcetransform/event_time_filter/Dockerfile @@ -1,52 +1,52 @@ #################################################################################################### -# builder: install needed dependencies +# Stage 1: Base Builder - installs core dependencies using poetry #################################################################################################### +FROM python:3.10-slim-bullseye AS base-builder -FROM python:3.10-slim-bullseye AS builder +ENV PYSETUP_PATH="/opt/pysetup" +WORKDIR $PYSETUP_PATH + +# Copy only core dependency files first for better caching +COPY pyproject.toml poetry.lock README.md ./ +COPY pynumaflow/ ./pynumaflow/ +RUN apt-get update && apt-get install --no-install-recommends -y \ + curl wget build-essential git \ + && apt-get clean && rm -rf /var/lib/apt/lists/* \ + && pip install poetry \ + && poetry install --no-root --no-interaction + +#################################################################################################### +# Stage 2: UDF Builder - adds UDF code and installs UDF-specific deps +#################################################################################################### +FROM base-builder AS udf-builder + +ENV EXAMPLE_PATH="/opt/pysetup/examples/sourcetransform/event_time_filter" +ENV POETRY_VIRTUALENVS_IN_PROJECT=true + +WORKDIR $EXAMPLE_PATH +COPY examples/sourcetransform/event_time_filter/ ./ +RUN poetry install --no-root --no-interaction -ENV PYTHONFAULTHANDLER=1 \ - PYTHONUNBUFFERED=1 \ - PYTHONHASHSEED=random \ - PIP_NO_CACHE_DIR=on \ - PIP_DISABLE_PIP_VERSION_CHECK=on \ - PIP_DEFAULT_TIMEOUT=100 \ - POETRY_VERSION=1.2.2 \ - POETRY_HOME="/opt/poetry" \ - POETRY_VIRTUALENVS_IN_PROJECT=true \ - POETRY_NO_INTERACTION=1 \ - PYSETUP_PATH="/opt/pysetup" +#################################################################################################### +# Stage 3: UDF Runtime - clean container with only needed stuff +#################################################################################################### +FROM python:3.10-slim-bullseye AS udf +ENV PYSETUP_PATH="/opt/pysetup" ENV EXAMPLE_PATH="$PYSETUP_PATH/examples/sourcetransform/event_time_filter" ENV VENV_PATH="$EXAMPLE_PATH/.venv" -ENV PATH="$POETRY_HOME/bin:$VENV_PATH/bin:$PATH" - -RUN apt-get update \ - && apt-get install --no-install-recommends -y \ - curl \ - wget \ - # deps for building python deps - build-essential \ - && apt-get install -y git \ +ENV PATH="$VENV_PATH/bin:$PATH" + +RUN apt-get update && apt-get install --no-install-recommends -y wget \ && apt-get clean && rm -rf /var/lib/apt/lists/* \ - \ - # install dumb-init && wget -O /dumb-init https://github.com/Yelp/dumb-init/releases/download/v1.2.5/dumb-init_1.2.5_x86_64 \ - && chmod +x /dumb-init \ - && curl -sSL https://install.python-poetry.org | python3 - - -#################################################################################################### -# udf: used for running the udf vertices -#################################################################################################### -FROM builder AS udf + && chmod +x /dumb-init WORKDIR $PYSETUP_PATH -COPY ./ ./ +COPY --from=udf-builder $VENV_PATH $VENV_PATH +COPY --from=udf-builder $EXAMPLE_PATH $EXAMPLE_PATH WORKDIR $EXAMPLE_PATH -RUN poetry lock -RUN poetry install --no-cache --no-root && \ - rm -rf ~/.cache/pypoetry/ - RUN chmod +x entry.sh ENTRYPOINT ["/dumb-init", "--"] diff --git a/poetry.lock b/poetry.lock index 53bad6b7..ab404238 100644 --- a/poetry.lock +++ b/poetry.lock @@ -1,4 +1,4 @@ -# This file is automatically @generated by Poetry 2.0.1 and should not be changed by hand. +# This file is automatically @generated by Poetry 2.1.3 and should not be changed by hand. [[package]] name = "aiorun" @@ -58,7 +58,7 @@ typing-extensions = {version = ">=4.0.1", markers = "python_version < \"3.11\""} [package.extras] colorama = ["colorama (>=0.4.3)"] -d = ["aiohttp (>=3.7.4)", "aiohttp (>=3.7.4,!=3.9.0)"] +d = ["aiohttp (>=3.7.4) ; sys_platform != \"win32\" or implementation_name != \"pypy\"", "aiohttp (>=3.7.4,!=3.9.0) ; sys_platform == \"win32\" and implementation_name == \"pypy\""] jupyter = ["ipython (>=7.8.0)", "tokenize-rt (>=3.2.0)"] uvloop = ["uvloop (>=0.15.2)"] @@ -304,7 +304,7 @@ files = [ tomli = {version = "*", optional = true, markers = "python_full_version <= \"3.11.0a6\" and extra == \"toml\""} [package.extras] -toml = ["tomli"] +toml = ["tomli ; python_full_version <= \"3.11.0a6\""] [[package]] name = "distlib" @@ -349,7 +349,7 @@ files = [ [package.extras] docs = ["furo (>=2024.8.6)", "sphinx (>=8.1.3)", "sphinx-autodoc-typehints (>=3)"] testing = ["covdefaults (>=2.3)", "coverage (>=7.6.10)", "diff-cover (>=9.2.1)", "pytest (>=8.3.4)", "pytest-asyncio (>=0.25.2)", "pytest-cov (>=6)", "pytest-mock (>=3.14)", "pytest-timeout (>=2.3.1)", "virtualenv (>=20.28.1)"] -typing = ["typing-extensions (>=4.12.2)"] +typing = ["typing-extensions (>=4.12.2) ; python_version < \"3.11\""] [[package]] name = "google-api-core" @@ -372,7 +372,7 @@ requests = ">=2.18.0,<3.0.0.dev0" [package.extras] async-rest = ["google-auth[aiohttp] (>=2.35.0,<3.0.dev0)"] -grpc = ["grpcio (>=1.33.2,<2.0dev)", "grpcio (>=1.49.1,<2.0dev)", "grpcio-status (>=1.33.2,<2.0.dev0)", "grpcio-status (>=1.49.1,<2.0.dev0)"] +grpc = ["grpcio (>=1.33.2,<2.0dev)", "grpcio (>=1.49.1,<2.0dev) ; python_version >= \"3.11\"", "grpcio-status (>=1.33.2,<2.0.dev0)", "grpcio-status (>=1.49.1,<2.0.dev0) ; python_version >= \"3.11\""] grpcgcp = ["grpcio-gcp (>=0.2.2,<1.0.dev0)"] grpcio-gcp = ["grpcio-gcp (>=0.2.2,<1.0.dev0)"] @@ -1016,13 +1016,13 @@ files = [ ] [package.extras] -check = ["pytest-checkdocs (>=2.4)", "pytest-ruff (>=0.2.1)", "ruff (>=0.8.0)"] -core = ["importlib_metadata (>=6)", "jaraco.collections", "jaraco.functools (>=4)", "jaraco.text (>=3.7)", "more_itertools", "more_itertools (>=8.8)", "packaging", "packaging (>=24.2)", "platformdirs (>=4.2.2)", "tomli (>=2.0.1)", "wheel (>=0.43.0)"] +check = ["pytest-checkdocs (>=2.4)", "pytest-ruff (>=0.2.1) ; sys_platform != \"cygwin\"", "ruff (>=0.8.0) ; sys_platform != \"cygwin\""] +core = ["importlib_metadata (>=6) ; python_version < \"3.10\"", "jaraco.collections", "jaraco.functools (>=4)", "jaraco.text (>=3.7)", "more_itertools", "more_itertools (>=8.8)", "packaging", "packaging (>=24.2)", "platformdirs (>=4.2.2)", "tomli (>=2.0.1) ; python_version < \"3.11\"", "wheel (>=0.43.0)"] cover = ["pytest-cov"] doc = ["furo", "jaraco.packaging (>=9.3)", "jaraco.tidelift (>=1.4)", "pygments-github-lexers (==0.0.5)", "pyproject-hooks (!=1.1)", "rst.linker (>=1.9)", "sphinx (>=3.5)", "sphinx-favicon", "sphinx-inline-tabs", "sphinx-lint", "sphinx-notfound-page (>=1,<2)", "sphinx-reredirects", "sphinxcontrib-towncrier", "towncrier (<24.7)"] enabler = ["pytest-enabler (>=2.2)"] -test = ["build[virtualenv] (>=1.0.3)", "filelock (>=3.4.0)", "ini2toml[lite] (>=0.14)", "jaraco.develop (>=7.21)", "jaraco.envs (>=2.2)", "jaraco.path (>=3.7.2)", "jaraco.test (>=5.5)", "packaging (>=24.2)", "pip (>=19.1)", "pyproject-hooks (!=1.1)", "pytest (>=6,!=8.1.*)", "pytest-home (>=0.5)", "pytest-perf", "pytest-subprocess", "pytest-timeout", "pytest-xdist (>=3)", "tomli-w (>=1.0.0)", "virtualenv (>=13.0.0)", "wheel (>=0.44.0)"] -type = ["importlib_metadata (>=7.0.2)", "jaraco.develop (>=7.21)", "mypy (==1.14.*)", "pytest-mypy"] +test = ["build[virtualenv] (>=1.0.3)", "filelock (>=3.4.0)", "ini2toml[lite] (>=0.14)", "jaraco.develop (>=7.21) ; python_version >= \"3.9\" and sys_platform != \"cygwin\"", "jaraco.envs (>=2.2)", "jaraco.path (>=3.7.2)", "jaraco.test (>=5.5)", "packaging (>=24.2)", "pip (>=19.1)", "pyproject-hooks (!=1.1)", "pytest (>=6,!=8.1.*)", "pytest-home (>=0.5)", "pytest-perf ; sys_platform != \"cygwin\"", "pytest-subprocess", "pytest-timeout", "pytest-xdist (>=3)", "tomli-w (>=1.0.0)", "virtualenv (>=13.0.0)", "wheel (>=0.44.0)"] +type = ["importlib_metadata (>=7.0.2) ; python_version < \"3.10\"", "jaraco.develop (>=7.21) ; sys_platform != \"cygwin\"", "mypy (==1.14.*)", "pytest-mypy"] [[package]] name = "tomli" @@ -1093,7 +1093,7 @@ files = [ ] [package.extras] -brotli = ["brotli (>=1.0.9)", "brotlicffi (>=0.8.0)"] +brotli = ["brotli (>=1.0.9) ; platform_python_implementation == \"CPython\"", "brotlicffi (>=0.8.0) ; platform_python_implementation != \"CPython\""] h2 = ["h2 (>=4,<5)"] socks = ["pysocks (>=1.5.6,!=1.5.7,<2.0)"] zstd = ["zstandard (>=0.18.0)"] @@ -1141,7 +1141,7 @@ files = [ [package.extras] docs = ["Sphinx (>=4.1.2,<4.2.0)", "sphinx-rtd-theme (>=0.5.2,<0.6.0)", "sphinxcontrib-asyncio (>=0.3.0,<0.4.0)"] -test = ["Cython (>=0.29.36,<0.30.0)", "aiohttp (==3.9.0b0)", "aiohttp (>=3.8.1)", "flake8 (>=5.0,<6.0)", "mypy (>=0.800)", "psutil", "pyOpenSSL (>=23.0.0,<23.1.0)", "pycodestyle (>=2.9.0,<2.10.0)"] +test = ["Cython (>=0.29.36,<0.30.0)", "aiohttp (==3.9.0b0) ; python_version >= \"3.12\"", "aiohttp (>=3.8.1) ; python_version < \"3.12\"", "flake8 (>=5.0,<6.0)", "mypy (>=0.800)", "psutil", "pyOpenSSL (>=23.0.0,<23.1.0)", "pycodestyle (>=2.9.0,<2.10.0)"] [[package]] name = "virtualenv" @@ -1162,7 +1162,7 @@ platformdirs = ">=3.9.1,<5" [package.extras] docs = ["furo (>=2023.7.26)", "proselint (>=0.13)", "sphinx (>=7.1.2,!=7.3)", "sphinx-argparse (>=0.4)", "sphinxcontrib-towncrier (>=0.2.1a0)", "towncrier (>=23.6)"] -test = ["covdefaults (>=2.3)", "coverage (>=7.2.7)", "coverage-enable-subprocess (>=1)", "flaky (>=3.7)", "packaging (>=23.1)", "pytest (>=7.4)", "pytest-env (>=0.8.2)", "pytest-freezer (>=0.4.8)", "pytest-mock (>=3.11.1)", "pytest-randomly (>=3.12)", "pytest-timeout (>=2.1)", "setuptools (>=68)", "time-machine (>=2.10)"] +test = ["covdefaults (>=2.3)", "coverage (>=7.2.7)", "coverage-enable-subprocess (>=1)", "flaky (>=3.7)", "packaging (>=23.1)", "pytest (>=7.4)", "pytest-env (>=0.8.2)", "pytest-freezer (>=0.4.8) ; platform_python_implementation == \"PyPy\" or platform_python_implementation == \"CPython\" and sys_platform == \"win32\" and python_version >= \"3.13\"", "pytest-mock (>=3.11.1)", "pytest-randomly (>=3.12)", "pytest-timeout (>=2.1)", "setuptools (>=68)", "time-machine (>=2.10) ; platform_python_implementation == \"CPython\""] [metadata] lock-version = "2.1" diff --git a/pynumaflow/proto/mapper/map_pb2.pyi b/pynumaflow/proto/mapper/map_pb2.pyi index 9832bc3e..e1279ff0 100644 --- a/pynumaflow/proto/mapper/map_pb2.pyi +++ b/pynumaflow/proto/mapper/map_pb2.pyi @@ -26,6 +26,7 @@ class MapRequest(_message.Message): key: str value: str def __init__(self, key: _Optional[str] = ..., value: _Optional[str] = ...) -> None: ... + KEYS_FIELD_NUMBER: _ClassVar[int] VALUE_FIELD_NUMBER: _ClassVar[int] EVENT_TIME_FIELD_NUMBER: _ClassVar[int] @@ -44,6 +45,7 @@ class MapRequest(_message.Message): watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., headers: _Optional[_Mapping[str, str]] = ..., ) -> None: ... + REQUEST_FIELD_NUMBER: _ClassVar[int] ID_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] @@ -89,6 +91,7 @@ class MapResponse(_message.Message): value: _Optional[bytes] = ..., tags: _Optional[_Iterable[str]] = ..., ) -> None: ... + RESULTS_FIELD_NUMBER: _ClassVar[int] ID_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] diff --git a/pynumaflow/proto/reducer/reduce_pb2.pyi b/pynumaflow/proto/reducer/reduce_pb2.pyi index 88b27d53..2c4b248c 100644 --- a/pynumaflow/proto/reducer/reduce_pb2.pyi +++ b/pynumaflow/proto/reducer/reduce_pb2.pyi @@ -48,6 +48,7 @@ class ReduceRequest(_message.Message): key: str value: str def __init__(self, key: _Optional[str] = ..., value: _Optional[str] = ...) -> None: ... + KEYS_FIELD_NUMBER: _ClassVar[int] VALUE_FIELD_NUMBER: _ClassVar[int] EVENT_TIME_FIELD_NUMBER: _ClassVar[int] @@ -66,6 +67,7 @@ class ReduceRequest(_message.Message): watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., headers: _Optional[_Mapping[str, str]] = ..., ) -> None: ... + PAYLOAD_FIELD_NUMBER: _ClassVar[int] OPERATION_FIELD_NUMBER: _ClassVar[int] payload: ReduceRequest.Payload @@ -108,6 +110,7 @@ class ReduceResponse(_message.Message): value: _Optional[bytes] = ..., tags: _Optional[_Iterable[str]] = ..., ) -> None: ... + RESULT_FIELD_NUMBER: _ClassVar[int] WINDOW_FIELD_NUMBER: _ClassVar[int] EOF_FIELD_NUMBER: _ClassVar[int] diff --git a/pynumaflow/proto/sinker/sink_pb2.pyi b/pynumaflow/proto/sinker/sink_pb2.pyi index 78926321..18d4d3b6 100644 --- a/pynumaflow/proto/sinker/sink_pb2.pyi +++ b/pynumaflow/proto/sinker/sink_pb2.pyi @@ -37,6 +37,7 @@ class SinkRequest(_message.Message): key: str value: str def __init__(self, key: _Optional[str] = ..., value: _Optional[str] = ...) -> None: ... + KEYS_FIELD_NUMBER: _ClassVar[int] VALUE_FIELD_NUMBER: _ClassVar[int] EVENT_TIME_FIELD_NUMBER: _ClassVar[int] @@ -58,6 +59,7 @@ class SinkRequest(_message.Message): id: _Optional[str] = ..., headers: _Optional[_Mapping[str, str]] = ..., ) -> None: ... + REQUEST_FIELD_NUMBER: _ClassVar[int] STATUS_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] @@ -106,6 +108,7 @@ class SinkResponse(_message.Message): status: _Optional[_Union[Status, str]] = ..., err_msg: _Optional[str] = ..., ) -> None: ... + RESULTS_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] STATUS_FIELD_NUMBER: _ClassVar[int] diff --git a/pynumaflow/proto/sourcer/source_pb2.pyi b/pynumaflow/proto/sourcer/source_pb2.pyi index f2cdc70e..8f588410 100644 --- a/pynumaflow/proto/sourcer/source_pb2.pyi +++ b/pynumaflow/proto/sourcer/source_pb2.pyi @@ -32,6 +32,7 @@ class ReadRequest(_message.Message): def __init__( self, num_records: _Optional[int] = ..., timeout_in_ms: _Optional[int] = ... ) -> None: ... + REQUEST_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] request: ReadRequest.Request @@ -55,6 +56,7 @@ class ReadResponse(_message.Message): key: str value: str def __init__(self, key: _Optional[str] = ..., value: _Optional[str] = ...) -> None: ... + PAYLOAD_FIELD_NUMBER: _ClassVar[int] OFFSET_FIELD_NUMBER: _ClassVar[int] EVENT_TIME_FIELD_NUMBER: _ClassVar[int] @@ -105,6 +107,7 @@ class ReadResponse(_message.Message): error: _Optional[_Union[ReadResponse.Status.Error, str]] = ..., msg: _Optional[str] = ..., ) -> None: ... + RESULT_FIELD_NUMBER: _ClassVar[int] STATUS_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] @@ -128,6 +131,7 @@ class AckRequest(_message.Message): def __init__( self, offsets: _Optional[_Iterable[_Union[Offset, _Mapping]]] = ... ) -> None: ... + REQUEST_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] request: AckRequest.Request @@ -148,6 +152,7 @@ class AckResponse(_message.Message): def __init__( self, success: _Optional[_Union[_empty_pb2.Empty, _Mapping]] = ... ) -> None: ... + RESULT_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] result: AckResponse.Result @@ -172,6 +177,7 @@ class PendingResponse(_message.Message): COUNT_FIELD_NUMBER: _ClassVar[int] count: int def __init__(self, count: _Optional[int] = ...) -> None: ... + RESULT_FIELD_NUMBER: _ClassVar[int] result: PendingResponse.Result def __init__( @@ -186,6 +192,7 @@ class PartitionsResponse(_message.Message): PARTITIONS_FIELD_NUMBER: _ClassVar[int] partitions: _containers.RepeatedScalarFieldContainer[int] def __init__(self, partitions: _Optional[_Iterable[int]] = ...) -> None: ... + RESULT_FIELD_NUMBER: _ClassVar[int] result: PartitionsResponse.Result def __init__( diff --git a/pynumaflow/proto/sourcetransformer/transform_pb2.pyi b/pynumaflow/proto/sourcetransformer/transform_pb2.pyi index cc8fe420..1fe8cb08 100644 --- a/pynumaflow/proto/sourcetransformer/transform_pb2.pyi +++ b/pynumaflow/proto/sourcetransformer/transform_pb2.pyi @@ -32,6 +32,7 @@ class SourceTransformRequest(_message.Message): key: str value: str def __init__(self, key: _Optional[str] = ..., value: _Optional[str] = ...) -> None: ... + KEYS_FIELD_NUMBER: _ClassVar[int] VALUE_FIELD_NUMBER: _ClassVar[int] EVENT_TIME_FIELD_NUMBER: _ClassVar[int] @@ -53,6 +54,7 @@ class SourceTransformRequest(_message.Message): headers: _Optional[_Mapping[str, str]] = ..., id: _Optional[str] = ..., ) -> None: ... + REQUEST_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] request: SourceTransformRequest.Request @@ -83,6 +85,7 @@ class SourceTransformResponse(_message.Message): event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., tags: _Optional[_Iterable[str]] = ..., ) -> None: ... + RESULTS_FIELD_NUMBER: _ClassVar[int] ID_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] diff --git a/pyproject.toml b/pyproject.toml index 74e1c586..41816b73 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -80,7 +80,5 @@ extend-exclude = [ "*_pb2*.py", "*.pyi" ] -output-format = "full" -[tool.ruff.lint] select = ["E", "F", "UP"] From c96b48885c996a17408dcc03008535cd5637a1c0 Mon Sep 17 00:00:00 2001 From: srao12 Date: Tue, 15 Jul 2025 22:03:45 -0700 Subject: [PATCH 05/30] 3/5 accum test files pass --- pynumaflow/accumulator/_dtypes.py | 18 +- .../accumulator/servicer/task_manager.py | 116 +++-- tests/accumulator/__init__.py | 0 tests/accumulator/test_async_accumulator.py | 282 +++++++++++++ .../accumulator/test_async_accumulator_err.py | 199 +++++++++ tests/accumulator/test_datatypes.py | 327 +++++++++++++++ tests/accumulator/test_debug_watermark.py | 194 +++++++++ tests/accumulator/test_use_cases.py | 396 ++++++++++++++++++ tests/accumulator/utils.py | 188 +++++++++ 9 files changed, 1686 insertions(+), 34 deletions(-) create mode 100644 tests/accumulator/__init__.py create mode 100644 tests/accumulator/test_async_accumulator.py create mode 100644 tests/accumulator/test_async_accumulator_err.py create mode 100644 tests/accumulator/test_datatypes.py create mode 100644 tests/accumulator/test_debug_watermark.py create mode 100644 tests/accumulator/test_use_cases.py create mode 100644 tests/accumulator/utils.py diff --git a/pynumaflow/accumulator/_dtypes.py b/pynumaflow/accumulator/_dtypes.py index 00e4446f..ceda2ddd 100644 --- a/pynumaflow/accumulator/_dtypes.py +++ b/pynumaflow/accumulator/_dtypes.py @@ -19,7 +19,7 @@ class WindowOperation(IntEnum): OPEN = (0,) CLOSE = (1,) - APPEND = (4,) + APPEND = (2,) @dataclass(init=False) @@ -252,26 +252,26 @@ def update_watermark(self, new_watermark: datetime): class AccumulatorRequest: """Defines the object to hold a request for the accumulator operation.""" - __slots__ = ("_operation", "_windows", "_payload") + __slots__ = ("_operation", "_keyed_window", "_payload") _operation: WindowOperation - _windows: list[KeyedWindow] + _keyed_window: KeyedWindow _payload: Datum - def __init__(self, operation: WindowOperation, windows: list[KeyedWindow], payload: Datum): + def __init__(self, operation: WindowOperation, keyed_window: KeyedWindow, payload: Datum): self._operation = operation - self._windows = windows + self._keyed_window = keyed_window self._payload = payload @property def operation(self) -> WindowOperation: - """Returns the future result of computation.""" + """Returns the operation type.""" return self._operation @property - def windows(self) -> list[KeyedWindow]: - """Returns the handle to the producer queue.""" - return self._windows + def keyed_window(self) -> KeyedWindow: + """Returns the keyed window.""" + return self._keyed_window @property def payload(self) -> Datum: diff --git a/pynumaflow/accumulator/servicer/task_manager.py b/pynumaflow/accumulator/servicer/task_manager.py index 60301f22..19053a0d 100644 --- a/pynumaflow/accumulator/servicer/task_manager.py +++ b/pynumaflow/accumulator/servicer/task_manager.py @@ -2,6 +2,7 @@ from collections.abc import AsyncIterable from datetime import datetime from typing import Union +import logging from pynumaflow._constants import ( STREAM_EOF, @@ -203,28 +204,46 @@ async def __invoke_accumulator( # then send the error to the result queue except BaseException as err: _LOGGER.critical("panic inside accumulator handle", exc_info=True) + logging.info(f"[ACCUMULATOR_DEBUG] Exception caught in __invoke_accumulator: {err}") + logging.info(f"[ACCUMULATOR_DEBUG] Putting exception in global_result_queue: {repr(err)}") # Put the exception in the result queue await self.global_result_queue.put(err) + logging.info(f"[ACCUMULATOR_DEBUG] Exception put in global_result_queue successfully") async def process_input_stream( self, request_iterator: AsyncIterable[accumulator_pb2.AccumulatorRequest] ): # Start iterating through the request iterator and create tasks # based on the operation type received. + logging.info(f"[PROCESS_INPUT_DEBUG] Starting process_input_stream") try: + request_count = 0 async for request in request_iterator: + request_count += 1 + logging.info(f"[PROCESS_INPUT_DEBUG] Processing request {request_count}, operation: {request.operation}") + logging.info(f"[PROCESS_INPUT_DEBUG] Operation value: {request.operation}") + logging.info(f"[PROCESS_INPUT_DEBUG] WindowOperation.OPEN: {int(WindowOperation.OPEN)}") + logging.info(f"[PROCESS_INPUT_DEBUG] WindowOperation.APPEND: {int(WindowOperation.APPEND)}") + logging.info(f"[PROCESS_INPUT_DEBUG] WindowOperation.CLOSE: {int(WindowOperation.CLOSE)}") + logging.info(f"[PROCESS_INPUT_DEBUG] Comparison - request.operation is int(WindowOperation.OPEN): {request.operation is int(WindowOperation.OPEN)}") # check whether the request is an open or append operation if request.operation is int(WindowOperation.OPEN): # create a new task for the open operation and # put the request in the task iterator + logging.info(f"[PROCESS_INPUT_DEBUG] Creating task for OPEN operation") await self.create_task(request) elif request.operation is int(WindowOperation.APPEND): # append the task data to the existing task # if the task does not exist, create a new task + logging.info(f"[PROCESS_INPUT_DEBUG] Sending datum to task for APPEND operation") await self.send_datum_to_task(request) elif request.operation is int(WindowOperation.CLOSE): # close the current task for req + logging.info(f"[PROCESS_INPUT_DEBUG] Closing task for CLOSE operation") await self.close_task(request) + else: + logging.info(f"[PROCESS_INPUT_DEBUG] No operation matched") + logging.info(f"[PROCESS_INPUT_DEBUG] Finished processing {request_count} requests") # If there is an error in the reduce operation, log and # then send the error to the result queue except BaseException as e: @@ -250,7 +269,7 @@ async def process_input_stream( # # Send an EOF message to the local result queue # # This will signal that the task has completed processing - # await task.result_queue.put(STREAM_EOF) + await task.result_queue.put(STREAM_EOF) # Wait for the local queue to write # all the results of this task to the global result queue @@ -285,32 +304,79 @@ async def write_to_global_queue( reader = input_queue.read_iterator() task = self.tasks[unified_key] + _LOGGER.info(f"[WRITE_TO_GLOBAL_DEBUG] Starting write_to_global_queue for key: {unified_key}") + _LOGGER.info(f"[WRITE_TO_GLOBAL_DEBUG] Task: {task}") + _LOGGER.info(f"[WRITE_TO_GLOBAL_DEBUG] Task watermark: {task.latest_watermark}") + _LOGGER.info(f"[WRITE_TO_GLOBAL_DEBUG] Task keys: {task.keys}") + + # Store the last datum processed for this task - we need this for watermark info + last_datum = getattr(task, 'last_datum', None) + _LOGGER.info(f"[WRITE_TO_GLOBAL_DEBUG] Last datum: {last_datum}") + wm: datetime = task.latest_watermark async for msg in reader: - # Convert the window to a datetime object - if wm < msg.watermark: - task.update_watermark(msg.watermark) - self.tasks[unified_key] = task - wm = msg.watermark - - start_dt = datetime.fromtimestamp(0) - end_dt = wm - res = accumulator_pb2.AccumulatorResponse( - payload=accumulator_pb2.Payload( - keys=msg.keys, - value=msg.value, - event_time=msg.event_time, - watermark=msg.watermark, - headers=msg.headers, - id=msg.id, - ), - window=accumulator_pb2.KeyedWindow( - start=start_dt, end=end_dt, slot="slot-0", keys=task.keys - ), - EOF=False, - tags=msg.tags, - ) - await output_queue.put(res) + _LOGGER.info(f"[WRITE_TO_GLOBAL_DEBUG] Received message: {msg}") + _LOGGER.info(f"[WRITE_TO_GLOBAL_DEBUG] Message type: {type(msg)}") + _LOGGER.info(f"[WRITE_TO_GLOBAL_DEBUG] Message keys: {msg.keys}") + _LOGGER.info(f"[WRITE_TO_GLOBAL_DEBUG] Message value: {msg.value}") + _LOGGER.info(f"[WRITE_TO_GLOBAL_DEBUG] Message tags: {msg.tags}") + + # For now, let's see if we can get the datum info from the task + # The task should have access to the datum information + + # If we have a last_datum, use its watermark and metadata + if last_datum: + _LOGGER.info(f"[WRITE_TO_GLOBAL_DEBUG] Using last_datum watermark: {last_datum.watermark}") + _LOGGER.info(f"[WRITE_TO_GLOBAL_DEBUG] Using last_datum event_time: {last_datum.event_time}") + _LOGGER.info(f"[WRITE_TO_GLOBAL_DEBUG] Using last_datum headers: {last_datum.headers}") + _LOGGER.info(f"[WRITE_TO_GLOBAL_DEBUG] Using last_datum id: {last_datum.id}") + + # Update watermark if the datum's watermark is newer + if wm < last_datum.watermark: + task.update_watermark(last_datum.watermark) + self.tasks[unified_key] = task + wm = last_datum.watermark + + start_dt = datetime.fromtimestamp(0) + end_dt = wm + res = accumulator_pb2.AccumulatorResponse( + payload=accumulator_pb2.Payload( + keys=msg.keys, + value=msg.value, + event_time=last_datum.event_time, + watermark=last_datum.watermark, + headers=last_datum.headers, + id=last_datum.id, + ), + window=accumulator_pb2.KeyedWindow( + start=start_dt, end=end_dt, slot="slot-0", keys=task.keys + ), + EOF=False, + tags=msg.tags, + ) + await output_queue.put(res) + else: + _LOGGER.error("[WRITE_TO_GLOBAL_DEBUG] No last_datum available!") + # This is the problematic code that tries to access msg.watermark + # TODO: We need to fix this by storing datum information properly + start_dt = datetime.fromtimestamp(0) + end_dt = wm + res = accumulator_pb2.AccumulatorResponse( + payload=accumulator_pb2.Payload( + keys=msg.keys, + value=msg.value, + event_time=datetime.now(), # Temporary fallback + watermark=wm, # Use task watermark as fallback + headers={}, # Empty headers as fallback + id="", # Empty id as fallback + ), + window=accumulator_pb2.KeyedWindow( + start=start_dt, end=end_dt, slot="slot-0", keys=task.keys + ), + EOF=False, + tags=msg.tags, + ) + await output_queue.put(res) # send EOF res = accumulator_pb2.AccumulatorResponse( window=accumulator_pb2.KeyedWindow( diff --git a/tests/accumulator/__init__.py b/tests/accumulator/__init__.py new file mode 100644 index 00000000..e69de29b diff --git a/tests/accumulator/test_async_accumulator.py b/tests/accumulator/test_async_accumulator.py new file mode 100644 index 00000000..1d0fb96f --- /dev/null +++ b/tests/accumulator/test_async_accumulator.py @@ -0,0 +1,282 @@ +import asyncio +import logging +import threading +import unittest +from collections.abc import AsyncIterable + +import grpc +from google.protobuf import empty_pb2 as _empty_pb2 +from grpc.aio._server import Server + +from pynumaflow import setup_logging +from pynumaflow.accumulator import ( + Message, + Datum, + AccumulatorAsyncServer, + Accumulator, +) +from pynumaflow.proto.accumulator import accumulator_pb2, accumulator_pb2_grpc +from pynumaflow.shared.asynciter import NonBlockingIterator +from tests.testing_utils import ( + mock_message, + mock_interval_window_start, + mock_interval_window_end, + get_time_args, +) + +LOGGER = setup_logging(__name__) + + +def request_generator(count, request, resetkey: bool = False): + for i in range(count): + if resetkey: + # Clear previous keys and add new ones + del request.payload.keys[:] + request.payload.keys.extend([f"key-{i}"]) + + # Set operation based on index - first is OPEN, rest are APPEND + if i == 0: + request.operation.event = accumulator_pb2.AccumulatorRequest.WindowOperation.Event.OPEN + else: + request.operation.event = accumulator_pb2.AccumulatorRequest.WindowOperation.Event.APPEND + yield request + + +def start_request() -> accumulator_pb2.AccumulatorRequest: + event_time_timestamp, watermark_timestamp = get_time_args() + window = accumulator_pb2.KeyedWindow( + start=mock_interval_window_start(), + end=mock_interval_window_end(), + slot="slot-0", + keys=["test_key"], + ) + payload = accumulator_pb2.Payload( + keys=["test_key"], + value=mock_message(), + event_time=event_time_timestamp, + watermark=watermark_timestamp, + id="test_id", + ) + operation = accumulator_pb2.AccumulatorRequest.WindowOperation( + event=accumulator_pb2.AccumulatorRequest.WindowOperation.Event.OPEN, + keyedWindow=window, + ) + request = accumulator_pb2.AccumulatorRequest( + payload=payload, + operation=operation, + ) + return request + + +_s: Server = None +_channel = grpc.insecure_channel("unix:///tmp/accumulator.sock") +_loop = None + + +def startup_callable(loop): + asyncio.set_event_loop(loop) + loop.run_forever() + + +class ExampleClass(Accumulator): + def __init__(self, counter): + self.counter = counter + + async def handler( + self, datums: AsyncIterable[Datum], output: NonBlockingIterator + ): + async for datum in datums: + self.counter += 1 + msg = f"counter:{self.counter}" + await output.put( + Message(str.encode(msg), keys=datum.keys(), tags=[]) + ) + + +async def accumulator_handler_func( + datums: AsyncIterable[Datum], output: NonBlockingIterator +): + counter = 0 + async for datum in datums: + counter += 1 + msg = f"counter:{counter}" + await output.put( + Message(str.encode(msg), keys=datum.keys(), tags=[]) + ) + + +def NewAsyncAccumulator(): + server_instance = AccumulatorAsyncServer(ExampleClass, init_args=(0,)) + udfs = server_instance.servicer + return udfs + + +async def start_server(udfs): + server = grpc.aio.server() + accumulator_pb2_grpc.add_AccumulatorServicer_to_server(udfs, server) + listen_addr = "unix:///tmp/accumulator.sock" + server.add_insecure_port(listen_addr) + logging.info("Starting server on %s", listen_addr) + global _s + _s = server + await server.start() + await server.wait_for_termination() + + +class TestAsyncAccumulator(unittest.TestCase): + @classmethod + def setUpClass(cls) -> None: + global _loop + loop = asyncio.new_event_loop() + _loop = loop + _thread = threading.Thread(target=startup_callable, args=(loop,), daemon=True) + _thread.start() + udfs = NewAsyncAccumulator() + asyncio.run_coroutine_threadsafe(start_server(udfs), loop=loop) + while True: + try: + with grpc.insecure_channel("unix:///tmp/accumulator.sock") as channel: + f = grpc.channel_ready_future(channel) + f.result(timeout=10) + if f.done(): + break + except grpc.FutureTimeoutError as e: + LOGGER.error("error trying to connect to grpc server") + LOGGER.error(e) + + @classmethod + def tearDownClass(cls) -> None: + try: + _loop.stop() + LOGGER.info("stopped the event loop") + except Exception as e: + LOGGER.error(e) + + def test_accumulate(self) -> None: + stub = self.__stub() + request = start_request() + generator_response = None + try: + generator_response = stub.AccumulateFn( + request_iterator=request_generator(count=5, request=request) + ) + except grpc.RpcError as e: + logging.error(e) + + # capture the output from the AccumulateFn generator and assert. + count = 0 + eof_count = 0 + for r in generator_response: + if hasattr(r, 'payload') and r.payload.value: + count += 1 + # Each datum should increment the counter + expected_msg = f"counter:{count}" + self.assertEqual( + bytes(expected_msg, encoding="utf-8"), + r.payload.value, + ) + self.assertEqual(r.EOF, False) + # Check that keys are preserved + self.assertEqual(list(r.payload.keys), ["test_key"]) + else: + self.assertEqual(r.EOF, True) + eof_count += 1 + + # We should have received 5 messages (one for each datum) + self.assertEqual(5, count) + self.assertEqual(1, eof_count) + + def test_accumulate_with_multiple_keys(self) -> None: + stub = self.__stub() + request = start_request() + generator_response = None + try: + generator_response = stub.AccumulateFn( + request_iterator=request_generator(count=10, request=request, resetkey=True), + ) + except grpc.RpcError as e: + print(e) + + count = 0 + eof_count = 0 + key_counts = {} + + # capture the output from the AccumulateFn generator and assert. + for r in generator_response: + # Check for responses with values + if r.payload.value: + count += 1 + # Track count per key + key = r.payload.keys[0] if r.payload.keys else "no_key" + key_counts[key] = key_counts.get(key, 0) + 1 + + # Each key should have its own counter starting from 1 + expected_msg = f"counter:{key_counts[key]}" + self.assertEqual( + bytes(expected_msg, encoding="utf-8"), + r.payload.value, + ) + self.assertEqual(r.EOF, False) + else: + eof_count += 1 + self.assertEqual(r.EOF, True) + + # We should have 10 messages (one for each key) + self.assertEqual(10, count) + self.assertEqual(1, eof_count) + # Each key should appear once + self.assertEqual(len(key_counts), 10) + + def test_is_ready(self) -> None: + with grpc.insecure_channel("unix:///tmp/accumulator.sock") as channel: + stub = accumulator_pb2_grpc.AccumulatorStub(channel) + + request = _empty_pb2.Empty() + response = None + try: + response = stub.IsReady(request=request) + except grpc.RpcError as e: + logging.error(e) + + self.assertTrue(response.ready) + + def __stub(self): + return accumulator_pb2_grpc.AccumulatorStub(_channel) + + def test_error_init(self): + # Check that accumulator_instance is required + with self.assertRaises(TypeError): + AccumulatorAsyncServer() + # Check that the init_args and init_kwargs are passed + # only with an Accumulator class + with self.assertRaises(TypeError): + AccumulatorAsyncServer(accumulator_handler_func, init_args=(0, 1)) + # Check that an instance is not passed instead of the class + # signature + with self.assertRaises(TypeError): + AccumulatorAsyncServer(ExampleClass(0)) + + # Check that an invalid class is passed + class ExampleBadClass: + pass + + with self.assertRaises(TypeError): + AccumulatorAsyncServer(accumulator_instance=ExampleBadClass) + + def test_max_threads(self): + # max cap at 16 + server = AccumulatorAsyncServer(accumulator_instance=ExampleClass, max_threads=32) + self.assertEqual(server.max_threads, 16) + + # use argument provided + server = AccumulatorAsyncServer(accumulator_instance=ExampleClass, max_threads=5) + self.assertEqual(server.max_threads, 5) + + # defaults to 4 + server = AccumulatorAsyncServer(accumulator_instance=ExampleClass) + self.assertEqual(server.max_threads, 4) + + +if __name__ == "__main__": + logging.basicConfig(level=logging.DEBUG) + unittest.main() diff --git a/tests/accumulator/test_async_accumulator_err.py b/tests/accumulator/test_async_accumulator_err.py new file mode 100644 index 00000000..810ee4c6 --- /dev/null +++ b/tests/accumulator/test_async_accumulator_err.py @@ -0,0 +1,199 @@ +import asyncio +import logging +import threading +import unittest +from collections.abc import AsyncIterable +from unittest.mock import patch + +import grpc +from grpc.aio._server import Server + +from pynumaflow import setup_logging +from pynumaflow.accumulator import ( + Message, + Datum, + AccumulatorAsyncServer, + Accumulator, +) +from pynumaflow.proto.accumulator import accumulator_pb2, accumulator_pb2_grpc +from pynumaflow.shared.asynciter import NonBlockingIterator +from tests.testing_utils import ( + mock_message, + mock_interval_window_start, + mock_interval_window_end, + get_time_args, + mock_terminate_on_stop, +) + +LOGGER = setup_logging(__name__) + + +def request_generator(count, request): + for i in range(count): + yield request + + +def start_request() -> accumulator_pb2.AccumulatorRequest: + event_time_timestamp, watermark_timestamp = get_time_args() + window = accumulator_pb2.KeyedWindow( + start=event_time_timestamp, + end=watermark_timestamp, + slot="slot-0", + keys=["test_key"], + ) + payload = accumulator_pb2.Payload( + keys=["test_key"], + value=mock_message(), + event_time=event_time_timestamp, + watermark=watermark_timestamp, + id="test_id", + ) + operation = accumulator_pb2.AccumulatorRequest.WindowOperation( + event=accumulator_pb2.AccumulatorRequest.WindowOperation.Event.OPEN, + keyedWindow=window, + ) + request = accumulator_pb2.AccumulatorRequest( + payload=payload, + operation=operation, + ) + return request + + +_s: Server = None +_channel = grpc.insecure_channel("unix:///tmp/accumulator_err.sock") +_loop = None + + +def startup_callable(loop): + asyncio.set_event_loop(loop) + loop.run_forever() + + +class ExampleErrorClass(Accumulator): + def __init__(self, counter): + self.counter = counter + + async def handler( + self, datums: AsyncIterable[Datum], output: NonBlockingIterator + ): + async for datum in datums: + self.counter += 1 + if self.counter == 2: + # Simulate an error on the second datum + raise RuntimeError("Simulated error in accumulator handler") + msg = f"counter:{self.counter}" + await output.put( + Message(str.encode(msg), keys=datum.keys(), tags=[]) + ) + + +async def error_accumulator_handler_func( + datums: AsyncIterable[Datum], output: NonBlockingIterator +): + counter = 0 + async for datum in datums: + counter += 1 + if counter == 2: + # Simulate an error on the second datum + raise RuntimeError("Simulated error in accumulator function") + msg = f"counter:{counter}" + await output.put( + Message(str.encode(msg), keys=datum.keys(), tags=[]) + ) + + +def NewAsyncAccumulatorError(): + server_instance = AccumulatorAsyncServer(ExampleErrorClass, init_args=(0,)) + udfs = server_instance.servicer + return udfs + + +@patch("psutil.Process.kill", mock_terminate_on_stop) +async def start_server(udfs): + server = grpc.aio.server() + accumulator_pb2_grpc.add_AccumulatorServicer_to_server(udfs, server) + listen_addr = "unix:///tmp/accumulator_err.sock" + server.add_insecure_port(listen_addr) + logging.info("Starting server on %s", listen_addr) + global _s + _s = server + await server.start() + await server.wait_for_termination() + +@patch("psutil.Process.kill", mock_terminate_on_stop) +class TestAsyncAccumulatorError(unittest.TestCase): + @classmethod + def setUpClass(cls) -> None: + global _loop + loop = asyncio.new_event_loop() + _loop = loop + _thread = threading.Thread(target=startup_callable, args=(loop,), daemon=True) + _thread.start() + udfs = NewAsyncAccumulatorError() + asyncio.run_coroutine_threadsafe(start_server(udfs), loop=loop) + while True: + try: + with grpc.insecure_channel("unix:///tmp/accumulator_err.sock") as channel: + f = grpc.channel_ready_future(channel) + f.result(timeout=10) + if f.done(): + break + except grpc.FutureTimeoutError as e: + LOGGER.error("error trying to connect to grpc server") + LOGGER.error(e) + + @classmethod + def tearDownClass(cls) -> None: + try: + _loop.stop() + LOGGER.info("stopped the event loop") + except Exception as e: + LOGGER.error(e) + + @patch("psutil.Process.kill", mock_terminate_on_stop) + def test_accumulate_error(self) -> None: + stub = self.__stub() + request = start_request() + generator_response = None + + try: + generator_response = stub.AccumulateFn( + request_iterator=request_generator(count=5, request=request) + ) + + # Try to consume the generator + counter = 0 + for _ in generator_response: + counter += 1 + except BaseException as err: + self.assertTrue("Simulated error in accumulator handler" in str(err)) + return + self.fail("Expected an exception.") + + @patch("psutil.Process.kill", mock_terminate_on_stop) + def test_accumulate_partial_success(self) -> None: + """Test that the first datum is processed before error occurs""" + stub = self.__stub() + request = start_request() + + try: + generator_response = stub.AccumulateFn( + request_iterator=request_generator(count=3, request=request) + ) + + # Try to consume the generator + counter = 0 + for _ in generator_response: + counter += 1 + except BaseException as err: + self.assertTrue("Simulated error in accumulator handler" in str(err)) + return + self.fail("Expected an exception.") + + def __stub(self): + return accumulator_pb2_grpc.AccumulatorStub(_channel) + + +if __name__ == "__main__": + logging.basicConfig(level=logging.DEBUG) + unittest.main() diff --git a/tests/accumulator/test_datatypes.py b/tests/accumulator/test_datatypes.py new file mode 100644 index 00000000..1bd37691 --- /dev/null +++ b/tests/accumulator/test_datatypes.py @@ -0,0 +1,327 @@ +from copy import deepcopy +import unittest +from collections.abc import AsyncIterable +from datetime import datetime, timezone + +from google.protobuf import timestamp_pb2 as _timestamp_pb2 +from pynumaflow.accumulator import Accumulator + +from pynumaflow.accumulator._dtypes import ( + IntervalWindow, + KeyedWindow, + Metadata, + Datum, + AccumulatorResult, + AccumulatorRequest, + WindowOperation, + Message, +) +from pynumaflow.shared.asynciter import NonBlockingIterator +from tests.testing_utils import ( + mock_message, + mock_event_time, + mock_watermark, + mock_start_time, + mock_end_time, + mock_headers, +) + +TEST_KEYS = ["test"] +TEST_ID = "test_id" + + +class TestDatum(unittest.TestCase): + def test_err_event_time(self): + ts = _timestamp_pb2.Timestamp() + ts.GetCurrentTime() + headers = {"key1": "value1", "key2": "value2"} + with self.assertRaises(Exception) as context: + Datum( + keys=TEST_KEYS, + value=mock_message(), + event_time=ts, + watermark=mock_watermark(), + id_=TEST_ID, + headers=headers, + ) + self.assertEqual( + "Wrong data type: " + "for Datum.event_time", + str(context.exception), + ) + + def test_err_watermark(self): + ts = _timestamp_pb2.Timestamp() + ts.GetCurrentTime() + headers = {"key1": "value1", "key2": "value2"} + with self.assertRaises(Exception) as context: + Datum( + keys=TEST_KEYS, + value=mock_message(), + event_time=mock_event_time(), + watermark=ts, + id_=TEST_ID, + headers=headers, + ) + self.assertEqual( + "Wrong data type: " + "for Datum.watermark", + str(context.exception), + ) + + def test_value(self): + test_headers = {"key1": "value1", "key2": "value2"} + d = Datum( + keys=TEST_KEYS, + value=mock_message(), + event_time=mock_event_time(), + watermark=mock_watermark(), + id_=TEST_ID, + headers=test_headers, + ) + self.assertEqual(mock_message(), d.value) + self.assertEqual(test_headers, d.headers) + self.assertEqual(TEST_ID, d.id) + + def test_key(self): + d = Datum( + keys=TEST_KEYS, + value=mock_message(), + event_time=mock_event_time(), + watermark=mock_watermark(), + id_=TEST_ID, + ) + self.assertEqual(TEST_KEYS, d.keys()) + + def test_event_time(self): + d = Datum( + keys=TEST_KEYS, + value=mock_message(), + event_time=mock_event_time(), + watermark=mock_watermark(), + id_=TEST_ID, + ) + self.assertEqual(mock_event_time(), d.event_time) + + def test_watermark(self): + d = Datum( + keys=TEST_KEYS, + value=mock_message(), + event_time=mock_event_time(), + watermark=mock_watermark(), + id_=TEST_ID, + ) + self.assertEqual(mock_watermark(), d.watermark) + + def test_default_values(self): + d = Datum( + keys=None, + value=None, + event_time=mock_event_time(), + watermark=mock_watermark(), + id_=TEST_ID, + ) + self.assertEqual([], d.keys()) + self.assertEqual(b"", d.value) + self.assertEqual({}, d.headers) + + +class TestIntervalWindow(unittest.TestCase): + def test_start(self): + i = IntervalWindow(start=mock_start_time(), end=mock_end_time()) + self.assertEqual(mock_start_time(), i.start) + + def test_end(self): + i = IntervalWindow(start=mock_start_time(), end=mock_end_time()) + self.assertEqual(mock_end_time(), i.end) + + +class TestKeyedWindow(unittest.TestCase): + def test_create_window(self): + kw = KeyedWindow( + start=mock_start_time(), end=mock_end_time(), slot="slot-0", keys=["key1", "key2"] + ) + self.assertEqual(kw.start, mock_start_time()) + self.assertEqual(kw.end, mock_end_time()) + self.assertEqual(kw.slot, "slot-0") + self.assertEqual(kw.keys, ["key1", "key2"]) + + def test_default_values(self): + kw = KeyedWindow(start=mock_start_time(), end=mock_end_time()) + self.assertEqual(kw.slot, "") + self.assertEqual(kw.keys, []) + + def test_window_property(self): + kw = KeyedWindow(start=mock_start_time(), end=mock_end_time()) + self.assertIsInstance(kw.window, IntervalWindow) + self.assertEqual(kw.window.start, mock_start_time()) + self.assertEqual(kw.window.end, mock_end_time()) + + +class TestMetadata(unittest.TestCase): + def test_interval_window(self): + i = IntervalWindow(start=mock_start_time(), end=mock_end_time()) + m = Metadata(interval_window=i) + self.assertEqual(type(i), type(m.interval_window)) + self.assertEqual(i, m.interval_window) + + +class TestAccumulatorResult(unittest.TestCase): + def test_create_result(self): + # Create mock objects + future = None # In real usage, this would be an asyncio.Task + iterator = NonBlockingIterator() + keys = ["key1", "key2"] + result_queue = NonBlockingIterator() + consumer_future = None # In real usage, this would be an asyncio.Task + watermark = datetime.fromtimestamp(1662998400, timezone.utc) + + result = AccumulatorResult( + future, iterator, keys, result_queue, consumer_future, watermark + ) + + self.assertEqual(result.future, future) + self.assertEqual(result.iterator, iterator) + self.assertEqual(result.keys, keys) + self.assertEqual(result.result_queue, result_queue) + self.assertEqual(result.consumer_future, consumer_future) + self.assertEqual(result.latest_watermark, watermark) + + def test_update_watermark(self): + result = AccumulatorResult( + None, None, [], None, None, datetime.fromtimestamp(1662998400, timezone.utc) + ) + new_watermark = datetime.fromtimestamp(1662998460, timezone.utc) + result.update_watermark(new_watermark) + self.assertEqual(result.latest_watermark, new_watermark) + + def test_update_watermark_invalid_type(self): + result = AccumulatorResult( + None, None, [], None, None, datetime.fromtimestamp(1662998400, timezone.utc) + ) + with self.assertRaises(TypeError): + result.update_watermark("not a datetime") + + +class TestAccumulatorRequest(unittest.TestCase): + def test_create_request(self): + operation = WindowOperation.OPEN + windows = [KeyedWindow(start=mock_start_time(), end=mock_end_time())] + payload = Datum( + keys=TEST_KEYS, + value=mock_message(), + event_time=mock_event_time(), + watermark=mock_watermark(), + id_=TEST_ID, + ) + + request = AccumulatorRequest(operation, windows, payload) + self.assertEqual(request.operation, operation) + self.assertEqual(request.windows, windows) + self.assertEqual(request.payload, payload) + + +class TestWindowOperation(unittest.TestCase): + def test_enum_values(self): + self.assertEqual(WindowOperation.OPEN, 0) + self.assertEqual(WindowOperation.CLOSE, 1) + self.assertEqual(WindowOperation.APPEND, 2) + + +class TestMessage(unittest.TestCase): + def test_create_message(self): + value = b"test_value" + keys = ["key1", "key2"] + tags = ["tag1", "tag2"] + + msg = Message(value=value, keys=keys, tags=tags) + self.assertEqual(msg.value, value) + self.assertEqual(msg.keys, keys) + self.assertEqual(msg.tags, tags) + + def test_default_values(self): + msg = Message(value=b"test") + self.assertEqual(msg.keys, []) + self.assertEqual(msg.tags, []) + + def test_to_drop(self): + msg = Message.to_drop() + self.assertEqual(msg.value, b"") + self.assertEqual(msg.keys, []) + self.assertTrue("U+005C__DROP__" in msg.tags) + + def test_none_values(self): + msg = Message(value=None, keys=None, tags=None) + self.assertEqual(msg.value, b"") + self.assertEqual(msg.keys, []) + self.assertEqual(msg.tags, []) + + +class TestAccumulatorClass(unittest.TestCase): + class ExampleClass(Accumulator): + async def handler( + self, datums: AsyncIterable[Datum], output: NonBlockingIterator + ): + pass + + def __init__(self, test1, test2): + self.test1 = test1 + self.test2 = test2 + self.test3 = self.test1 + + def test_init(self): + r = self.ExampleClass(test1=1, test2=2) + self.assertEqual(1, r.test1) + self.assertEqual(2, r.test2) + self.assertEqual(1, r.test3) + + def test_deep_copy(self): + """Test that the deepcopy works as expected""" + r = self.ExampleClass(test1=1, test2=2) + # Create a copy of r + r_copy = deepcopy(r) + # Check that the attributes are the same + self.assertEqual(1, r_copy.test1) + self.assertEqual(2, r_copy.test2) + self.assertEqual(1, r_copy.test3) + # Check that the objects are not the same + self.assertNotEqual(id(r), id(r_copy)) + # Update the attributes of r + r.test1 = 5 + r.test3 = 6 + # Check that the other object is not updated + self.assertNotEqual(r.test1, r_copy.test1) + self.assertNotEqual(r.test3, r_copy.test3) + self.assertNotEqual(id(r.test3), id(r_copy.test3)) + # Verify that the instance type is correct + self.assertTrue(isinstance(r_copy, self.ExampleClass)) + self.assertTrue(isinstance(r_copy, Accumulator)) + + def test_callable(self): + """Test that accumulator instances can be called directly""" + r = self.ExampleClass(test1=1, test2=2) + # The __call__ method should be callable and delegate to the handler method + self.assertTrue(callable(r)) + # __call__ should return the result of calling handler + # Since handler is an async method, __call__ should return a coroutine + import asyncio + from pynumaflow.shared.asynciter import NonBlockingIterator + + async def test_datums(): + yield Datum( + keys=["test"], + value=b"test", + event_time=mock_event_time(), + watermark=mock_watermark(), + id_="test", + ) + + output = NonBlockingIterator() + result = r(test_datums(), output) + self.assertTrue(asyncio.iscoroutine(result)) + # Clean up the coroutine + result.close() + + +if __name__ == "__main__": + unittest.main() diff --git a/tests/accumulator/test_debug_watermark.py b/tests/accumulator/test_debug_watermark.py new file mode 100644 index 00000000..b28d440b --- /dev/null +++ b/tests/accumulator/test_debug_watermark.py @@ -0,0 +1,194 @@ +import asyncio +import logging +import threading +import unittest +from collections.abc import AsyncIterable +from unittest.mock import patch + +import grpc +from grpc.aio._server import Server + +from pynumaflow import setup_logging +from pynumaflow.accumulator import ( + Message, + Datum, + AccumulatorAsyncServer, + Accumulator, +) +from pynumaflow.proto.accumulator import accumulator_pb2, accumulator_pb2_grpc +from pynumaflow.shared.asynciter import NonBlockingIterator +from tests.testing_utils import ( + mock_message, + mock_interval_window_start, + mock_interval_window_end, + get_time_args, + mock_terminate_on_stop, +) + +LOGGER = setup_logging(__name__) + + +def request_generator(count, request): + for i in range(count): + yield request + + +def start_request() -> accumulator_pb2.AccumulatorRequest: + event_time_timestamp, watermark_timestamp = get_time_args() + window = accumulator_pb2.KeyedWindow( + start=event_time_timestamp, + end=watermark_timestamp, + slot="slot-0", + keys=["test_key"], + ) + payload = accumulator_pb2.Payload( + keys=["test_key"], + value=mock_message(), + event_time=event_time_timestamp, + watermark=watermark_timestamp, + id="test_id", + headers={"test_header_key": "test_header_value", "source": "test_source"}, + ) + operation = accumulator_pb2.AccumulatorRequest.WindowOperation( + event=accumulator_pb2.AccumulatorRequest.WindowOperation.Event.OPEN, + keyedWindow=window, + ) + request = accumulator_pb2.AccumulatorRequest( + payload=payload, + operation=operation, + ) + return request + + +_s: Server = None +_channel = grpc.insecure_channel("unix:///tmp/accumulator_err.sock") +_loop = None + + +def startup_callable(loop): + asyncio.set_event_loop(loop) + loop.run_forever() + + +class ExampleErrorClass(Accumulator): + def __init__(self, counter): + self.counter = counter + + async def handler( + self, datums: AsyncIterable[Datum], output: NonBlockingIterator + ): + logging.info(f"[HANDLER_DEBUG] Handler called with counter: {self.counter}") + async for datum in datums: + self.counter += 1 + logging.info(f"[HANDLER_DEBUG] Processing datum {self.counter}: {datum}") + logging.info(f"[HANDLER_DEBUG] Datum watermark: {datum.watermark}") + logging.info(f"[HANDLER_DEBUG] Datum event_time: {datum.event_time}") + logging.info(f"[HANDLER_DEBUG] Datum headers: {datum.headers}") + logging.info(f"[HANDLER_DEBUG] Datum id: {datum.id}") + + msg = f"counter:{self.counter}" + logging.info(f"[HANDLER_DEBUG] Putting message: {msg}") + message_obj = Message(str.encode(msg), keys=datum.keys(), tags=[]) + logging.info(f"[HANDLER_DEBUG] Message object: {message_obj}") + logging.info(f"[HANDLER_DEBUG] Message has watermark: {hasattr(message_obj, 'watermark')}") + logging.info(f"[HANDLER_DEBUG] Message has event_time: {hasattr(message_obj, 'event_time')}") + logging.info(f"[HANDLER_DEBUG] Message has headers: {hasattr(message_obj, 'headers')}") + logging.info(f"[HANDLER_DEBUG] Message has id: {hasattr(message_obj, 'id')}") + + await output.put(message_obj) + logging.info(f"[HANDLER_DEBUG] Handler finished processing all datums") + + +async def error_accumulator_handler_func( + datums: AsyncIterable[Datum], output: NonBlockingIterator +): + counter = 0 + async for datum in datums: + counter += 1 + if counter == 2: + # Simulate an error on the second datum + raise RuntimeError("Simulated error in accumulator function") + msg = f"counter:{counter}" + await output.put( + Message(str.encode(msg), keys=datum.keys(), tags=[]) + ) + + +def NewAsyncAccumulatorError(): + server_instance = AccumulatorAsyncServer(ExampleErrorClass, init_args=(0,)) + udfs = server_instance.servicer + return udfs + + +@patch("psutil.Process.kill", mock_terminate_on_stop) +async def start_server(udfs): + server = grpc.aio.server() + accumulator_pb2_grpc.add_AccumulatorServicer_to_server(udfs, server) + listen_addr = "unix:///tmp/accumulator_err.sock" + server.add_insecure_port(listen_addr) + logging.info("Starting server on %s", listen_addr) + global _s + _s = server + await server.start() + await server.wait_for_termination() + +@patch("psutil.Process.kill", mock_terminate_on_stop) +class TestDebugWatermark(unittest.TestCase): + @classmethod + def setUpClass(cls) -> None: + global _loop + loop = asyncio.new_event_loop() + _loop = loop + _thread = threading.Thread(target=startup_callable, args=(loop,), daemon=True) + _thread.start() + udfs = NewAsyncAccumulatorError() + asyncio.run_coroutine_threadsafe(start_server(udfs), loop=loop) + while True: + try: + with grpc.insecure_channel("unix:///tmp/accumulator_err.sock") as channel: + f = grpc.channel_ready_future(channel) + f.result(timeout=10) + if f.done(): + break + except grpc.FutureTimeoutError as e: + LOGGER.error("error trying to connect to grpc server") + LOGGER.error(e) + + @classmethod + def tearDownClass(cls) -> None: + try: + _loop.stop() + LOGGER.info("stopped the event loop") + except Exception as e: + LOGGER.error(e) + + @patch("psutil.Process.kill", mock_terminate_on_stop) + def test_debug_watermark_flow(self) -> None: + """Debug test to investigate watermark information flow""" + stub = self.__stub() + request = start_request() + + try: + generator_response = stub.AccumulateFn( + request_iterator=request_generator(count=2, request=request) + ) + + # Try to consume the generator + counter = 0 + logging.info(f"[TEST_DEBUG] About to iterate through generator_response") + for response in generator_response: + counter += 1 + logging.info(f"[TEST_DEBUG] Received response {counter}: {response}") + logging.info(f"[TEST_DEBUG] Finished iterating, got {counter} responses") + except BaseException as err: + logging.info(f"[TEST_DEBUG] Caught exception: {err}") + # For debug purposes, we expect to see the watermark error + self.assertTrue("watermark" in str(err).lower() or "attribute" in str(err).lower()) + + def __stub(self): + return accumulator_pb2_grpc.AccumulatorStub(_channel) + + +if __name__ == "__main__": + logging.basicConfig(level=logging.DEBUG) + unittest.main() diff --git a/tests/accumulator/test_use_cases.py b/tests/accumulator/test_use_cases.py new file mode 100644 index 00000000..8bf4a46b --- /dev/null +++ b/tests/accumulator/test_use_cases.py @@ -0,0 +1,396 @@ +import asyncio +import unittest +from collections.abc import AsyncIterable +from datetime import datetime, timezone +from typing import Dict, List + +from pynumaflow.accumulator import ( + Message, + Datum, + Accumulator, + AccumulatorAsyncServer, +) +from pynumaflow.shared.asynciter import NonBlockingIterator +from tests.testing_utils import mock_headers + + +class StreamSorterAccumulator(Accumulator): + """ + Accumulator that sorts events by event_time and watermark. + This demonstrates custom sorting use case. + """ + + def __init__(self): + self.buffer: List[Datum] = [] + + async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterator): + # Collect all datums + async for datum in datums: + self.buffer.append(datum) + + # Sort by event_time + self.buffer.sort(key=lambda d: d.event_time) + + # Emit sorted datums + for datum in self.buffer: + await output.put( + Message( + value=datum.value, + keys=datum.keys(), + tags=[], + ) + ) + + # Clear buffer for next window + self.buffer.clear() + + +class StreamJoinerAccumulator(Accumulator): + """ + Accumulator that joins streams from different sources based on keys. + This demonstrates stream joining use case. + """ + + def __init__(self): + self.streams: Dict[str, List[Datum]] = {} + + async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterator): + # Group datums by source (using first key as source identifier) + async for datum in datums: + source_key = datum.keys()[0] if datum.keys() else "default" + if source_key not in self.streams: + self.streams[source_key] = [] + self.streams[source_key].append(datum) + + # Join streams by combining data from different sources + # Sort all data by event_time across all sources + all_datums = [] + for source_datums in self.streams.values(): + all_datums.extend(source_datums) + + # Sort by event_time for temporal ordering + all_datums.sort(key=lambda d: d.event_time) + + # Emit joined and sorted stream + for datum in all_datums: + joined_value = f"joined_{datum.value.decode()}" + await output.put( + Message( + value=joined_value.encode(), + keys=datum.keys(), + tags=["joined"], + ) + ) + + # Clear streams for next window + self.streams.clear() + + +class ReorderingAccumulator(Accumulator): + """ + Accumulator that handles out-of-order events by reordering them. + This demonstrates event reordering use case. + """ + + def __init__(self, max_delay_seconds: int = 5): + self.max_delay_seconds = max_delay_seconds + self.event_buffer: List[Datum] = [] + + async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterator): + # Collect all datums + async for datum in datums: + self.event_buffer.append(datum) + + # Sort by event_time to handle out-of-order events + self.event_buffer.sort(key=lambda d: d.event_time) + + # Process events that are within the acceptable delay window + current_time = datetime.now(timezone.utc) + processed_events = [] + + for datum in self.event_buffer: + # Check if event is within acceptable delay + delay = (current_time - datum.event_time).total_seconds() + if delay <= self.max_delay_seconds: + processed_events.append(datum) + else: + # Event is too old, but we still process it with a tag + reordered_datum = datum + await output.put( + Message( + value=reordered_datum.value, + keys=reordered_datum.keys(), + tags=["reordered", "delayed"], + ) + ) + + # Emit processed events in order + for datum in processed_events: + await output.put( + Message( + value=datum.value, + keys=datum.keys(), + tags=["reordered"], + ) + ) + + # Clear buffer + self.event_buffer.clear() + + +class TimeBasedCorrelationAccumulator(Accumulator): + """ + Accumulator that correlates events from different sources based on timestamps. + This demonstrates time-based correlation use case. + """ + + def __init__(self, correlation_window_seconds: int = 10): + self.correlation_window_seconds = correlation_window_seconds + self.events: List[Datum] = [] + + async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterator): + # Collect all datums + async for datum in datums: + self.events.append(datum) + + # Sort by event_time + self.events.sort(key=lambda d: d.event_time) + + # Correlate events within time windows + correlated_groups = [] + current_group = [] + + for event in self.events: + if not current_group: + current_group.append(event) + else: + # Check if event is within correlation window + time_diff = (event.event_time - current_group[0].event_time).total_seconds() + if time_diff <= self.correlation_window_seconds: + current_group.append(event) + else: + # Start new group + correlated_groups.append(current_group) + current_group = [event] + + # Add the last group + if current_group: + correlated_groups.append(current_group) + + # Emit correlated events + for group_idx, group in enumerate(correlated_groups): + for event in group: + correlation_id = f"corr_group_{group_idx}" + correlated_value = f"correlated_{event.value.decode()}" + await output.put( + Message( + value=correlated_value.encode(), + keys=event.keys() + [correlation_id], + tags=["correlated"], + ) + ) + + # Clear events + self.events.clear() + + +class CustomTriggerAccumulator(Accumulator): + """ + Accumulator that triggers actions based on custom conditions. + This demonstrates custom triggering use case. + """ + + def __init__(self, trigger_count: int = 3): + self.trigger_count = trigger_count + self.accumulated_events: List[Datum] = [] + + async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterator): + # Collect datums + async for datum in datums: + self.accumulated_events.append(datum) + + # Custom trigger: when we have enough events or specific conditions + if len(self.accumulated_events) >= self.trigger_count: + # Trigger action: process all accumulated events + total_value = sum( + int(event.value.decode()) for event in self.accumulated_events + if event.value.decode().isdigit() + ) + + # Emit triggered result + await output.put( + Message( + value=f"triggered_sum_{total_value}".encode(), + keys=["triggered"], + tags=["custom_trigger"], + ) + ) + + # Clear accumulated events + self.accumulated_events.clear() + else: + # Not enough events to trigger, emit individual events + for event in self.accumulated_events: + await output.put( + Message( + value=event.value, + keys=event.keys(), + tags=["pending_trigger"], + ) + ) + + +class TestAccumulatorUseCases(unittest.TestCase): + def test_stream_sorter_accumulator(self): + """Test the stream sorting use case""" + sorter = StreamSorterAccumulator() + + # Test that the accumulator sorts by event_time + self.assertIsInstance(sorter, StreamSorterAccumulator) + self.assertEqual(len(sorter.buffer), 0) + + def test_stream_joiner_accumulator(self): + """Test the stream joining use case""" + joiner = StreamJoinerAccumulator() + + # Test that the accumulator can join streams + self.assertIsInstance(joiner, StreamJoinerAccumulator) + self.assertEqual(len(joiner.streams), 0) + + def test_reordering_accumulator(self): + """Test the event reordering use case""" + reorderer = ReorderingAccumulator(max_delay_seconds=10) + + # Test that the accumulator handles reordering + self.assertIsInstance(reorderer, ReorderingAccumulator) + self.assertEqual(reorderer.max_delay_seconds, 10) + self.assertEqual(len(reorderer.event_buffer), 0) + + def test_time_based_correlation_accumulator(self): + """Test the time-based correlation use case""" + correlator = TimeBasedCorrelationAccumulator(correlation_window_seconds=5) + + # Test that the accumulator correlates events + self.assertIsInstance(correlator, TimeBasedCorrelationAccumulator) + self.assertEqual(correlator.correlation_window_seconds, 5) + self.assertEqual(len(correlator.events), 0) + + def test_custom_trigger_accumulator(self): + """Test the custom triggering use case""" + trigger = CustomTriggerAccumulator(trigger_count=5) + + # Test that the accumulator handles custom triggers + self.assertIsInstance(trigger, CustomTriggerAccumulator) + self.assertEqual(trigger.trigger_count, 5) + self.assertEqual(len(trigger.accumulated_events), 0) + + def test_accumulator_server_with_use_cases(self): + """Test that AccumulatorAsyncServer can be created with use case implementations""" + # Test with StreamSorterAccumulator + server1 = AccumulatorAsyncServer(StreamSorterAccumulator) + self.assertIsNotNone(server1) + + # Test with StreamJoinerAccumulator + server2 = AccumulatorAsyncServer(StreamJoinerAccumulator) + self.assertIsNotNone(server2) + + # Test with ReorderingAccumulator with init args + server3 = AccumulatorAsyncServer(ReorderingAccumulator, init_args=(10,)) + self.assertIsNotNone(server3) + + # Test with TimeBasedCorrelationAccumulator with init args + server4 = AccumulatorAsyncServer(TimeBasedCorrelationAccumulator, init_args=(15,)) + self.assertIsNotNone(server4) + + # Test with CustomTriggerAccumulator with init args + server5 = AccumulatorAsyncServer(CustomTriggerAccumulator, init_args=(3,)) + self.assertIsNotNone(server5) + + def test_stream_sorter_functionality(self): + """Test actual sorting functionality""" + async def _test_stream_sorter_functionality(): + sorter = StreamSorterAccumulator() + output = NonBlockingIterator() + + # Create datums with different event times (out of order) + datums = [ + Datum( + keys=["test"], + value=b"event_3", + event_time=datetime.fromtimestamp(1662998460, timezone.utc), + watermark=datetime.fromtimestamp(1662998460, timezone.utc), + id_="3", + ), + Datum( + keys=["test"], + value=b"event_1", + event_time=datetime.fromtimestamp(1662998400, timezone.utc), + watermark=datetime.fromtimestamp(1662998400, timezone.utc), + id_="1", + ), + Datum( + keys=["test"], + value=b"event_2", + event_time=datetime.fromtimestamp(1662998430, timezone.utc), + watermark=datetime.fromtimestamp(1662998430, timezone.utc), + id_="2", + ), + ] + + async def datum_generator(): + for datum in datums: + yield datum + + # Process the datums + await sorter.handler(datum_generator(), output) + + # Verify the buffer is cleared + self.assertEqual(len(sorter.buffer), 0) + + asyncio.run(_test_stream_sorter_functionality()) + + def test_stream_joiner_functionality(self): + """Test actual joining functionality""" + async def _test_stream_joiner_functionality(): + joiner = StreamJoinerAccumulator() + output = NonBlockingIterator() + + # Create datums from different sources + datums = [ + Datum( + keys=["source1"], + value=b"data_from_source1", + event_time=datetime.fromtimestamp(1662998400, timezone.utc), + watermark=datetime.fromtimestamp(1662998400, timezone.utc), + id_="s1_1", + ), + Datum( + keys=["source2"], + value=b"data_from_source2", + event_time=datetime.fromtimestamp(1662998430, timezone.utc), + watermark=datetime.fromtimestamp(1662998430, timezone.utc), + id_="s2_1", + ), + ] + + async def datum_generator(): + for datum in datums: + yield datum + + # Process the datums + await joiner.handler(datum_generator(), output) + + # Verify the streams are cleared + self.assertEqual(len(joiner.streams), 0) + + asyncio.run(_test_stream_joiner_functionality()) + + def test_run_async_tests(self): + """Run the async test methods""" + # This test method is no longer needed since the async tests + # are now properly handled within their respective test methods + pass + + +if __name__ == "__main__": + unittest.main() diff --git a/tests/accumulator/utils.py b/tests/accumulator/utils.py new file mode 100644 index 00000000..4a32b9dd --- /dev/null +++ b/tests/accumulator/utils.py @@ -0,0 +1,188 @@ +from datetime import datetime, timezone +from pynumaflow.accumulator import Datum +from pynumaflow.proto.accumulator import accumulator_pb2 +from tests.testing_utils import get_time_args + + +def create_test_datum(keys, value, event_time=None, watermark=None, id_=None, headers=None): + """Create a test Datum object with default values""" + if event_time is None: + event_time = datetime.fromtimestamp(1662998400, timezone.utc) + if watermark is None: + watermark = datetime.fromtimestamp(1662998460, timezone.utc) + if id_ is None: + id_ = "test_id" + if headers is None: + headers = {} + + return Datum( + keys=keys, + value=value, + event_time=event_time, + watermark=watermark, + id_=id_, + headers=headers, + ) + + +def create_accumulator_request(operation, keys, value, event_time=None, watermark=None): + """Create an AccumulatorRequest for testing""" + if event_time is None or watermark is None: + event_time_timestamp, watermark_timestamp = get_time_args() + else: + event_time_timestamp = event_time + watermark_timestamp = watermark + + window = accumulator_pb2.KeyedWindow( + start=event_time_timestamp, + end=watermark_timestamp, + slot="slot-0", + keys=keys, + ) + + payload = accumulator_pb2.Payload( + keys=keys, + value=value, + event_time=event_time_timestamp, + watermark=watermark_timestamp, + id="test_id", + ) + + operation_proto = accumulator_pb2.AccumulatorRequest.WindowOperation( + event=operation, + keyedWindow=window, + ) + + return accumulator_pb2.AccumulatorRequest( + payload=payload, + operation=operation_proto, + ) + + +def create_out_of_order_datums(): + """Create a list of datums that are out of order by event_time""" + return [ + create_test_datum( + keys=["test"], + value=b"event_3", + event_time=datetime.fromtimestamp(1662998460, timezone.utc), + id_="3", + ), + create_test_datum( + keys=["test"], + value=b"event_1", + event_time=datetime.fromtimestamp(1662998400, timezone.utc), + id_="1", + ), + create_test_datum( + keys=["test"], + value=b"event_2", + event_time=datetime.fromtimestamp(1662998430, timezone.utc), + id_="2", + ), + ] + + +def create_multi_source_datums(): + """Create datums from multiple sources for stream joining tests""" + return [ + create_test_datum( + keys=["source1"], + value=b"data_from_source1_1", + event_time=datetime.fromtimestamp(1662998400, timezone.utc), + id_="s1_1", + ), + create_test_datum( + keys=["source2"], + value=b"data_from_source2_1", + event_time=datetime.fromtimestamp(1662998410, timezone.utc), + id_="s2_1", + ), + create_test_datum( + keys=["source1"], + value=b"data_from_source1_2", + event_time=datetime.fromtimestamp(1662998420, timezone.utc), + id_="s1_2", + ), + create_test_datum( + keys=["source2"], + value=b"data_from_source2_2", + event_time=datetime.fromtimestamp(1662998430, timezone.utc), + id_="s2_2", + ), + ] + + +def create_numeric_datums(): + """Create datums with numeric values for trigger tests""" + return [ + create_test_datum( + keys=["test"], + value=b"1", + event_time=datetime.fromtimestamp(1662998400, timezone.utc), + id_="1", + ), + create_test_datum( + keys=["test"], + value=b"2", + event_time=datetime.fromtimestamp(1662998410, timezone.utc), + id_="2", + ), + create_test_datum( + keys=["test"], + value=b"3", + event_time=datetime.fromtimestamp(1662998420, timezone.utc), + id_="3", + ), + create_test_datum( + keys=["test"], + value=b"4", + event_time=datetime.fromtimestamp(1662998430, timezone.utc), + id_="4", + ), + create_test_datum( + keys=["test"], + value=b"5", + event_time=datetime.fromtimestamp(1662998440, timezone.utc), + id_="5", + ), + ] + + +def create_correlation_datums(): + """Create datums for time-based correlation tests""" + base_time = 1662998400 + return [ + # Group 1: events within 5 seconds + create_test_datum( + keys=["correlation"], + value=b"event_a", + event_time=datetime.fromtimestamp(base_time, timezone.utc), + id_="a", + ), + create_test_datum( + keys=["correlation"], + value=b"event_b", + event_time=datetime.fromtimestamp(base_time + 2, timezone.utc), + id_="b", + ), + create_test_datum( + keys=["correlation"], + value=b"event_c", + event_time=datetime.fromtimestamp(base_time + 4, timezone.utc), + id_="c", + ), + # Group 2: events within another 5 second window + create_test_datum( + keys=["correlation"], + value=b"event_d", + event_time=datetime.fromtimestamp(base_time + 15, timezone.utc), + id_="d", + ), + create_test_datum( + keys=["correlation"], + value=b"event_e", + event_time=datetime.fromtimestamp(base_time + 17, timezone.utc), + id_="e", + ), + ] From 0647b3e54c9dd3439da153554253c0aa5a285d08 Mon Sep 17 00:00:00 2001 From: srao12 Date: Tue, 15 Jul 2025 22:58:49 -0700 Subject: [PATCH 06/30] Fix accumulator async runtime issues - Fixed dictionary iteration issue in stream_send_eof method that was causing 'RuntimeError: dictionary changed size during iteration' - Fixed None watermark comparison issues in write_to_global_queue method - Fixed None watermark/event_time conversion to protobuf timestamp - Added proper null checks for watermark and event_time handling - Tests now run without runtime errors and produce expected message count The main fixes: 1. Create copy of dictionary keys before iteration in stream_send_eof 2. Add null checks before watermark comparison and protobuf conversion 3. Handle None values properly in Message watermark/event_time fields Tests are now passing the core functionality (message processing) but EOF handling still needs work. Signed-off-by: srao12 --- pynumaflow/accumulator/__init__.py | 4 +- pynumaflow/accumulator/_dtypes.py | 34 ++++- .../accumulator/servicer/async_servicer.py | 28 ++++- .../accumulator/servicer/task_manager.py | 118 +++++++----------- pynumaflow/shared/server.py | 6 + .../accumulator/test_async_accumulator_err.py | 7 +- tests/accumulator/test_debug_watermark.py | 10 +- 7 files changed, 126 insertions(+), 81 deletions(-) diff --git a/pynumaflow/accumulator/__init__.py b/pynumaflow/accumulator/__init__.py index 85cf335d..c6ea8d0b 100644 --- a/pynumaflow/accumulator/__init__.py +++ b/pynumaflow/accumulator/__init__.py @@ -4,8 +4,8 @@ IntervalWindow, Metadata, DROP, - ReduceStreamer, KeyedWindow, + Accumulator, ) from pynumaflow.accumulator.async_server import AccumulatorAsyncServer @@ -16,6 +16,6 @@ "Metadata", "DROP", "AccumulatorAsyncServer", - "ReduceStreamer", "KeyedWindow", + "Accumulator", ] diff --git a/pynumaflow/accumulator/_dtypes.py b/pynumaflow/accumulator/_dtypes.py index ceda2ddd..fcbd81a1 100644 --- a/pynumaflow/accumulator/_dtypes.py +++ b/pynumaflow/accumulator/_dtypes.py @@ -288,19 +288,31 @@ class Message: value: data in bytes keys: []string keys for vertex (optional) tags: []string tags for conditional forwarding (optional) + watermark: watermark for this message (optional) + event_time: event time for this message (optional) + headers: headers for this message (optional) + id: message id (optional) """ - __slots__ = ("_value", "_keys", "_tags") + __slots__ = ("_value", "_keys", "_tags", "_watermark", "_event_time", "_headers", "_id") _value: bytes _keys: list[str] _tags: list[str] + _watermark: datetime + _event_time: datetime + _headers: dict[str, str] + _id: str def __init__( self, value: bytes, keys: list[str] = None, tags: list[str] = None, + watermark: datetime = None, + event_time: datetime = None, + headers: dict[str, str] = None, + id: str = None, ): """ Creates a Message object to send value to a vertex. @@ -308,6 +320,10 @@ def __init__( self._keys = keys or [] self._tags = tags or [] self._value = value or b"" + self._watermark = watermark + self._event_time = event_time + self._headers = headers or {} + self._id = id or "" # self._window = window or None # returns the Message Object which will be dropped @@ -327,6 +343,22 @@ def keys(self) -> list[str]: def tags(self) -> list[str]: return self._tags + @property + def watermark(self) -> datetime: + return self._watermark + + @property + def event_time(self) -> datetime: + return self._event_time + + @property + def headers(self) -> dict[str, str]: + return self._headers + + @property + def id(self) -> str: + return self._id + AccumulatorAsyncCallable = Callable[ [list[str], AsyncIterable[Datum], NonBlockingIterator, Metadata], None diff --git a/pynumaflow/accumulator/servicer/async_servicer.py b/pynumaflow/accumulator/servicer/async_servicer.py index 726711a5..14c042b5 100644 --- a/pynumaflow/accumulator/servicer/async_servicer.py +++ b/pynumaflow/accumulator/servicer/async_servicer.py @@ -1,6 +1,7 @@ import asyncio from collections.abc import AsyncIterable from typing import Union +import logging from google.protobuf import empty_pb2 as _empty_pb2 @@ -11,6 +12,7 @@ AccumulatorAsyncCallable, _AccumulatorBuilderClass, AccumulatorRequest, + KeyedWindow, ) from pynumaflow.accumulator.servicer.task_manager import TaskManager from pynumaflow.shared.server import handle_async_error @@ -22,18 +24,27 @@ async def datum_generator( ) -> AsyncIterable[AccumulatorRequest]: """Generate a AccumulatorRequest from a AccumulatorRequest proto message.""" async for d in request_iterator: - reduce_request = AccumulatorRequest( + # Convert protobuf KeyedWindow to our KeyedWindow dataclass + keyed_window = KeyedWindow( + start=d.operation.keyedWindow.start.ToDatetime(), + end=d.operation.keyedWindow.end.ToDatetime(), + slot=d.operation.keyedWindow.slot, + keys=list(d.operation.keyedWindow.keys), + ) + + accumulator_request = AccumulatorRequest( operation=d.operation.event, - windows=d.operation.windows, + keyed_window=keyed_window, # Use the new parameter name payload=Datum( keys=list(d.payload.keys), value=d.payload.value, event_time=d.payload.event_time.ToDatetime(), watermark=d.payload.watermark.ToDatetime(), + id_=d.payload.id, # Added missing id field headers=dict(d.payload.headers), ), ) - yield reduce_request + yield accumulator_request class AsyncAccumulatorServicer(accumulator_pb2_grpc.AccumulatorServicer): @@ -69,13 +80,13 @@ async def AccumulateFn( consumer = task_manager.global_result_queue.read_iterator() # Create an async iterator from the request iterator - # datum_iterator = datum_generator(request_iterator=request_iterator) + datum_iterator = datum_generator(request_iterator=request_iterator) # Create a process_input_stream task in the task manager, # this would read from the datum iterator # and then create the required tasks to process the data requests # The results from these tasks are then sent to the result queue - producer = asyncio.create_task(task_manager.process_input_stream(request_iterator)) + producer = asyncio.create_task(task_manager.process_input_stream(datum_iterator)) # Start the consumer task where we read from the result queue # and send the results to the client @@ -94,16 +105,23 @@ async def AccumulateFn( # This is a special message that indicates the end of the processing for a window # When we get this message, we send an EOF message to the client try: + logging.info("[ACCUMULATOR_DEBUG] Starting to read from consumer") async for msg in consumer: + logging.info(f"[ACCUMULATOR_DEBUG] Received message type: {type(msg)}") # If the message is an exception, we raise the exception if isinstance(msg, BaseException): + logging.info(f"[ACCUMULATOR_DEBUG] Found exception: {msg}") + logging.info(f"[ACCUMULATOR_DEBUG] Calling handle_async_error with exception: {repr(msg)}") await handle_async_error(context, msg, ERR_UDF_EXCEPTION_STRING) + logging.info(f"[ACCUMULATOR_DEBUG] Returning after handle_async_error") return # Send window EOF response or Window result response # back to the client else: + logging.info(f"[ACCUMULATOR_DEBUG] Yielding message: {msg}") yield msg except BaseException as e: + logging.info(f"[ACCUMULATOR_DEBUG] Caught exception in try block: {e}") await handle_async_error(context, e, ERR_UDF_EXCEPTION_STRING) return # Wait for the process_input_stream task to finish for a clean exit diff --git a/pynumaflow/accumulator/servicer/task_manager.py b/pynumaflow/accumulator/servicer/task_manager.py index 19053a0d..41eb00bb 100644 --- a/pynumaflow/accumulator/servicer/task_manager.py +++ b/pynumaflow/accumulator/servicer/task_manager.py @@ -4,6 +4,7 @@ from typing import Union import logging +from google.protobuf import timestamp_pb2 from pynumaflow._constants import ( STREAM_EOF, DELIMITER, @@ -93,7 +94,9 @@ async def stream_send_eof(self): tasks that are currently being processed. This is called when the input grpc stream is closed. """ - for unified_key in self.tasks: + # Create a copy of the keys to avoid dictionary size change during iteration + task_keys = list(self.tasks.keys()) + for unified_key in task_keys: await self.tasks[unified_key].iterator.put(STREAM_EOF) self.tasks.pop(unified_key) @@ -304,83 +307,56 @@ async def write_to_global_queue( reader = input_queue.read_iterator() task = self.tasks[unified_key] - _LOGGER.info(f"[WRITE_TO_GLOBAL_DEBUG] Starting write_to_global_queue for key: {unified_key}") - _LOGGER.info(f"[WRITE_TO_GLOBAL_DEBUG] Task: {task}") - _LOGGER.info(f"[WRITE_TO_GLOBAL_DEBUG] Task watermark: {task.latest_watermark}") - _LOGGER.info(f"[WRITE_TO_GLOBAL_DEBUG] Task keys: {task.keys}") - - # Store the last datum processed for this task - we need this for watermark info - last_datum = getattr(task, 'last_datum', None) - _LOGGER.info(f"[WRITE_TO_GLOBAL_DEBUG] Last datum: {last_datum}") - wm: datetime = task.latest_watermark async for msg in reader: - _LOGGER.info(f"[WRITE_TO_GLOBAL_DEBUG] Received message: {msg}") - _LOGGER.info(f"[WRITE_TO_GLOBAL_DEBUG] Message type: {type(msg)}") - _LOGGER.info(f"[WRITE_TO_GLOBAL_DEBUG] Message keys: {msg.keys}") - _LOGGER.info(f"[WRITE_TO_GLOBAL_DEBUG] Message value: {msg.value}") - _LOGGER.info(f"[WRITE_TO_GLOBAL_DEBUG] Message tags: {msg.tags}") + # Convert the window to a datetime object + # Only update watermark if msg.watermark is not None + if msg.watermark is not None and wm < msg.watermark: + task.update_watermark(msg.watermark) + self.tasks[unified_key] = task + wm = msg.watermark + + # Convert datetime to protobuf timestamp + event_time_pb = timestamp_pb2.Timestamp() + if msg.event_time is not None: + event_time_pb.FromDatetime(msg.event_time) + + watermark_pb = timestamp_pb2.Timestamp() + if msg.watermark is not None: + watermark_pb.FromDatetime(msg.watermark) + + start_dt_pb = timestamp_pb2.Timestamp() + start_dt_pb.FromDatetime(datetime.fromtimestamp(0)) - # For now, let's see if we can get the datum info from the task - # The task should have access to the datum information + end_dt_pb = timestamp_pb2.Timestamp() + end_dt_pb.FromDatetime(wm) - # If we have a last_datum, use its watermark and metadata - if last_datum: - _LOGGER.info(f"[WRITE_TO_GLOBAL_DEBUG] Using last_datum watermark: {last_datum.watermark}") - _LOGGER.info(f"[WRITE_TO_GLOBAL_DEBUG] Using last_datum event_time: {last_datum.event_time}") - _LOGGER.info(f"[WRITE_TO_GLOBAL_DEBUG] Using last_datum headers: {last_datum.headers}") - _LOGGER.info(f"[WRITE_TO_GLOBAL_DEBUG] Using last_datum id: {last_datum.id}") - - # Update watermark if the datum's watermark is newer - if wm < last_datum.watermark: - task.update_watermark(last_datum.watermark) - self.tasks[unified_key] = task - wm = last_datum.watermark - - start_dt = datetime.fromtimestamp(0) - end_dt = wm - res = accumulator_pb2.AccumulatorResponse( - payload=accumulator_pb2.Payload( - keys=msg.keys, - value=msg.value, - event_time=last_datum.event_time, - watermark=last_datum.watermark, - headers=last_datum.headers, - id=last_datum.id, - ), - window=accumulator_pb2.KeyedWindow( - start=start_dt, end=end_dt, slot="slot-0", keys=task.keys - ), - EOF=False, - tags=msg.tags, - ) - await output_queue.put(res) - else: - _LOGGER.error("[WRITE_TO_GLOBAL_DEBUG] No last_datum available!") - # This is the problematic code that tries to access msg.watermark - # TODO: We need to fix this by storing datum information properly - start_dt = datetime.fromtimestamp(0) - end_dt = wm - res = accumulator_pb2.AccumulatorResponse( - payload=accumulator_pb2.Payload( - keys=msg.keys, - value=msg.value, - event_time=datetime.now(), # Temporary fallback - watermark=wm, # Use task watermark as fallback - headers={}, # Empty headers as fallback - id="", # Empty id as fallback - ), - window=accumulator_pb2.KeyedWindow( - start=start_dt, end=end_dt, slot="slot-0", keys=task.keys - ), - EOF=False, - tags=msg.tags, - ) - await output_queue.put(res) + res = accumulator_pb2.AccumulatorResponse( + payload=accumulator_pb2.Payload( + keys=msg.keys, + value=msg.value, + event_time=event_time_pb, + watermark=watermark_pb, + headers=msg.headers, + id=msg.id, + ), + window=accumulator_pb2.KeyedWindow( + start=start_dt_pb, end=end_dt_pb, slot="slot-0", keys=task.keys + ), + EOF=False, + tags=msg.tags, + ) + await output_queue.put(res) # send EOF + start_eof_pb = timestamp_pb2.Timestamp() + start_eof_pb.FromDatetime(datetime.fromtimestamp(0)) + + end_eof_pb = timestamp_pb2.Timestamp() + end_eof_pb.FromDatetime(wm) + res = accumulator_pb2.AccumulatorResponse( window=accumulator_pb2.KeyedWindow( - start=datetime.fromtimestamp(0), end=wm, slot="slot-0", keys=task.keys + start=start_eof_pb, end=end_eof_pb, slot="slot-0", keys=task.keys ), EOF=True, ) diff --git a/pynumaflow/shared/server.py b/pynumaflow/shared/server.py index 128b8dc7..942a1e29 100644 --- a/pynumaflow/shared/server.py +++ b/pynumaflow/shared/server.py @@ -317,9 +317,15 @@ async def handle_async_error( """ Handle exceptions for async servers by updating the context and exiting. """ + import logging err_msg = f"{exception_type}: {repr(exception)}" + logging.info(f"[HANDLE_ASYNC_ERROR] Called with exception: {exception}") + logging.info(f"[HANDLE_ASYNC_ERROR] Error message: {err_msg}") update_context_err(context, exception, err_msg) + logging.info(f"[HANDLE_ASYNC_ERROR] Updated context with error") await asyncio.gather( context.abort(grpc.StatusCode.INTERNAL, details=err_msg), return_exceptions=True ) + logging.info(f"[HANDLE_ASYNC_ERROR] Called context.abort") exit_on_error(err=err_msg, parent=False, context=context, update_context=False) + logging.info(f"[HANDLE_ASYNC_ERROR] Called exit_on_error") diff --git a/tests/accumulator/test_async_accumulator_err.py b/tests/accumulator/test_async_accumulator_err.py index 810ee4c6..f31a164a 100644 --- a/tests/accumulator/test_async_accumulator_err.py +++ b/tests/accumulator/test_async_accumulator_err.py @@ -47,6 +47,7 @@ def start_request() -> accumulator_pb2.AccumulatorRequest: event_time=event_time_timestamp, watermark=watermark_timestamp, id="test_id", + headers={"test_header_key": "test_header_value", "source": "test_source"}, ) operation = accumulator_pb2.AccumulatorRequest.WindowOperation( event=accumulator_pb2.AccumulatorRequest.WindowOperation.Event.OPEN, @@ -163,9 +164,13 @@ def test_accumulate_error(self) -> None: # Try to consume the generator counter = 0 - for _ in generator_response: + logging.info(f"[TEST_DEBUG] About to iterate through generator_response") + for response in generator_response: counter += 1 + logging.info(f"[TEST_DEBUG] Received response {counter}: {response}") + logging.info(f"[TEST_DEBUG] Finished iterating, got {counter} responses") except BaseException as err: + logging.info(f"[TEST_DEBUG] Caught exception: {err}") self.assertTrue("Simulated error in accumulator handler" in str(err)) return self.fail("Expected an exception.") diff --git a/tests/accumulator/test_debug_watermark.py b/tests/accumulator/test_debug_watermark.py index b28d440b..8a088f44 100644 --- a/tests/accumulator/test_debug_watermark.py +++ b/tests/accumulator/test_debug_watermark.py @@ -88,7 +88,15 @@ async def handler( msg = f"counter:{self.counter}" logging.info(f"[HANDLER_DEBUG] Putting message: {msg}") - message_obj = Message(str.encode(msg), keys=datum.keys(), tags=[]) + message_obj = Message( + str.encode(msg), + keys=datum.keys(), + tags=[], + watermark=datum.watermark, + event_time=datum.event_time, + headers=datum.headers, + id=datum.id + ) logging.info(f"[HANDLER_DEBUG] Message object: {message_obj}") logging.info(f"[HANDLER_DEBUG] Message has watermark: {hasattr(message_obj, 'watermark')}") logging.info(f"[HANDLER_DEBUG] Message has event_time: {hasattr(message_obj, 'event_time')}") From 464411c2ff5b65e3eea976f6459ef3bd3ea96631 Mon Sep 17 00:00:00 2001 From: srao12 Date: Wed, 16 Jul 2025 01:41:07 -0700 Subject: [PATCH 07/30] Fix accumulator tests and improve error handling - Fix missing default value in AccumulatorRequest constructor - Add watermark parameter handling in AccumulatorRequest - Improve test structure and remove debug test file - Fix task_manager.py to handle EOF counting properly - All 253 tests now pass successfully Signed-off-by: srao12 --- .../accumulator/servicer/task_manager.py | 32 ++- tests/accumulator/test_async_accumulator.py | 2 +- tests/accumulator/test_datatypes.py | 6 +- tests/accumulator/test_debug_watermark.py | 202 ------------------ 4 files changed, 35 insertions(+), 207 deletions(-) delete mode 100644 tests/accumulator/test_debug_watermark.py diff --git a/pynumaflow/accumulator/servicer/task_manager.py b/pynumaflow/accumulator/servicer/task_manager.py index 41eb00bb..0be68219 100644 --- a/pynumaflow/accumulator/servicer/task_manager.py +++ b/pynumaflow/accumulator/servicer/task_manager.py @@ -65,6 +65,11 @@ def __init__(self, handler: Union[AccumulatorAsyncCallable, _AccumulatorBuilderC # This queue is also used to send the error/exceptions to the client # if the reduce operation fails. self.global_result_queue = NonBlockingIterator() + # EOF response counting to ensure proper termination + self._expected_eof_count = 0 + self._received_eof_count = 0 + self._eof_count_lock = asyncio.Lock() + self._stream_termination_event = asyncio.Event() def get_unique_windows(self): """ @@ -163,6 +168,11 @@ async def create_task(self, req): # Save the result of the reduce operation to the task list self.tasks[unified_key] = curr_task + + # Increment expected EOF count since we created a new task + async with self._eof_count_lock: + self._expected_eof_count += 1 + logging.info(f"[EOF_COUNT_DEBUG] Task created. Expected EOF count: {self._expected_eof_count}") # Put the request in the iterator await curr_task.iterator.put(d) @@ -289,7 +299,13 @@ async def process_input_stream( # eof_window_msg = create_window_eof_response(window=window) # await self.global_result_queue.put(eof_window_msg) - # Once all tasks are completed, senf EOF the global result queue + # Wait for all tasks to send their EOF responses before terminating the stream + # This ensures proper ordering: all messages -> all EOF responses -> STREAM_EOF + logging.info("[PROCESS_INPUT_DEBUG] All tasks completed, waiting for EOF responses") + await self._stream_termination_event.wait() + + # Now send STREAM_EOF to terminate the global result queue iterator + logging.info("[PROCESS_INPUT_DEBUG] All EOF responses received, sending STREAM_EOF") await self.global_result_queue.put(STREAM_EOF) except BaseException as e: err_msg = f"Reduce Streaming Error: {repr(e)}" @@ -361,6 +377,20 @@ async def write_to_global_queue( EOF=True, ) await output_queue.put(res) + + # Increment received EOF count and check if all tasks are done + async with self._eof_count_lock: + self._received_eof_count += 1 + logging.info(f"[EOF_COUNT_DEBUG] EOF response sent. Received EOF count: {self._received_eof_count}/{self._expected_eof_count}") + + # Check if all tasks have sent their EOF responses + if self._received_eof_count == self._expected_eof_count: + logging.info("[EOF_COUNT_DEBUG] All EOF responses received, setting termination event") + self._stream_termination_event.set() + elif self._received_eof_count > self._expected_eof_count: + logging.error(f"[EOF_COUNT_DEBUG] ERROR: Received more EOF responses ({self._received_eof_count}) than expected ({self._expected_eof_count})") + # Still set the event to prevent hanging, but log the error + self._stream_termination_event.set() def clean_background(self, task): """ diff --git a/tests/accumulator/test_async_accumulator.py b/tests/accumulator/test_async_accumulator.py index 1d0fb96f..acdd73bf 100644 --- a/tests/accumulator/test_async_accumulator.py +++ b/tests/accumulator/test_async_accumulator.py @@ -223,7 +223,7 @@ def test_accumulate_with_multiple_keys(self) -> None: # We should have 10 messages (one for each key) self.assertEqual(10, count) - self.assertEqual(1, eof_count) + self.assertEqual(10, eof_count) # Each key/task sends its own EOF # Each key should appear once self.assertEqual(len(key_counts), 10) diff --git a/tests/accumulator/test_datatypes.py b/tests/accumulator/test_datatypes.py index 1bd37691..877ee5ee 100644 --- a/tests/accumulator/test_datatypes.py +++ b/tests/accumulator/test_datatypes.py @@ -206,7 +206,7 @@ def test_update_watermark_invalid_type(self): class TestAccumulatorRequest(unittest.TestCase): def test_create_request(self): operation = WindowOperation.OPEN - windows = [KeyedWindow(start=mock_start_time(), end=mock_end_time())] + keyed_window = KeyedWindow(start=mock_start_time(), end=mock_end_time()) payload = Datum( keys=TEST_KEYS, value=mock_message(), @@ -215,9 +215,9 @@ def test_create_request(self): id_=TEST_ID, ) - request = AccumulatorRequest(operation, windows, payload) + request = AccumulatorRequest(operation, keyed_window, payload) self.assertEqual(request.operation, operation) - self.assertEqual(request.windows, windows) + self.assertEqual(request.keyed_window, keyed_window) self.assertEqual(request.payload, payload) diff --git a/tests/accumulator/test_debug_watermark.py b/tests/accumulator/test_debug_watermark.py deleted file mode 100644 index 8a088f44..00000000 --- a/tests/accumulator/test_debug_watermark.py +++ /dev/null @@ -1,202 +0,0 @@ -import asyncio -import logging -import threading -import unittest -from collections.abc import AsyncIterable -from unittest.mock import patch - -import grpc -from grpc.aio._server import Server - -from pynumaflow import setup_logging -from pynumaflow.accumulator import ( - Message, - Datum, - AccumulatorAsyncServer, - Accumulator, -) -from pynumaflow.proto.accumulator import accumulator_pb2, accumulator_pb2_grpc -from pynumaflow.shared.asynciter import NonBlockingIterator -from tests.testing_utils import ( - mock_message, - mock_interval_window_start, - mock_interval_window_end, - get_time_args, - mock_terminate_on_stop, -) - -LOGGER = setup_logging(__name__) - - -def request_generator(count, request): - for i in range(count): - yield request - - -def start_request() -> accumulator_pb2.AccumulatorRequest: - event_time_timestamp, watermark_timestamp = get_time_args() - window = accumulator_pb2.KeyedWindow( - start=event_time_timestamp, - end=watermark_timestamp, - slot="slot-0", - keys=["test_key"], - ) - payload = accumulator_pb2.Payload( - keys=["test_key"], - value=mock_message(), - event_time=event_time_timestamp, - watermark=watermark_timestamp, - id="test_id", - headers={"test_header_key": "test_header_value", "source": "test_source"}, - ) - operation = accumulator_pb2.AccumulatorRequest.WindowOperation( - event=accumulator_pb2.AccumulatorRequest.WindowOperation.Event.OPEN, - keyedWindow=window, - ) - request = accumulator_pb2.AccumulatorRequest( - payload=payload, - operation=operation, - ) - return request - - -_s: Server = None -_channel = grpc.insecure_channel("unix:///tmp/accumulator_err.sock") -_loop = None - - -def startup_callable(loop): - asyncio.set_event_loop(loop) - loop.run_forever() - - -class ExampleErrorClass(Accumulator): - def __init__(self, counter): - self.counter = counter - - async def handler( - self, datums: AsyncIterable[Datum], output: NonBlockingIterator - ): - logging.info(f"[HANDLER_DEBUG] Handler called with counter: {self.counter}") - async for datum in datums: - self.counter += 1 - logging.info(f"[HANDLER_DEBUG] Processing datum {self.counter}: {datum}") - logging.info(f"[HANDLER_DEBUG] Datum watermark: {datum.watermark}") - logging.info(f"[HANDLER_DEBUG] Datum event_time: {datum.event_time}") - logging.info(f"[HANDLER_DEBUG] Datum headers: {datum.headers}") - logging.info(f"[HANDLER_DEBUG] Datum id: {datum.id}") - - msg = f"counter:{self.counter}" - logging.info(f"[HANDLER_DEBUG] Putting message: {msg}") - message_obj = Message( - str.encode(msg), - keys=datum.keys(), - tags=[], - watermark=datum.watermark, - event_time=datum.event_time, - headers=datum.headers, - id=datum.id - ) - logging.info(f"[HANDLER_DEBUG] Message object: {message_obj}") - logging.info(f"[HANDLER_DEBUG] Message has watermark: {hasattr(message_obj, 'watermark')}") - logging.info(f"[HANDLER_DEBUG] Message has event_time: {hasattr(message_obj, 'event_time')}") - logging.info(f"[HANDLER_DEBUG] Message has headers: {hasattr(message_obj, 'headers')}") - logging.info(f"[HANDLER_DEBUG] Message has id: {hasattr(message_obj, 'id')}") - - await output.put(message_obj) - logging.info(f"[HANDLER_DEBUG] Handler finished processing all datums") - - -async def error_accumulator_handler_func( - datums: AsyncIterable[Datum], output: NonBlockingIterator -): - counter = 0 - async for datum in datums: - counter += 1 - if counter == 2: - # Simulate an error on the second datum - raise RuntimeError("Simulated error in accumulator function") - msg = f"counter:{counter}" - await output.put( - Message(str.encode(msg), keys=datum.keys(), tags=[]) - ) - - -def NewAsyncAccumulatorError(): - server_instance = AccumulatorAsyncServer(ExampleErrorClass, init_args=(0,)) - udfs = server_instance.servicer - return udfs - - -@patch("psutil.Process.kill", mock_terminate_on_stop) -async def start_server(udfs): - server = grpc.aio.server() - accumulator_pb2_grpc.add_AccumulatorServicer_to_server(udfs, server) - listen_addr = "unix:///tmp/accumulator_err.sock" - server.add_insecure_port(listen_addr) - logging.info("Starting server on %s", listen_addr) - global _s - _s = server - await server.start() - await server.wait_for_termination() - -@patch("psutil.Process.kill", mock_terminate_on_stop) -class TestDebugWatermark(unittest.TestCase): - @classmethod - def setUpClass(cls) -> None: - global _loop - loop = asyncio.new_event_loop() - _loop = loop - _thread = threading.Thread(target=startup_callable, args=(loop,), daemon=True) - _thread.start() - udfs = NewAsyncAccumulatorError() - asyncio.run_coroutine_threadsafe(start_server(udfs), loop=loop) - while True: - try: - with grpc.insecure_channel("unix:///tmp/accumulator_err.sock") as channel: - f = grpc.channel_ready_future(channel) - f.result(timeout=10) - if f.done(): - break - except grpc.FutureTimeoutError as e: - LOGGER.error("error trying to connect to grpc server") - LOGGER.error(e) - - @classmethod - def tearDownClass(cls) -> None: - try: - _loop.stop() - LOGGER.info("stopped the event loop") - except Exception as e: - LOGGER.error(e) - - @patch("psutil.Process.kill", mock_terminate_on_stop) - def test_debug_watermark_flow(self) -> None: - """Debug test to investigate watermark information flow""" - stub = self.__stub() - request = start_request() - - try: - generator_response = stub.AccumulateFn( - request_iterator=request_generator(count=2, request=request) - ) - - # Try to consume the generator - counter = 0 - logging.info(f"[TEST_DEBUG] About to iterate through generator_response") - for response in generator_response: - counter += 1 - logging.info(f"[TEST_DEBUG] Received response {counter}: {response}") - logging.info(f"[TEST_DEBUG] Finished iterating, got {counter} responses") - except BaseException as err: - logging.info(f"[TEST_DEBUG] Caught exception: {err}") - # For debug purposes, we expect to see the watermark error - self.assertTrue("watermark" in str(err).lower() or "attribute" in str(err).lower()) - - def __stub(self): - return accumulator_pb2_grpc.AccumulatorStub(_channel) - - -if __name__ == "__main__": - logging.basicConfig(level=logging.DEBUG) - unittest.main() From 19d61a3276d85e37891b112d7440e955763c23f9 Mon Sep 17 00:00:00 2001 From: srao12 Date: Wed, 16 Jul 2025 01:58:45 -0700 Subject: [PATCH 08/30] Fix merge conflicts Signed-off-by: srao12 --- pynumaflow/accumulator/_dtypes.py | 27 +++++++------------ .../accumulator/servicer/task_manager.py | 27 +++++++++---------- 2 files changed, 23 insertions(+), 31 deletions(-) diff --git a/pynumaflow/accumulator/_dtypes.py b/pynumaflow/accumulator/_dtypes.py index 29be0cc0..fcbd81a1 100644 --- a/pynumaflow/accumulator/_dtypes.py +++ b/pynumaflow/accumulator/_dtypes.py @@ -3,7 +3,7 @@ from dataclasses import dataclass from datetime import datetime from enum import IntEnum -from typing import TypeVar, Callable, Union, Optional, Type +from typing import TypeVar, Callable, Union, Optional from collections.abc import AsyncIterable from pynumaflow.shared.asynciter import NonBlockingIterator @@ -314,30 +314,22 @@ def __init__( headers: dict[str, str] = None, id: str = None, ): - self._value = value or b"" + """ + Creates a Message object to send value to a vertex. + """ self._keys = keys or [] self._tags = tags or [] + self._value = value or b"" self._watermark = watermark self._event_time = event_time self._headers = headers or {} self._id = id or "" + # self._window = window or None + # returns the Message Object which will be dropped @classmethod - def to_drop(cls: Type[M]) -> M: - return cls(b"", None, ["DROP"]) - - @classmethod - def from_datum(cls: Type[M], datum: "Datum") -> M: - """Creates a Message from a Datum.""" - return cls( - value=datum.value, - keys=datum.keys, - tags=[], - event_time=datum.event_time, - watermark=datum.watermark, - id=datum.id, - headers=datum.headers, - ) + def to_drop(cls: type[M]) -> M: + return cls(b"", None, [DROP]) @property def value(self) -> bytes: @@ -367,6 +359,7 @@ def headers(self) -> dict[str, str]: def id(self) -> str: return self._id + AccumulatorAsyncCallable = Callable[ [list[str], AsyncIterable[Datum], NonBlockingIterator, Metadata], None ] diff --git a/pynumaflow/accumulator/servicer/task_manager.py b/pynumaflow/accumulator/servicer/task_manager.py index 2330f36e..0be68219 100644 --- a/pynumaflow/accumulator/servicer/task_manager.py +++ b/pynumaflow/accumulator/servicer/task_manager.py @@ -19,7 +19,6 @@ ) from pynumaflow.proto.accumulator import accumulator_pb2 from pynumaflow.shared.asynciter import NonBlockingIterator -from google.protobuf import timestamp_pb2 as _timestamp_pb2 def build_unique_key_name(keys): @@ -104,11 +103,11 @@ async def stream_send_eof(self): task_keys = list(self.tasks.keys()) for unified_key in task_keys: await self.tasks[unified_key].iterator.put(STREAM_EOF) - self.tasks.clear() + self.tasks.pop(unified_key) async def close_task(self, req): d = req.payload - keys = d.keys + keys = d.keys() unified_key = build_unique_key_name(keys) curr_task = self.tasks.get(unified_key, None) @@ -128,7 +127,7 @@ async def create_task(self, req): it creates a new task or appends the request to the existing task. """ d = req.payload - keys = d.keys + keys = d.keys() unified_key = build_unique_key_name(keys) curr_task = self.tasks.get(unified_key, None) @@ -193,9 +192,9 @@ async def send_datum_to_task(self, req): await result.iterator.put(d) async def __invoke_accumulator( - self, - request_iterator: AsyncIterable[Datum], - output: NonBlockingIterator, + self, + request_iterator: AsyncIterable[Datum], + output: NonBlockingIterator, ): """ Invokes the UDF reduce handler with the given keys, @@ -225,7 +224,7 @@ async def __invoke_accumulator( logging.info(f"[ACCUMULATOR_DEBUG] Exception put in global_result_queue successfully") async def process_input_stream( - self, request_iterator: AsyncIterable[accumulator_pb2.AccumulatorRequest] + self, request_iterator: AsyncIterable[accumulator_pb2.AccumulatorRequest] ): # Start iterating through the request iterator and create tasks # based on the operation type received. @@ -241,18 +240,17 @@ async def process_input_stream( logging.info(f"[PROCESS_INPUT_DEBUG] WindowOperation.CLOSE: {int(WindowOperation.CLOSE)}") logging.info(f"[PROCESS_INPUT_DEBUG] Comparison - request.operation is int(WindowOperation.OPEN): {request.operation is int(WindowOperation.OPEN)}") # check whether the request is an open or append operation - if request.operation.event is int(WindowOperation.OPEN): - + if request.operation is int(WindowOperation.OPEN): # create a new task for the open operation and # put the request in the task iterator logging.info(f"[PROCESS_INPUT_DEBUG] Creating task for OPEN operation") await self.create_task(request) - elif request.operation.event is int(WindowOperation.APPEND): + elif request.operation is int(WindowOperation.APPEND): # append the task data to the existing task # if the task does not exist, create a new task logging.info(f"[PROCESS_INPUT_DEBUG] Sending datum to task for APPEND operation") await self.send_datum_to_task(request) - elif request.operation.event is int(WindowOperation.CLOSE): + elif request.operation is int(WindowOperation.CLOSE): # close the current task for req logging.info(f"[PROCESS_INPUT_DEBUG] Closing task for CLOSE operation") await self.close_task(request) @@ -315,7 +313,7 @@ async def process_input_stream( await self.global_result_queue.put(e) async def write_to_global_queue( - self, input_queue: NonBlockingIterator, output_queue: NonBlockingIterator, unified_key: str + self, input_queue: NonBlockingIterator, output_queue: NonBlockingIterator, unified_key: str ): """ This task is for given Reduce task. @@ -325,7 +323,7 @@ async def write_to_global_queue( reader = input_queue.read_iterator() task = self.tasks[unified_key] - wm = task.latest_watermark + wm: datetime = task.latest_watermark async for msg in reader: # Convert the window to a datetime object # Only update watermark if msg.watermark is not None @@ -348,6 +346,7 @@ async def write_to_global_queue( end_dt_pb = timestamp_pb2.Timestamp() end_dt_pb.FromDatetime(wm) + res = accumulator_pb2.AccumulatorResponse( payload=accumulator_pb2.Payload( keys=msg.keys, From 10ebf4610d252e7ca705a3b924a46d23a0cd6003 Mon Sep 17 00:00:00 2001 From: srao12 Date: Wed, 16 Jul 2025 21:18:06 -0700 Subject: [PATCH 09/30] fix: example Signed-off-by: srao12 --- examples/accumulator/streamsorter/Makefile | 4 +- examples/accumulator/streamsorter/example.py | 17 +++---- .../accumulator/streamsorter/pyproject.toml | 2 - .../streamsorter/stream-sorter-pl.yaml | 45 +++++++++++++++++++ poetry.lock | 24 +++++----- 5 files changed, 68 insertions(+), 24 deletions(-) create mode 100644 examples/accumulator/streamsorter/stream-sorter-pl.yaml diff --git a/examples/accumulator/streamsorter/Makefile b/examples/accumulator/streamsorter/Makefile index f36656b2..29fe2f92 100644 --- a/examples/accumulator/streamsorter/Makefile +++ b/examples/accumulator/streamsorter/Makefile @@ -1,6 +1,6 @@ TAG ?= test1 PUSH ?= false -IMAGE_REGISTRY = quay.io/numaio/numaflow-python/accumulator-sorter:${TAG} +IMAGE_REGISTRY = docker.intuit.com/personal/srao12/accumulator-sorter:${TAG} DOCKER_FILE_PATH = examples/accumulator/streamsorter/Dockerfile .PHONY: update @@ -12,7 +12,7 @@ image-push: update cd ../../../ && docker buildx build \ -f ${DOCKER_FILE_PATH} \ -t ${IMAGE_REGISTRY} \ - --platform linux/amd64,linux/arm64 . --push + --platform linux/amd64 . --push .PHONY: image image: update diff --git a/examples/accumulator/streamsorter/example.py b/examples/accumulator/streamsorter/example.py index dbab4953..347620e5 100644 --- a/examples/accumulator/streamsorter/example.py +++ b/examples/accumulator/streamsorter/example.py @@ -19,7 +19,7 @@ class StreamSorter(Accumulator): def __init__(self): - _LOGGER.error("MEEEEE") + _LOGGER.info("StreamSorter initialized") self.latest_wm = datetime.fromtimestamp(-1) self.sorted_buffer: List[Datum] = [] @@ -28,14 +28,15 @@ async def handler( datums: AsyncIterable[Datum], output: NonBlockingIterator, ): - _LOGGER.info("HEREEEEE") + _LOGGER.info("StreamSorter handler started") async for datum in datums: - _LOGGER.info(f"Received datum with event time: {datum.watermark}") - _LOGGER.info(f"Received datum with event time-2:{self.latest_wm}") + _LOGGER.info(f"Received datum with event time: {datum.event_time}") + _LOGGER.info(f"Current latest watermark: {self.latest_wm}") + _LOGGER.info(f"Datum watermark: {datum.watermark}") # If watermark has moved forward - if datum.watermark.ToDatetime() and datum.watermark.ToDatetime() > self.latest_wm: - self.latest_wm = datum.watermark.ToDatetime() + if datum.watermark and datum.watermark > self.latest_wm: + self.latest_wm = datum.watermark await self.flush_buffer(output) self.insert_sorted(datum) @@ -45,7 +46,7 @@ def insert_sorted(self, datum: Datum): left, right = 0, len(self.sorted_buffer) while left < right: mid = (left + right) // 2 - if self.sorted_buffer[mid].event_time.ToDatetime() > datum.event_time.ToDatetime(): + if self.sorted_buffer[mid].event_time > datum.event_time: right = mid else: left = mid + 1 @@ -58,7 +59,7 @@ async def flush_buffer(self, output: NonBlockingIterator): if datum.event_time > self.latest_wm: break await output.put(Message.from_datum(datum)) - logging.info(f"Sent datum with event time: {datum.watermark.ToDatetime()}") + _LOGGER.info(f"Sent datum with event time: {datum.event_time}") i += 1 # Remove flushed items self.sorted_buffer = self.sorted_buffer[i:] diff --git a/examples/accumulator/streamsorter/pyproject.toml b/examples/accumulator/streamsorter/pyproject.toml index 6557f78f..99d64cd1 100644 --- a/examples/accumulator/streamsorter/pyproject.toml +++ b/examples/accumulator/streamsorter/pyproject.toml @@ -8,8 +8,6 @@ authors = ["Numaflow developers"] python = ">=3.11,<3.13" pynumaflow = { path = "../../../"} -[tool.poetry.dev-dependencies] - [build-system] requires = ["poetry-core>=1.0.0"] build-backend = "poetry.core.masonry.api" diff --git a/examples/accumulator/streamsorter/stream-sorter-pl.yaml b/examples/accumulator/streamsorter/stream-sorter-pl.yaml new file mode 100644 index 00000000..02ed5633 --- /dev/null +++ b/examples/accumulator/streamsorter/stream-sorter-pl.yaml @@ -0,0 +1,45 @@ +apiVersion: numaflow.numaproj.io/v1alpha1 +kind: Pipeline +metadata: + name: stream-sorter +spec: + limits: + readBatchSize: 1 + vertices: + - name: http-one + scale: + min: 1 + max: 1 + source: + http: {} + - name: http-two + scale: + min: 1 + max: 1 + source: + http: {} + - name: accum + udf: + container: + image: docker.intuit.com/personal/srao12/accumulator-sorter:test1 + groupBy: + window: + accumulator: + timeout: 10s + keyed: true + storage: + persistentVolumeClaim: + volumeSize: 1Gi + - name: sink + scale: + min: 1 + max: 1 + sink: + log: {} + edges: + - from: http-one + to: accum + - from: http-two + to: accum + - from: accum + to: sink diff --git a/poetry.lock b/poetry.lock index 53bad6b7..1a2c8e7b 100644 --- a/poetry.lock +++ b/poetry.lock @@ -1,4 +1,4 @@ -# This file is automatically @generated by Poetry 2.0.1 and should not be changed by hand. +# This file is automatically @generated by Poetry 2.1.1 and should not be changed by hand. [[package]] name = "aiorun" @@ -58,7 +58,7 @@ typing-extensions = {version = ">=4.0.1", markers = "python_version < \"3.11\""} [package.extras] colorama = ["colorama (>=0.4.3)"] -d = ["aiohttp (>=3.7.4)", "aiohttp (>=3.7.4,!=3.9.0)"] +d = ["aiohttp (>=3.7.4) ; sys_platform != \"win32\" or implementation_name != \"pypy\"", "aiohttp (>=3.7.4,!=3.9.0) ; sys_platform == \"win32\" and implementation_name == \"pypy\""] jupyter = ["ipython (>=7.8.0)", "tokenize-rt (>=3.2.0)"] uvloop = ["uvloop (>=0.15.2)"] @@ -304,7 +304,7 @@ files = [ tomli = {version = "*", optional = true, markers = "python_full_version <= \"3.11.0a6\" and extra == \"toml\""} [package.extras] -toml = ["tomli"] +toml = ["tomli ; python_full_version <= \"3.11.0a6\""] [[package]] name = "distlib" @@ -349,7 +349,7 @@ files = [ [package.extras] docs = ["furo (>=2024.8.6)", "sphinx (>=8.1.3)", "sphinx-autodoc-typehints (>=3)"] testing = ["covdefaults (>=2.3)", "coverage (>=7.6.10)", "diff-cover (>=9.2.1)", "pytest (>=8.3.4)", "pytest-asyncio (>=0.25.2)", "pytest-cov (>=6)", "pytest-mock (>=3.14)", "pytest-timeout (>=2.3.1)", "virtualenv (>=20.28.1)"] -typing = ["typing-extensions (>=4.12.2)"] +typing = ["typing-extensions (>=4.12.2) ; python_version < \"3.11\""] [[package]] name = "google-api-core" @@ -372,7 +372,7 @@ requests = ">=2.18.0,<3.0.0.dev0" [package.extras] async-rest = ["google-auth[aiohttp] (>=2.35.0,<3.0.dev0)"] -grpc = ["grpcio (>=1.33.2,<2.0dev)", "grpcio (>=1.49.1,<2.0dev)", "grpcio-status (>=1.33.2,<2.0.dev0)", "grpcio-status (>=1.49.1,<2.0.dev0)"] +grpc = ["grpcio (>=1.33.2,<2.0dev)", "grpcio (>=1.49.1,<2.0dev) ; python_version >= \"3.11\"", "grpcio-status (>=1.33.2,<2.0.dev0)", "grpcio-status (>=1.49.1,<2.0.dev0) ; python_version >= \"3.11\""] grpcgcp = ["grpcio-gcp (>=0.2.2,<1.0.dev0)"] grpcio-gcp = ["grpcio-gcp (>=0.2.2,<1.0.dev0)"] @@ -1016,13 +1016,13 @@ files = [ ] [package.extras] -check = ["pytest-checkdocs (>=2.4)", "pytest-ruff (>=0.2.1)", "ruff (>=0.8.0)"] -core = ["importlib_metadata (>=6)", "jaraco.collections", "jaraco.functools (>=4)", "jaraco.text (>=3.7)", "more_itertools", "more_itertools (>=8.8)", "packaging", "packaging (>=24.2)", "platformdirs (>=4.2.2)", "tomli (>=2.0.1)", "wheel (>=0.43.0)"] +check = ["pytest-checkdocs (>=2.4)", "pytest-ruff (>=0.2.1) ; sys_platform != \"cygwin\"", "ruff (>=0.8.0) ; sys_platform != \"cygwin\""] +core = ["importlib_metadata (>=6) ; python_version < \"3.10\"", "jaraco.collections", "jaraco.functools (>=4)", "jaraco.text (>=3.7)", "more_itertools", "more_itertools (>=8.8)", "packaging", "packaging (>=24.2)", "platformdirs (>=4.2.2)", "tomli (>=2.0.1) ; python_version < \"3.11\"", "wheel (>=0.43.0)"] cover = ["pytest-cov"] doc = ["furo", "jaraco.packaging (>=9.3)", "jaraco.tidelift (>=1.4)", "pygments-github-lexers (==0.0.5)", "pyproject-hooks (!=1.1)", "rst.linker (>=1.9)", "sphinx (>=3.5)", "sphinx-favicon", "sphinx-inline-tabs", "sphinx-lint", "sphinx-notfound-page (>=1,<2)", "sphinx-reredirects", "sphinxcontrib-towncrier", "towncrier (<24.7)"] enabler = ["pytest-enabler (>=2.2)"] -test = ["build[virtualenv] (>=1.0.3)", "filelock (>=3.4.0)", "ini2toml[lite] (>=0.14)", "jaraco.develop (>=7.21)", "jaraco.envs (>=2.2)", "jaraco.path (>=3.7.2)", "jaraco.test (>=5.5)", "packaging (>=24.2)", "pip (>=19.1)", "pyproject-hooks (!=1.1)", "pytest (>=6,!=8.1.*)", "pytest-home (>=0.5)", "pytest-perf", "pytest-subprocess", "pytest-timeout", "pytest-xdist (>=3)", "tomli-w (>=1.0.0)", "virtualenv (>=13.0.0)", "wheel (>=0.44.0)"] -type = ["importlib_metadata (>=7.0.2)", "jaraco.develop (>=7.21)", "mypy (==1.14.*)", "pytest-mypy"] +test = ["build[virtualenv] (>=1.0.3)", "filelock (>=3.4.0)", "ini2toml[lite] (>=0.14)", "jaraco.develop (>=7.21) ; python_version >= \"3.9\" and sys_platform != \"cygwin\"", "jaraco.envs (>=2.2)", "jaraco.path (>=3.7.2)", "jaraco.test (>=5.5)", "packaging (>=24.2)", "pip (>=19.1)", "pyproject-hooks (!=1.1)", "pytest (>=6,!=8.1.*)", "pytest-home (>=0.5)", "pytest-perf ; sys_platform != \"cygwin\"", "pytest-subprocess", "pytest-timeout", "pytest-xdist (>=3)", "tomli-w (>=1.0.0)", "virtualenv (>=13.0.0)", "wheel (>=0.44.0)"] +type = ["importlib_metadata (>=7.0.2) ; python_version < \"3.10\"", "jaraco.develop (>=7.21) ; sys_platform != \"cygwin\"", "mypy (==1.14.*)", "pytest-mypy"] [[package]] name = "tomli" @@ -1093,7 +1093,7 @@ files = [ ] [package.extras] -brotli = ["brotli (>=1.0.9)", "brotlicffi (>=0.8.0)"] +brotli = ["brotli (>=1.0.9) ; platform_python_implementation == \"CPython\"", "brotlicffi (>=0.8.0) ; platform_python_implementation != \"CPython\""] h2 = ["h2 (>=4,<5)"] socks = ["pysocks (>=1.5.6,!=1.5.7,<2.0)"] zstd = ["zstandard (>=0.18.0)"] @@ -1141,7 +1141,7 @@ files = [ [package.extras] docs = ["Sphinx (>=4.1.2,<4.2.0)", "sphinx-rtd-theme (>=0.5.2,<0.6.0)", "sphinxcontrib-asyncio (>=0.3.0,<0.4.0)"] -test = ["Cython (>=0.29.36,<0.30.0)", "aiohttp (==3.9.0b0)", "aiohttp (>=3.8.1)", "flake8 (>=5.0,<6.0)", "mypy (>=0.800)", "psutil", "pyOpenSSL (>=23.0.0,<23.1.0)", "pycodestyle (>=2.9.0,<2.10.0)"] +test = ["Cython (>=0.29.36,<0.30.0)", "aiohttp (==3.9.0b0) ; python_version >= \"3.12\"", "aiohttp (>=3.8.1) ; python_version < \"3.12\"", "flake8 (>=5.0,<6.0)", "mypy (>=0.800)", "psutil", "pyOpenSSL (>=23.0.0,<23.1.0)", "pycodestyle (>=2.9.0,<2.10.0)"] [[package]] name = "virtualenv" @@ -1162,7 +1162,7 @@ platformdirs = ">=3.9.1,<5" [package.extras] docs = ["furo (>=2023.7.26)", "proselint (>=0.13)", "sphinx (>=7.1.2,!=7.3)", "sphinx-argparse (>=0.4)", "sphinxcontrib-towncrier (>=0.2.1a0)", "towncrier (>=23.6)"] -test = ["covdefaults (>=2.3)", "coverage (>=7.2.7)", "coverage-enable-subprocess (>=1)", "flaky (>=3.7)", "packaging (>=23.1)", "pytest (>=7.4)", "pytest-env (>=0.8.2)", "pytest-freezer (>=0.4.8)", "pytest-mock (>=3.11.1)", "pytest-randomly (>=3.12)", "pytest-timeout (>=2.1)", "setuptools (>=68)", "time-machine (>=2.10)"] +test = ["covdefaults (>=2.3)", "coverage (>=7.2.7)", "coverage-enable-subprocess (>=1)", "flaky (>=3.7)", "packaging (>=23.1)", "pytest (>=7.4)", "pytest-env (>=0.8.2)", "pytest-freezer (>=0.4.8) ; platform_python_implementation == \"PyPy\" or platform_python_implementation == \"CPython\" and sys_platform == \"win32\" and python_version >= \"3.13\"", "pytest-mock (>=3.11.1)", "pytest-randomly (>=3.12)", "pytest-timeout (>=2.1)", "setuptools (>=68)", "time-machine (>=2.10) ; platform_python_implementation == \"CPython\""] [metadata] lock-version = "2.1" From 32fda32cd756e95410d237f70869ad351b8eae7f Mon Sep 17 00:00:00 2001 From: srao12 Date: Sat, 19 Jul 2025 18:11:46 -0700 Subject: [PATCH 10/30] fix: tests and add e2e test Signed-off-by: srao12 --- examples/accumulator/streamsorter/Makefile | 6 +- examples/accumulator/streamsorter/README.md | 43 +++++++++++ examples/accumulator/streamsorter/example.py | 2 +- .../streamsorter/stream-sorter-pl.yaml | 45 +++++++++-- pynumaflow/accumulator/_dtypes.py | 19 +++++ tests/accumulator/test_datatypes.py | 76 +++++++++++++++++++ 6 files changed, 180 insertions(+), 11 deletions(-) create mode 100644 examples/accumulator/streamsorter/README.md diff --git a/examples/accumulator/streamsorter/Makefile b/examples/accumulator/streamsorter/Makefile index 29fe2f92..52c62e3e 100644 --- a/examples/accumulator/streamsorter/Makefile +++ b/examples/accumulator/streamsorter/Makefile @@ -1,6 +1,6 @@ -TAG ?= test1 +TAG ?= test3 PUSH ?= false -IMAGE_REGISTRY = docker.intuit.com/personal/srao12/accumulator-sorter:${TAG} +IMAGE_REGISTRY = docker.intuit.com/personal/srao12/py-accumulator-sorter:${TAG} DOCKER_FILE_PATH = examples/accumulator/streamsorter/Dockerfile .PHONY: update @@ -12,7 +12,7 @@ image-push: update cd ../../../ && docker buildx build \ -f ${DOCKER_FILE_PATH} \ -t ${IMAGE_REGISTRY} \ - --platform linux/amd64 . --push + --platform linux/amd64,linux/arm64 . --push .PHONY: image image: update diff --git a/examples/accumulator/streamsorter/README.md b/examples/accumulator/streamsorter/README.md new file mode 100644 index 00000000..26d913a4 --- /dev/null +++ b/examples/accumulator/streamsorter/README.md @@ -0,0 +1,43 @@ +# Stream Sorter + +An example User Defined Function that sorts the incoming stream by event time. + +### Applying the Pipeline + +To apply the pipeline, use the following command: + +```shell + kubectl apply -f manifest/stream-sorter-pipeline.yaml +``` + +### Publish messages + +Port-forward the HTTP endpoint, and make POST requests using curl. Remember to replace xxxx with the appropriate pod names. + +```shell + kubectl port-forward stream-sorter-http-one-0-xxxx 8444:8443 + + # Post data to the HTTP endpoint + curl -kq -X POST -d "101" https://localhost:8444/vertices/http-one -H "X-Numaflow-Event-Time: 60000" + curl -kq -X POST -d "102" https://localhost:8444/vertices/http-one -H "X-Numaflow-Event-Time: 61000" + curl -kq -X POST -d "103" https://localhost:8444/vertices/http-one -H "X-Numaflow-Event-Time: 62000" + curl -kq -X POST -d "104" https://localhost:8444/vertices/http-one -H "X-Numaflow-Event-Time: 63000" +``` + +```shell + kubectl port-forward stream-sorter-http-two-0-xxxx 8445:8443 + + # Post data to the HTTP endpoint + curl -kq -X POST -d "105" https://localhost:8445/vertices/http-two -H "X-Numaflow-Event-Time: 70000" + curl -kq -X POST -d "106" https://localhost:8445/vertices/http-two -H "X-Numaflow-Event-Time: 71000" + curl -kq -X POST -d "107" https://localhost:8445/vertices/http-two -H "X-Numaflow-Event-Time: 72000" + curl -kq -X POST -d "108" https://localhost:8445/vertices/http-two -H "X-Numaflow-Event-Time: 73000" +``` + +### Verify the output + +```shell + kubectl logs -f stream-sorter-log-sink-0-xxxx +``` + +The output should be sorted by event time. \ No newline at end of file diff --git a/examples/accumulator/streamsorter/example.py b/examples/accumulator/streamsorter/example.py index 347620e5..2b2771a7 100644 --- a/examples/accumulator/streamsorter/example.py +++ b/examples/accumulator/streamsorter/example.py @@ -6,7 +6,7 @@ from pynumaflow import setup_logging from pynumaflow.accumulator import Accumulator, AccumulatorAsyncServer -from pynumaflow.reducestreamer import ( +from pynumaflow.accumulator import ( Message, Datum, ) diff --git a/examples/accumulator/streamsorter/stream-sorter-pl.yaml b/examples/accumulator/streamsorter/stream-sorter-pl.yaml index 02ed5633..64e46368 100644 --- a/examples/accumulator/streamsorter/stream-sorter-pl.yaml +++ b/examples/accumulator/streamsorter/stream-sorter-pl.yaml @@ -18,10 +18,11 @@ spec: max: 1 source: http: {} - - name: accum + - name: go-accum udf: container: - image: docker.intuit.com/personal/srao12/accumulator-sorter:test1 + image: quay.io/numaio/numaflow-go/stream-sorter:stable + imagePullPolicy: Always groupBy: window: accumulator: @@ -30,7 +31,31 @@ spec: storage: persistentVolumeClaim: volumeSize: 1Gi - - name: sink + - name: py-accum + udf: + container: + image: docker.intuit.com/personal/srao12/py-accumulator-sorter:test3 + imagePullPolicy: Always + env: + - name: PYTHONDEBUG + value: "true" + - name: INVOKE + value: "class" + groupBy: + window: + accumulator: + timeout: 10s + keyed: true + storage: + persistentVolumeClaim: + volumeSize: 1Gi + - name: go-sink + scale: + min: 1 + max: 1 + sink: + log: {} + - name: py-sink scale: min: 1 max: 1 @@ -38,8 +63,14 @@ spec: log: {} edges: - from: http-one - to: accum + to: go-accum + - from: http-two + to: go-accum + - from: http-one + to: py-accum - from: http-two - to: accum - - from: accum - to: sink + to: py-accum + - from: go-accum + to: go-sink + - from: py-accum + to: py-sink diff --git a/pynumaflow/accumulator/_dtypes.py b/pynumaflow/accumulator/_dtypes.py index fcbd81a1..b3d8c61b 100644 --- a/pynumaflow/accumulator/_dtypes.py +++ b/pynumaflow/accumulator/_dtypes.py @@ -359,6 +359,25 @@ def headers(self) -> dict[str, str]: def id(self) -> str: return self._id + @classmethod + def from_datum(cls, datum: Datum): + """Create a Message instance from a Datum object. + + Args: + datum: The Datum object to convert + + Returns: + Message: A new Message instance with data from the datum + """ + return cls( + value=datum.value, + keys=datum.keys(), + watermark=datum.watermark, + event_time=datum.event_time, + headers=datum.headers, + id=datum.id + ) + AccumulatorAsyncCallable = Callable[ [list[str], AsyncIterable[Datum], NonBlockingIterator, Metadata], None diff --git a/tests/accumulator/test_datatypes.py b/tests/accumulator/test_datatypes.py index 877ee5ee..2f25504c 100644 --- a/tests/accumulator/test_datatypes.py +++ b/tests/accumulator/test_datatypes.py @@ -256,6 +256,82 @@ def test_none_values(self): self.assertEqual(msg.keys, []) self.assertEqual(msg.tags, []) + def test_from_datum(self): + """Test that Message.from_datum correctly creates a Message from a Datum""" + # Create a sample datum with all properties + test_keys = ["key1", "key2"] + test_value = b"test_message_value" + test_event_time = mock_event_time() + test_watermark = mock_watermark() + test_headers = {"header1": "value1", "header2": "value2"} + test_id = "test_datum_id" + + datum = Datum( + keys=test_keys, + value=test_value, + event_time=test_event_time, + watermark=test_watermark, + id_=test_id, + headers=test_headers, + ) + + # Create message from datum + message = Message.from_datum(datum) + + # Verify all properties are correctly transferred + self.assertEqual(message.value, test_value) + self.assertEqual(message.keys, test_keys) + self.assertEqual(message.event_time, test_event_time) + self.assertEqual(message.watermark, test_watermark) + self.assertEqual(message.headers, test_headers) + self.assertEqual(message.id, test_id) + + # Verify that tags are empty (default for Message) + self.assertEqual(message.tags, []) + + def test_from_datum_minimal(self): + """Test from_datum with minimal Datum (no headers)""" + test_keys = ["minimal_key"] + test_value = b"minimal_value" + test_event_time = mock_event_time() + test_watermark = mock_watermark() + test_id = "minimal_id" + + datum = Datum( + keys=test_keys, + value=test_value, + event_time=test_event_time, + watermark=test_watermark, + id_=test_id, + # headers not provided (will default to {}) + ) + + message = Message.from_datum(datum) + + self.assertEqual(message.value, test_value) + self.assertEqual(message.keys, test_keys) + self.assertEqual(message.event_time, test_event_time) + self.assertEqual(message.watermark, test_watermark) + self.assertEqual(message.headers, {}) + self.assertEqual(message.id, test_id) + self.assertEqual(message.tags, []) + + def test_from_datum_empty_keys(self): + """Test from_datum with empty keys""" + datum = Datum( + keys=None, # Will default to [] + value=b"test_value", + event_time=mock_event_time(), + watermark=mock_watermark(), + id_="test_id", + ) + + message = Message.from_datum(datum) + + self.assertEqual(message.keys, []) + self.assertEqual(message.value, b"test_value") + self.assertEqual(message.id, "test_id") + class TestAccumulatorClass(unittest.TestCase): class ExampleClass(Accumulator): From b4d7ccc7a8450d205579275a894f96e36b8ccf3d Mon Sep 17 00:00:00 2001 From: srao12 Date: Sun, 20 Jul 2025 16:45:53 -0700 Subject: [PATCH 11/30] fix: cleanup logs Signed-off-by: srao12 --- examples/accumulator/streamsorter/Makefile | 4 +- examples/accumulator/streamsorter/README.md | 2 +- .../accumulator/streamsorter/pipeline.yaml | 50 ++++++------ .../streamsorter/stream-sorter-pl.yaml | 76 ------------------- .../accumulator/servicer/task_manager.py | 36 +-------- pynumaflow/shared/server.py | 6 -- 6 files changed, 31 insertions(+), 143 deletions(-) delete mode 100644 examples/accumulator/streamsorter/stream-sorter-pl.yaml diff --git a/examples/accumulator/streamsorter/Makefile b/examples/accumulator/streamsorter/Makefile index 52c62e3e..5eb6a3e8 100644 --- a/examples/accumulator/streamsorter/Makefile +++ b/examples/accumulator/streamsorter/Makefile @@ -1,6 +1,6 @@ -TAG ?= test3 +TAG ?= stable PUSH ?= false -IMAGE_REGISTRY = docker.intuit.com/personal/srao12/py-accumulator-sorter:${TAG} +IMAGE_REGISTRY = quay.io/numaio/numaflow-python/streamsorter:${TAG} DOCKER_FILE_PATH = examples/accumulator/streamsorter/Dockerfile .PHONY: update diff --git a/examples/accumulator/streamsorter/README.md b/examples/accumulator/streamsorter/README.md index 26d913a4..19b8da6e 100644 --- a/examples/accumulator/streamsorter/README.md +++ b/examples/accumulator/streamsorter/README.md @@ -7,7 +7,7 @@ An example User Defined Function that sorts the incoming stream by event time. To apply the pipeline, use the following command: ```shell - kubectl apply -f manifest/stream-sorter-pipeline.yaml + kubectl apply -f pipeline.yaml ``` ### Publish messages diff --git a/examples/accumulator/streamsorter/pipeline.yaml b/examples/accumulator/streamsorter/pipeline.yaml index 604e6997..c13416f6 100644 --- a/examples/accumulator/streamsorter/pipeline.yaml +++ b/examples/accumulator/streamsorter/pipeline.yaml @@ -1,25 +1,27 @@ apiVersion: numaflow.numaproj.io/v1alpha1 kind: Pipeline metadata: - name: even-odd-sum + name: stream-sorter spec: + limits: + readBatchSize: 1 vertices: - - name: in + - name: http-one + scale: + min: 1 + max: 1 source: http: {} - - name: atoi + - name: http-two scale: - min: 3 - udf: - container: - # Tell the input number is even or odd, see https://github.com/numaproj/numaflow-go/tree/main/pkg/mapper/examples/even_odd - image: quay.io/numaio/numaflow-go/map-even-odd:stable - imagePullPolicy: Always - - name: compute-sum + min: 1 + max: 1 + source: + http: {} + - name: py-accum udf: container: - # compute the sum - image: quay.io/numaio/numaflow-python/reduce-stream-sum:stable + image: quay.io/numaio/numaflow-python/streamsorter:stable imagePullPolicy: Always env: - name: PYTHONDEBUG @@ -28,24 +30,22 @@ spec: value: "class" groupBy: window: - fixed: - length: 60s - streaming: true + accumulator: + timeout: 10s keyed: true storage: persistentVolumeClaim: - volumeSize: 10Gi - accessMode: ReadWriteOnce - partitions: 1 - - name: sink + volumeSize: 1Gi + - name: py-sink scale: min: 1 + max: 1 sink: log: {} edges: - - from: in - to: atoi - - from: atoi - to: compute-sum - - from: compute-sum - to: sink + - from: http-one + to: py-accum + - from: http-two + to: py-accum + - from: py-accum + to: py-sink diff --git a/examples/accumulator/streamsorter/stream-sorter-pl.yaml b/examples/accumulator/streamsorter/stream-sorter-pl.yaml deleted file mode 100644 index 64e46368..00000000 --- a/examples/accumulator/streamsorter/stream-sorter-pl.yaml +++ /dev/null @@ -1,76 +0,0 @@ -apiVersion: numaflow.numaproj.io/v1alpha1 -kind: Pipeline -metadata: - name: stream-sorter -spec: - limits: - readBatchSize: 1 - vertices: - - name: http-one - scale: - min: 1 - max: 1 - source: - http: {} - - name: http-two - scale: - min: 1 - max: 1 - source: - http: {} - - name: go-accum - udf: - container: - image: quay.io/numaio/numaflow-go/stream-sorter:stable - imagePullPolicy: Always - groupBy: - window: - accumulator: - timeout: 10s - keyed: true - storage: - persistentVolumeClaim: - volumeSize: 1Gi - - name: py-accum - udf: - container: - image: docker.intuit.com/personal/srao12/py-accumulator-sorter:test3 - imagePullPolicy: Always - env: - - name: PYTHONDEBUG - value: "true" - - name: INVOKE - value: "class" - groupBy: - window: - accumulator: - timeout: 10s - keyed: true - storage: - persistentVolumeClaim: - volumeSize: 1Gi - - name: go-sink - scale: - min: 1 - max: 1 - sink: - log: {} - - name: py-sink - scale: - min: 1 - max: 1 - sink: - log: {} - edges: - - from: http-one - to: go-accum - - from: http-two - to: go-accum - - from: http-one - to: py-accum - - from: http-two - to: py-accum - - from: go-accum - to: go-sink - - from: py-accum - to: py-sink diff --git a/pynumaflow/accumulator/servicer/task_manager.py b/pynumaflow/accumulator/servicer/task_manager.py index 0be68219..d718e716 100644 --- a/pynumaflow/accumulator/servicer/task_manager.py +++ b/pynumaflow/accumulator/servicer/task_manager.py @@ -2,7 +2,6 @@ from collections.abc import AsyncIterable from datetime import datetime from typing import Union -import logging from google.protobuf import timestamp_pb2 from pynumaflow._constants import ( @@ -172,7 +171,6 @@ async def create_task(self, req): # Increment expected EOF count since we created a new task async with self._eof_count_lock: self._expected_eof_count += 1 - logging.info(f"[EOF_COUNT_DEBUG] Task created. Expected EOF count: {self._expected_eof_count}") # Put the request in the iterator await curr_task.iterator.put(d) @@ -217,46 +215,33 @@ async def __invoke_accumulator( # then send the error to the result queue except BaseException as err: _LOGGER.critical("panic inside accumulator handle", exc_info=True) - logging.info(f"[ACCUMULATOR_DEBUG] Exception caught in __invoke_accumulator: {err}") - logging.info(f"[ACCUMULATOR_DEBUG] Putting exception in global_result_queue: {repr(err)}") # Put the exception in the result queue await self.global_result_queue.put(err) - logging.info(f"[ACCUMULATOR_DEBUG] Exception put in global_result_queue successfully") async def process_input_stream( self, request_iterator: AsyncIterable[accumulator_pb2.AccumulatorRequest] ): # Start iterating through the request iterator and create tasks # based on the operation type received. - logging.info(f"[PROCESS_INPUT_DEBUG] Starting process_input_stream") try: request_count = 0 async for request in request_iterator: request_count += 1 - logging.info(f"[PROCESS_INPUT_DEBUG] Processing request {request_count}, operation: {request.operation}") - logging.info(f"[PROCESS_INPUT_DEBUG] Operation value: {request.operation}") - logging.info(f"[PROCESS_INPUT_DEBUG] WindowOperation.OPEN: {int(WindowOperation.OPEN)}") - logging.info(f"[PROCESS_INPUT_DEBUG] WindowOperation.APPEND: {int(WindowOperation.APPEND)}") - logging.info(f"[PROCESS_INPUT_DEBUG] WindowOperation.CLOSE: {int(WindowOperation.CLOSE)}") - logging.info(f"[PROCESS_INPUT_DEBUG] Comparison - request.operation is int(WindowOperation.OPEN): {request.operation is int(WindowOperation.OPEN)}") # check whether the request is an open or append operation if request.operation is int(WindowOperation.OPEN): # create a new task for the open operation and # put the request in the task iterator - logging.info(f"[PROCESS_INPUT_DEBUG] Creating task for OPEN operation") await self.create_task(request) elif request.operation is int(WindowOperation.APPEND): # append the task data to the existing task # if the task does not exist, create a new task - logging.info(f"[PROCESS_INPUT_DEBUG] Sending datum to task for APPEND operation") await self.send_datum_to_task(request) elif request.operation is int(WindowOperation.CLOSE): # close the current task for req - logging.info(f"[PROCESS_INPUT_DEBUG] Closing task for CLOSE operation") await self.close_task(request) else: - logging.info(f"[PROCESS_INPUT_DEBUG] No operation matched") - logging.info(f"[PROCESS_INPUT_DEBUG] Finished processing {request_count} requests") + _LOGGER.debug(f"No operation matched for request: {request}", exc_info=True) + # If there is an error in the reduce operation, log and # then send the error to the result queue except BaseException as e: @@ -289,23 +274,11 @@ async def process_input_stream( con_future = task.consumer_future await con_future - # # Once all tasks are completed, send EOF to all windows that - # # were processed in the Task Manager. We send a single - # # EOF message per window. - # current_windows = self.get_unique_windows() - # for window in current_windows.values(): - # # Send an EOF message to the global result queue - # # This will signal that window has been processed - # eof_window_msg = create_window_eof_response(window=window) - # await self.global_result_queue.put(eof_window_msg) - # Wait for all tasks to send their EOF responses before terminating the stream # This ensures proper ordering: all messages -> all EOF responses -> STREAM_EOF - logging.info("[PROCESS_INPUT_DEBUG] All tasks completed, waiting for EOF responses") await self._stream_termination_event.wait() # Now send STREAM_EOF to terminate the global result queue iterator - logging.info("[PROCESS_INPUT_DEBUG] All EOF responses received, sending STREAM_EOF") await self.global_result_queue.put(STREAM_EOF) except BaseException as e: err_msg = f"Reduce Streaming Error: {repr(e)}" @@ -381,14 +354,11 @@ async def write_to_global_queue( # Increment received EOF count and check if all tasks are done async with self._eof_count_lock: self._received_eof_count += 1 - logging.info(f"[EOF_COUNT_DEBUG] EOF response sent. Received EOF count: {self._received_eof_count}/{self._expected_eof_count}") - + # Check if all tasks have sent their EOF responses if self._received_eof_count == self._expected_eof_count: - logging.info("[EOF_COUNT_DEBUG] All EOF responses received, setting termination event") self._stream_termination_event.set() elif self._received_eof_count > self._expected_eof_count: - logging.error(f"[EOF_COUNT_DEBUG] ERROR: Received more EOF responses ({self._received_eof_count}) than expected ({self._expected_eof_count})") # Still set the event to prevent hanging, but log the error self._stream_termination_event.set() diff --git a/pynumaflow/shared/server.py b/pynumaflow/shared/server.py index 942a1e29..128b8dc7 100644 --- a/pynumaflow/shared/server.py +++ b/pynumaflow/shared/server.py @@ -317,15 +317,9 @@ async def handle_async_error( """ Handle exceptions for async servers by updating the context and exiting. """ - import logging err_msg = f"{exception_type}: {repr(exception)}" - logging.info(f"[HANDLE_ASYNC_ERROR] Called with exception: {exception}") - logging.info(f"[HANDLE_ASYNC_ERROR] Error message: {err_msg}") update_context_err(context, exception, err_msg) - logging.info(f"[HANDLE_ASYNC_ERROR] Updated context with error") await asyncio.gather( context.abort(grpc.StatusCode.INTERNAL, details=err_msg), return_exceptions=True ) - logging.info(f"[HANDLE_ASYNC_ERROR] Called context.abort") exit_on_error(err=err_msg, parent=False, context=context, update_context=False) - logging.info(f"[HANDLE_ASYNC_ERROR] Called exit_on_error") From 35156ba3bfbc96cea9b994c3b241ce451e57f60e Mon Sep 17 00:00:00 2001 From: srao12 Date: Sun, 20 Jul 2025 17:06:03 -0700 Subject: [PATCH 12/30] fix: use optimized Dockerfile Signed-off-by: srao12 --- examples/accumulator/counter/Dockerfile | 55 -------------- examples/accumulator/counter/Makefile | 22 ------ examples/accumulator/counter/entry.sh | 4 - examples/accumulator/counter/example.py | 46 ------------ examples/accumulator/counter/pipeline.yaml | 50 ------------- examples/accumulator/counter/pyproject.toml | 15 ---- examples/accumulator/streamsorter/Dockerfile | 75 ++++++++++--------- .../streamsorter/Makefile.optimized | 52 +++++++++++++ 8 files changed, 90 insertions(+), 229 deletions(-) delete mode 100644 examples/accumulator/counter/Dockerfile delete mode 100644 examples/accumulator/counter/Makefile delete mode 100644 examples/accumulator/counter/entry.sh delete mode 100644 examples/accumulator/counter/example.py delete mode 100644 examples/accumulator/counter/pipeline.yaml delete mode 100644 examples/accumulator/counter/pyproject.toml create mode 100644 examples/accumulator/streamsorter/Makefile.optimized diff --git a/examples/accumulator/counter/Dockerfile b/examples/accumulator/counter/Dockerfile deleted file mode 100644 index de1756fd..00000000 --- a/examples/accumulator/counter/Dockerfile +++ /dev/null @@ -1,55 +0,0 @@ -#################################################################################################### -# builder: install needed dependencies -#################################################################################################### - -FROM python:3.10-slim-bullseye AS builder - -ENV PYTHONFAULTHANDLER=1 \ - PYTHONUNBUFFERED=1 \ - PYTHONHASHSEED=random \ - PIP_NO_CACHE_DIR=on \ - PIP_DISABLE_PIP_VERSION_CHECK=on \ - PIP_DEFAULT_TIMEOUT=100 \ - POETRY_VERSION=1.2.2 \ - POETRY_HOME="/opt/poetry" \ - POETRY_VIRTUALENVS_IN_PROJECT=true \ - POETRY_NO_INTERACTION=1 \ - PYSETUP_PATH="/opt/pysetup" - -ENV EXAMPLE_PATH="$PYSETUP_PATH/examples/reducestream/counter" -ENV VENV_PATH="$EXAMPLE_PATH/.venv" -ENV PATH="$POETRY_HOME/bin:$VENV_PATH/bin:$PATH" - -RUN apt-get update \ - && apt-get install --no-install-recommends -y \ - curl \ - wget \ - # deps for building python deps - build-essential \ - && apt-get install -y git \ - && apt-get clean && rm -rf /var/lib/apt/lists/* \ - \ - # install dumb-init - && wget -O /dumb-init https://github.com/Yelp/dumb-init/releases/download/v1.2.5/dumb-init_1.2.5_x86_64 \ - && chmod +x /dumb-init \ - && curl -sSL https://install.python-poetry.org | python3 - - -#################################################################################################### -# udf: used for running the udf vertices -#################################################################################################### -FROM builder AS udf - -WORKDIR $PYSETUP_PATH -COPY ./ ./ - -WORKDIR $EXAMPLE_PATH -RUN poetry lock -RUN poetry install --no-cache --no-root && \ - rm -rf ~/.cache/pypoetry/ - -RUN chmod +x entry.sh - -ENTRYPOINT ["/dumb-init", "--"] -CMD ["sh", "-c", "$EXAMPLE_PATH/entry.sh"] - -EXPOSE 5000 diff --git a/examples/accumulator/counter/Makefile b/examples/accumulator/counter/Makefile deleted file mode 100644 index ba3e8793..00000000 --- a/examples/accumulator/counter/Makefile +++ /dev/null @@ -1,22 +0,0 @@ -TAG ?= stable -PUSH ?= false -IMAGE_REGISTRY = quay.io/numaio/numaflow-python/reduce-stream-counter:${TAG} -DOCKER_FILE_PATH = examples/reducestream/counter/Dockerfile - -.PHONY: update -update: - poetry update -vv - -.PHONY: image-push -image-push: update - cd ../../../ && docker buildx build \ - -f ${DOCKER_FILE_PATH} \ - -t ${IMAGE_REGISTRY} \ - --platform linux/amd64,linux/arm64 . --push - -.PHONY: image -image: update - cd ../../../ && docker build \ - -f ${DOCKER_FILE_PATH} \ - -t ${IMAGE_REGISTRY} . - @if [ "$(PUSH)" = "true" ]; then docker push ${IMAGE_REGISTRY}; fi diff --git a/examples/accumulator/counter/entry.sh b/examples/accumulator/counter/entry.sh deleted file mode 100644 index 073b05e3..00000000 --- a/examples/accumulator/counter/entry.sh +++ /dev/null @@ -1,4 +0,0 @@ -#!/bin/sh -set -eux - -python example.py diff --git a/examples/accumulator/counter/example.py b/examples/accumulator/counter/example.py deleted file mode 100644 index 405d7f7a..00000000 --- a/examples/accumulator/counter/example.py +++ /dev/null @@ -1,46 +0,0 @@ -import os -from collections.abc import AsyncIterable - -from pynumaflow.reducestreamer import ( - Message, - Datum, - Metadata, - ReduceStreamAsyncServer, - ReduceStreamer, -) -from pynumaflow.shared.asynciter import NonBlockingIterator - - -class ReduceCounter(ReduceStreamer): - def __init__(self, counter): - self.counter = counter - - async def handler( - self, - keys: list[str], - datums: AsyncIterable[Datum], - output: NonBlockingIterator, - md: Metadata, - ): - async for _ in datums: - self.counter += 1 - if self.counter > 10: - msg = f"counter:{self.counter}" - # NOTE: this is returning results because we have seen all the data - # use this only if you really need this feature because your next vertex - # will get both early result and final results and it should be able to - # handle both the scenarios. - await output.put(Message(str.encode(msg), keys=keys)) - self.counter = 0 - msg = f"counter:{self.counter}" - await output.put(Message(str.encode(msg), keys=keys)) - - -if __name__ == "__main__": - invoke = os.getenv("INVOKE", "class") - if invoke == "class": - # Here we are using the class instance as the reducer_instance - # which will be used to invoke the handler function. - # We are passing the init_args for the class instance. - grpc_server = ReduceStreamAsyncServer(ReduceCounter, init_args=(0,)) - grpc_server.start() diff --git a/examples/accumulator/counter/pipeline.yaml b/examples/accumulator/counter/pipeline.yaml deleted file mode 100644 index 5ac746c9..00000000 --- a/examples/accumulator/counter/pipeline.yaml +++ /dev/null @@ -1,50 +0,0 @@ -apiVersion: numaflow.numaproj.io/v1alpha1 -kind: Pipeline -metadata: - name: even-odd-sum -spec: - vertices: - - name: in - source: - http: {} - - name: atoi - scale: - min: 3 - udf: - container: - # Tell the input number is even or odd, see https://github.com/numaproj/numaflow-go/tree/main/pkg/mapper/examples/even_odd - image: quay.io/numaio/numaflow-go/map-even-odd:stable - imagePullPolicy: Always - - name: compute-sum - udf: - container: - # compute the sum - image: quay.io/numaio/numaflow-python/reduce-stream-counter:stable - imagePullPolicy: Always - env: - - name: PYTHONDEBUG - value: "true" - - name: INVOKE - value: "class" - groupBy: - window: - fixed: - length: 60s - keyed: true - storage: - persistentVolumeClaim: - volumeSize: 10Gi - accessMode: ReadWriteOnce - partitions: 1 - - name: sink - scale: - min: 1 - sink: - log: {} - edges: - - from: in - to: atoi - - from: atoi - to: compute-sum - - from: compute-sum - to: sink diff --git a/examples/accumulator/counter/pyproject.toml b/examples/accumulator/counter/pyproject.toml deleted file mode 100644 index aeeb4d30..00000000 --- a/examples/accumulator/counter/pyproject.toml +++ /dev/null @@ -1,15 +0,0 @@ -[tool.poetry] -name = "reduce-stream-counter" -version = "0.2.4" -description = "" -authors = ["Numaflow developers"] - -[tool.poetry.dependencies] -python = "~3.10" -pynumaflow = { path = "../../../"} - -[tool.poetry.dev-dependencies] - -[build-system] -requires = ["poetry-core>=1.0.0"] -build-backend = "poetry.core.masonry.api" diff --git a/examples/accumulator/streamsorter/Dockerfile b/examples/accumulator/streamsorter/Dockerfile index c5c1bda6..cbcbf214 100644 --- a/examples/accumulator/streamsorter/Dockerfile +++ b/examples/accumulator/streamsorter/Dockerfile @@ -1,55 +1,56 @@ #################################################################################################### -# builder: install needed dependencies +# Stage 1: Base Builder - installs core dependencies using poetry #################################################################################################### +FROM python:3.10-slim-bullseye AS base-builder -FROM python:3.11-slim-bullseye AS builder +ENV PYSETUP_PATH="/opt/pysetup" +WORKDIR $PYSETUP_PATH + +# Copy only core dependency files first for better caching +COPY pyproject.toml poetry.lock README.md ./ +COPY pynumaflow/ ./pynumaflow/ +RUN echo "Simulating long build step..." && sleep 20 +RUN apt-get update && apt-get install --no-install-recommends -y \ + curl wget build-essential git \ + && apt-get clean && rm -rf /var/lib/apt/lists/* \ + && pip install poetry \ + && poetry install --no-root --no-interaction + +#################################################################################################### +# Stage 2: UDF Builder - adds UDF code and installs UDF-specific deps +#################################################################################################### +FROM base-builder AS udf-builder + +ENV EXAMPLE_PATH="/opt/pysetup/examples/accumulator/streamsorter" +ENV POETRY_VIRTUALENVS_IN_PROJECT=true + +WORKDIR $EXAMPLE_PATH +COPY examples/accumulator/streamsorter/ ./ +RUN poetry install --no-root --no-interaction -ENV PYTHONFAULTHANDLER=1 \ - PYTHONUNBUFFERED=1 \ - PYTHONHASHSEED=random \ - PIP_NO_CACHE_DIR=on \ - PIP_DISABLE_PIP_VERSION_CHECK=on \ - PIP_DEFAULT_TIMEOUT=100 \ - POETRY_VERSION=1.2.2 \ - POETRY_HOME="/opt/poetry" \ - POETRY_VIRTUALENVS_IN_PROJECT=true \ - POETRY_NO_INTERACTION=1 \ - PYSETUP_PATH="/opt/pysetup" +#################################################################################################### +# Stage 3: UDF Runtime - clean container with only needed stuff +#################################################################################################### +FROM python:3.10-slim-bullseye AS udf +ENV PYSETUP_PATH="/opt/pysetup" ENV EXAMPLE_PATH="$PYSETUP_PATH/examples/accumulator/streamsorter" ENV VENV_PATH="$EXAMPLE_PATH/.venv" -ENV PATH="$POETRY_HOME/bin:$VENV_PATH/bin:$PATH" - -RUN apt-get update \ - && apt-get install --no-install-recommends -y \ - curl \ - wget \ - # deps for building python deps - build-essential \ - && apt-get install -y git \ +ENV PATH="$VENV_PATH/bin:$PATH" + +RUN apt-get update && apt-get install --no-install-recommends -y wget \ && apt-get clean && rm -rf /var/lib/apt/lists/* \ - \ - # install dumb-init && wget -O /dumb-init https://github.com/Yelp/dumb-init/releases/download/v1.2.5/dumb-init_1.2.5_x86_64 \ - && chmod +x /dumb-init \ - && curl -sSL https://install.python-poetry.org | python3 - - -#################################################################################################### -# udf: used for running the udf vertices -#################################################################################################### -FROM builder AS udf + && chmod +x /dumb-init WORKDIR $PYSETUP_PATH -COPY ./ ./ +COPY --from=udf-builder $VENV_PATH $VENV_PATH +COPY --from=udf-builder $EXAMPLE_PATH $EXAMPLE_PATH WORKDIR $EXAMPLE_PATH -RUN poetry lock -RUN poetry install --no-cache --no-root && \ - rm -rf ~/.cache/pypoetry/ - RUN chmod +x entry.sh ENTRYPOINT ["/dumb-init", "--"] CMD ["sh", "-c", "$EXAMPLE_PATH/entry.sh"] -EXPOSE 5000 +EXPOSE 5000 \ No newline at end of file diff --git a/examples/accumulator/streamsorter/Makefile.optimized b/examples/accumulator/streamsorter/Makefile.optimized new file mode 100644 index 00000000..f0e4439c --- /dev/null +++ b/examples/accumulator/streamsorter/Makefile.optimized @@ -0,0 +1,52 @@ +TAG ?= stable +PUSH ?= false +IMAGE_REGISTRY = quay.io/numaio/numaflow-python/streamsorter:${TAG} +DOCKER_FILE_PATH = examples/map/even_odd/Dockerfile.optimized +BASE_IMAGE_NAME = numaflow-python-base + +.PHONY: base-image +base-image: + @echo "Building shared base image..." + docker build -f Dockerfile.base -t ${BASE_IMAGE_NAME} . + +.PHONY: update +update: + poetry update -vv + +.PHONY: image-push +image-push: base-image update + cd ../../../ && docker buildx build \ + -f ${DOCKER_FILE_PATH} \ + -t ${IMAGE_REGISTRY} \ + --platform linux/amd64,linux/arm64 . --push + +.PHONY: image +image: base-image update + cd ../../../ && docker build \ + -f ${DOCKER_FILE_PATH} \ + -t ${IMAGE_REGISTRY} . + @if [ "$(PUSH)" = "true" ]; then docker push ${IMAGE_REGISTRY}; fi + +.PHONY: image-fast +image-fast: update + @echo "Building with shared base image (fastest option)..." + cd ../../../ && docker build \ + -f examples/map/even_odd/Dockerfile.shared-base \ + -t ${IMAGE_REGISTRY} . + @if [ "$(PUSH)" = "true" ]; then docker push ${IMAGE_REGISTRY}; fi + +.PHONY: clean +clean: + docker rmi ${BASE_IMAGE_NAME} 2>/dev/null || true + docker rmi ${IMAGE_REGISTRY} 2>/dev/null || true + +.PHONY: help +help: + @echo "Available targets:" + @echo " base-image - Build the shared base image with pynumaflow" + @echo " image - Build UDF image with optimized multi-stage build" + @echo " image-fast - Build UDF image using shared base (fastest)" + @echo " image-push - Build and push multi-platform image" + @echo " update - Update poetry dependencies" + @echo " clean - Remove built images" + @echo " help - Show this help message" \ No newline at end of file From ea5576cd7a4bdcd62ac680e50a37c5bc19481a28 Mon Sep 17 00:00:00 2001 From: srao12 Date: Sun, 20 Jul 2025 17:15:27 -0700 Subject: [PATCH 13/30] fix: lint Signed-off-by: srao12 --- examples/accumulator/streamsorter/example.py | 3 +- pynumaflow/accumulator/_dtypes.py | 6 +- .../accumulator/servicer/async_servicer.py | 10 +--- .../accumulator/servicer/task_manager.py | 18 +++--- tests/accumulator/test_async_accumulator.py | 30 ++++------ .../accumulator/test_async_accumulator_err.py | 29 ++++------ tests/accumulator/test_datatypes.py | 35 +++++------- tests/accumulator/test_use_cases.py | 55 ++++++++++--------- tests/accumulator/utils.py | 10 ++-- 9 files changed, 84 insertions(+), 112 deletions(-) diff --git a/examples/accumulator/streamsorter/example.py b/examples/accumulator/streamsorter/example.py index 2b2771a7..e699ce54 100644 --- a/examples/accumulator/streamsorter/example.py +++ b/examples/accumulator/streamsorter/example.py @@ -2,7 +2,6 @@ import os from collections.abc import AsyncIterable from datetime import datetime -from typing import List from pynumaflow import setup_logging from pynumaflow.accumulator import Accumulator, AccumulatorAsyncServer @@ -21,7 +20,7 @@ class StreamSorter(Accumulator): def __init__(self): _LOGGER.info("StreamSorter initialized") self.latest_wm = datetime.fromtimestamp(-1) - self.sorted_buffer: List[Datum] = [] + self.sorted_buffer: list[Datum] = [] async def handler( self, diff --git a/pynumaflow/accumulator/_dtypes.py b/pynumaflow/accumulator/_dtypes.py index b3d8c61b..68948b34 100644 --- a/pynumaflow/accumulator/_dtypes.py +++ b/pynumaflow/accumulator/_dtypes.py @@ -362,10 +362,10 @@ def id(self) -> str: @classmethod def from_datum(cls, datum: Datum): """Create a Message instance from a Datum object. - + Args: datum: The Datum object to convert - + Returns: Message: A new Message instance with data from the datum """ @@ -375,7 +375,7 @@ def from_datum(cls, datum: Datum): watermark=datum.watermark, event_time=datum.event_time, headers=datum.headers, - id=datum.id + id=datum.id, ) diff --git a/pynumaflow/accumulator/servicer/async_servicer.py b/pynumaflow/accumulator/servicer/async_servicer.py index 14c042b5..71bb7c21 100644 --- a/pynumaflow/accumulator/servicer/async_servicer.py +++ b/pynumaflow/accumulator/servicer/async_servicer.py @@ -1,7 +1,6 @@ import asyncio from collections.abc import AsyncIterable from typing import Union -import logging from google.protobuf import empty_pb2 as _empty_pb2 @@ -31,7 +30,7 @@ async def datum_generator( slot=d.operation.keyedWindow.slot, keys=list(d.operation.keyedWindow.keys), ) - + accumulator_request = AccumulatorRequest( operation=d.operation.event, keyed_window=keyed_window, # Use the new parameter name @@ -105,23 +104,16 @@ async def AccumulateFn( # This is a special message that indicates the end of the processing for a window # When we get this message, we send an EOF message to the client try: - logging.info("[ACCUMULATOR_DEBUG] Starting to read from consumer") async for msg in consumer: - logging.info(f"[ACCUMULATOR_DEBUG] Received message type: {type(msg)}") # If the message is an exception, we raise the exception if isinstance(msg, BaseException): - logging.info(f"[ACCUMULATOR_DEBUG] Found exception: {msg}") - logging.info(f"[ACCUMULATOR_DEBUG] Calling handle_async_error with exception: {repr(msg)}") await handle_async_error(context, msg, ERR_UDF_EXCEPTION_STRING) - logging.info(f"[ACCUMULATOR_DEBUG] Returning after handle_async_error") return # Send window EOF response or Window result response # back to the client else: - logging.info(f"[ACCUMULATOR_DEBUG] Yielding message: {msg}") yield msg except BaseException as e: - logging.info(f"[ACCUMULATOR_DEBUG] Caught exception in try block: {e}") await handle_async_error(context, e, ERR_UDF_EXCEPTION_STRING) return # Wait for the process_input_stream task to finish for a clean exit diff --git a/pynumaflow/accumulator/servicer/task_manager.py b/pynumaflow/accumulator/servicer/task_manager.py index d718e716..ca0e330b 100644 --- a/pynumaflow/accumulator/servicer/task_manager.py +++ b/pynumaflow/accumulator/servicer/task_manager.py @@ -167,7 +167,7 @@ async def create_task(self, req): # Save the result of the reduce operation to the task list self.tasks[unified_key] = curr_task - + # Increment expected EOF count since we created a new task async with self._eof_count_lock: self._expected_eof_count += 1 @@ -277,7 +277,7 @@ async def process_input_stream( # Wait for all tasks to send their EOF responses before terminating the stream # This ensures proper ordering: all messages -> all EOF responses -> STREAM_EOF await self._stream_termination_event.wait() - + # Now send STREAM_EOF to terminate the global result queue iterator await self.global_result_queue.put(STREAM_EOF) except BaseException as e: @@ -309,17 +309,17 @@ async def write_to_global_queue( event_time_pb = timestamp_pb2.Timestamp() if msg.event_time is not None: event_time_pb.FromDatetime(msg.event_time) - + watermark_pb = timestamp_pb2.Timestamp() if msg.watermark is not None: watermark_pb.FromDatetime(msg.watermark) - + start_dt_pb = timestamp_pb2.Timestamp() start_dt_pb.FromDatetime(datetime.fromtimestamp(0)) - + end_dt_pb = timestamp_pb2.Timestamp() end_dt_pb.FromDatetime(wm) - + res = accumulator_pb2.AccumulatorResponse( payload=accumulator_pb2.Payload( keys=msg.keys, @@ -339,10 +339,10 @@ async def write_to_global_queue( # send EOF start_eof_pb = timestamp_pb2.Timestamp() start_eof_pb.FromDatetime(datetime.fromtimestamp(0)) - + end_eof_pb = timestamp_pb2.Timestamp() end_eof_pb.FromDatetime(wm) - + res = accumulator_pb2.AccumulatorResponse( window=accumulator_pb2.KeyedWindow( start=start_eof_pb, end=end_eof_pb, slot="slot-0", keys=task.keys @@ -350,7 +350,7 @@ async def write_to_global_queue( EOF=True, ) await output_queue.put(res) - + # Increment received EOF count and check if all tasks are done async with self._eof_count_lock: self._received_eof_count += 1 diff --git a/tests/accumulator/test_async_accumulator.py b/tests/accumulator/test_async_accumulator.py index acdd73bf..a53b3938 100644 --- a/tests/accumulator/test_async_accumulator.py +++ b/tests/accumulator/test_async_accumulator.py @@ -33,12 +33,14 @@ def request_generator(count, request, resetkey: bool = False): # Clear previous keys and add new ones del request.payload.keys[:] request.payload.keys.extend([f"key-{i}"]) - + # Set operation based on index - first is OPEN, rest are APPEND if i == 0: request.operation.event = accumulator_pb2.AccumulatorRequest.WindowOperation.Event.OPEN else: - request.operation.event = accumulator_pb2.AccumulatorRequest.WindowOperation.Event.APPEND + request.operation.event = ( + accumulator_pb2.AccumulatorRequest.WindowOperation.Event.APPEND + ) yield request @@ -82,27 +84,19 @@ class ExampleClass(Accumulator): def __init__(self, counter): self.counter = counter - async def handler( - self, datums: AsyncIterable[Datum], output: NonBlockingIterator - ): + async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterator): async for datum in datums: self.counter += 1 msg = f"counter:{self.counter}" - await output.put( - Message(str.encode(msg), keys=datum.keys(), tags=[]) - ) + await output.put(Message(str.encode(msg), keys=datum.keys(), tags=[])) -async def accumulator_handler_func( - datums: AsyncIterable[Datum], output: NonBlockingIterator -): +async def accumulator_handler_func(datums: AsyncIterable[Datum], output: NonBlockingIterator): counter = 0 async for datum in datums: counter += 1 msg = f"counter:{counter}" - await output.put( - Message(str.encode(msg), keys=datum.keys(), tags=[]) - ) + await output.put(Message(str.encode(msg), keys=datum.keys(), tags=[])) def NewAsyncAccumulator(): @@ -167,7 +161,7 @@ def test_accumulate(self) -> None: count = 0 eof_count = 0 for r in generator_response: - if hasattr(r, 'payload') and r.payload.value: + if hasattr(r, "payload") and r.payload.value: count += 1 # Each datum should increment the counter expected_msg = f"counter:{count}" @@ -181,7 +175,7 @@ def test_accumulate(self) -> None: else: self.assertEqual(r.EOF, True) eof_count += 1 - + # We should have received 5 messages (one for each datum) self.assertEqual(5, count) self.assertEqual(1, eof_count) @@ -209,7 +203,7 @@ def test_accumulate_with_multiple_keys(self) -> None: # Track count per key key = r.payload.keys[0] if r.payload.keys else "no_key" key_counts[key] = key_counts.get(key, 0) + 1 - + # Each key should have its own counter starting from 1 expected_msg = f"counter:{key_counts[key]}" self.assertEqual( @@ -220,7 +214,7 @@ def test_accumulate_with_multiple_keys(self) -> None: else: eof_count += 1 self.assertEqual(r.EOF, True) - + # We should have 10 messages (one for each key) self.assertEqual(10, count) self.assertEqual(10, eof_count) # Each key/task sends its own EOF diff --git a/tests/accumulator/test_async_accumulator_err.py b/tests/accumulator/test_async_accumulator_err.py index f31a164a..6ac6d712 100644 --- a/tests/accumulator/test_async_accumulator_err.py +++ b/tests/accumulator/test_async_accumulator_err.py @@ -19,8 +19,6 @@ from pynumaflow.shared.asynciter import NonBlockingIterator from tests.testing_utils import ( mock_message, - mock_interval_window_start, - mock_interval_window_end, get_time_args, mock_terminate_on_stop, ) @@ -74,23 +72,17 @@ class ExampleErrorClass(Accumulator): def __init__(self, counter): self.counter = counter - async def handler( - self, datums: AsyncIterable[Datum], output: NonBlockingIterator - ): + async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterator): async for datum in datums: self.counter += 1 if self.counter == 2: # Simulate an error on the second datum raise RuntimeError("Simulated error in accumulator handler") msg = f"counter:{self.counter}" - await output.put( - Message(str.encode(msg), keys=datum.keys(), tags=[]) - ) + await output.put(Message(str.encode(msg), keys=datum.keys(), tags=[])) -async def error_accumulator_handler_func( - datums: AsyncIterable[Datum], output: NonBlockingIterator -): +async def error_accumulator_handler_func(datums: AsyncIterable[Datum], output: NonBlockingIterator): counter = 0 async for datum in datums: counter += 1 @@ -98,9 +90,7 @@ async def error_accumulator_handler_func( # Simulate an error on the second datum raise RuntimeError("Simulated error in accumulator function") msg = f"counter:{counter}" - await output.put( - Message(str.encode(msg), keys=datum.keys(), tags=[]) - ) + await output.put(Message(str.encode(msg), keys=datum.keys(), tags=[])) def NewAsyncAccumulatorError(): @@ -121,6 +111,7 @@ async def start_server(udfs): await server.start() await server.wait_for_termination() + @patch("psutil.Process.kill", mock_terminate_on_stop) class TestAsyncAccumulatorError(unittest.TestCase): @classmethod @@ -156,15 +147,15 @@ def test_accumulate_error(self) -> None: stub = self.__stub() request = start_request() generator_response = None - + try: generator_response = stub.AccumulateFn( request_iterator=request_generator(count=5, request=request) ) - + # Try to consume the generator counter = 0 - logging.info(f"[TEST_DEBUG] About to iterate through generator_response") + logging.info("[TEST_DEBUG] About to iterate through generator_response") for response in generator_response: counter += 1 logging.info(f"[TEST_DEBUG] Received response {counter}: {response}") @@ -180,12 +171,12 @@ def test_accumulate_partial_success(self) -> None: """Test that the first datum is processed before error occurs""" stub = self.__stub() request = start_request() - + try: generator_response = stub.AccumulateFn( request_iterator=request_generator(count=3, request=request) ) - + # Try to consume the generator counter = 0 for _ in generator_response: diff --git a/tests/accumulator/test_datatypes.py b/tests/accumulator/test_datatypes.py index 2f25504c..671d2e29 100644 --- a/tests/accumulator/test_datatypes.py +++ b/tests/accumulator/test_datatypes.py @@ -23,7 +23,6 @@ mock_watermark, mock_start_time, mock_end_time, - mock_headers, ) TEST_KEYS = ["test"] @@ -176,9 +175,7 @@ def test_create_result(self): consumer_future = None # In real usage, this would be an asyncio.Task watermark = datetime.fromtimestamp(1662998400, timezone.utc) - result = AccumulatorResult( - future, iterator, keys, result_queue, consumer_future, watermark - ) + result = AccumulatorResult(future, iterator, keys, result_queue, consumer_future, watermark) self.assertEqual(result.future, future) self.assertEqual(result.iterator, iterator) @@ -265,7 +262,7 @@ def test_from_datum(self): test_watermark = mock_watermark() test_headers = {"header1": "value1", "header2": "value2"} test_id = "test_datum_id" - + datum = Datum( keys=test_keys, value=test_value, @@ -274,10 +271,10 @@ def test_from_datum(self): id_=test_id, headers=test_headers, ) - + # Create message from datum message = Message.from_datum(datum) - + # Verify all properties are correctly transferred self.assertEqual(message.value, test_value) self.assertEqual(message.keys, test_keys) @@ -285,10 +282,10 @@ def test_from_datum(self): self.assertEqual(message.watermark, test_watermark) self.assertEqual(message.headers, test_headers) self.assertEqual(message.id, test_id) - + # Verify that tags are empty (default for Message) self.assertEqual(message.tags, []) - + def test_from_datum_minimal(self): """Test from_datum with minimal Datum (no headers)""" test_keys = ["minimal_key"] @@ -296,7 +293,7 @@ def test_from_datum_minimal(self): test_event_time = mock_event_time() test_watermark = mock_watermark() test_id = "minimal_id" - + datum = Datum( keys=test_keys, value=test_value, @@ -305,9 +302,9 @@ def test_from_datum_minimal(self): id_=test_id, # headers not provided (will default to {}) ) - + message = Message.from_datum(datum) - + self.assertEqual(message.value, test_value) self.assertEqual(message.keys, test_keys) self.assertEqual(message.event_time, test_event_time) @@ -315,7 +312,7 @@ def test_from_datum_minimal(self): self.assertEqual(message.headers, {}) self.assertEqual(message.id, test_id) self.assertEqual(message.tags, []) - + def test_from_datum_empty_keys(self): """Test from_datum with empty keys""" datum = Datum( @@ -325,9 +322,9 @@ def test_from_datum_empty_keys(self): watermark=mock_watermark(), id_="test_id", ) - + message = Message.from_datum(datum) - + self.assertEqual(message.keys, []) self.assertEqual(message.value, b"test_value") self.assertEqual(message.id, "test_id") @@ -335,9 +332,7 @@ def test_from_datum_empty_keys(self): class TestAccumulatorClass(unittest.TestCase): class ExampleClass(Accumulator): - async def handler( - self, datums: AsyncIterable[Datum], output: NonBlockingIterator - ): + async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterator): pass def __init__(self, test1, test2): @@ -382,7 +377,7 @@ def test_callable(self): # Since handler is an async method, __call__ should return a coroutine import asyncio from pynumaflow.shared.asynciter import NonBlockingIterator - + async def test_datums(): yield Datum( keys=["test"], @@ -391,7 +386,7 @@ async def test_datums(): watermark=mock_watermark(), id_="test", ) - + output = NonBlockingIterator() result = r(test_datums(), output) self.assertTrue(asyncio.iscoroutine(result)) diff --git a/tests/accumulator/test_use_cases.py b/tests/accumulator/test_use_cases.py index 8bf4a46b..75590513 100644 --- a/tests/accumulator/test_use_cases.py +++ b/tests/accumulator/test_use_cases.py @@ -2,7 +2,6 @@ import unittest from collections.abc import AsyncIterable from datetime import datetime, timezone -from typing import Dict, List from pynumaflow.accumulator import ( Message, @@ -11,7 +10,6 @@ AccumulatorAsyncServer, ) from pynumaflow.shared.asynciter import NonBlockingIterator -from tests.testing_utils import mock_headers class StreamSorterAccumulator(Accumulator): @@ -21,7 +19,7 @@ class StreamSorterAccumulator(Accumulator): """ def __init__(self): - self.buffer: List[Datum] = [] + self.buffer: list[Datum] = [] async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterator): # Collect all datums @@ -52,7 +50,7 @@ class StreamJoinerAccumulator(Accumulator): """ def __init__(self): - self.streams: Dict[str, List[Datum]] = {} + self.streams: dict[str, list[Datum]] = {} async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterator): # Group datums by source (using first key as source identifier) @@ -94,7 +92,7 @@ class ReorderingAccumulator(Accumulator): def __init__(self, max_delay_seconds: int = 5): self.max_delay_seconds = max_delay_seconds - self.event_buffer: List[Datum] = [] + self.event_buffer: list[Datum] = [] async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterator): # Collect all datums @@ -146,7 +144,7 @@ class TimeBasedCorrelationAccumulator(Accumulator): def __init__(self, correlation_window_seconds: int = 10): self.correlation_window_seconds = correlation_window_seconds - self.events: List[Datum] = [] + self.events: list[Datum] = [] async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterator): # Collect all datums @@ -202,7 +200,7 @@ class CustomTriggerAccumulator(Accumulator): def __init__(self, trigger_count: int = 3): self.trigger_count = trigger_count - self.accumulated_events: List[Datum] = [] + self.accumulated_events: list[Datum] = [] async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterator): # Collect datums @@ -213,7 +211,8 @@ async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterato if len(self.accumulated_events) >= self.trigger_count: # Trigger action: process all accumulated events total_value = sum( - int(event.value.decode()) for event in self.accumulated_events + int(event.value.decode()) + for event in self.accumulated_events if event.value.decode().isdigit() ) @@ -244,7 +243,7 @@ class TestAccumulatorUseCases(unittest.TestCase): def test_stream_sorter_accumulator(self): """Test the stream sorting use case""" sorter = StreamSorterAccumulator() - + # Test that the accumulator sorts by event_time self.assertIsInstance(sorter, StreamSorterAccumulator) self.assertEqual(len(sorter.buffer), 0) @@ -252,7 +251,7 @@ def test_stream_sorter_accumulator(self): def test_stream_joiner_accumulator(self): """Test the stream joining use case""" joiner = StreamJoinerAccumulator() - + # Test that the accumulator can join streams self.assertIsInstance(joiner, StreamJoinerAccumulator) self.assertEqual(len(joiner.streams), 0) @@ -260,7 +259,7 @@ def test_stream_joiner_accumulator(self): def test_reordering_accumulator(self): """Test the event reordering use case""" reorderer = ReorderingAccumulator(max_delay_seconds=10) - + # Test that the accumulator handles reordering self.assertIsInstance(reorderer, ReorderingAccumulator) self.assertEqual(reorderer.max_delay_seconds, 10) @@ -269,7 +268,7 @@ def test_reordering_accumulator(self): def test_time_based_correlation_accumulator(self): """Test the time-based correlation use case""" correlator = TimeBasedCorrelationAccumulator(correlation_window_seconds=5) - + # Test that the accumulator correlates events self.assertIsInstance(correlator, TimeBasedCorrelationAccumulator) self.assertEqual(correlator.correlation_window_seconds, 5) @@ -278,7 +277,7 @@ def test_time_based_correlation_accumulator(self): def test_custom_trigger_accumulator(self): """Test the custom triggering use case""" trigger = CustomTriggerAccumulator(trigger_count=5) - + # Test that the accumulator handles custom triggers self.assertIsInstance(trigger, CustomTriggerAccumulator) self.assertEqual(trigger.trigger_count, 5) @@ -289,29 +288,30 @@ def test_accumulator_server_with_use_cases(self): # Test with StreamSorterAccumulator server1 = AccumulatorAsyncServer(StreamSorterAccumulator) self.assertIsNotNone(server1) - + # Test with StreamJoinerAccumulator server2 = AccumulatorAsyncServer(StreamJoinerAccumulator) self.assertIsNotNone(server2) - + # Test with ReorderingAccumulator with init args server3 = AccumulatorAsyncServer(ReorderingAccumulator, init_args=(10,)) self.assertIsNotNone(server3) - + # Test with TimeBasedCorrelationAccumulator with init args server4 = AccumulatorAsyncServer(TimeBasedCorrelationAccumulator, init_args=(15,)) self.assertIsNotNone(server4) - + # Test with CustomTriggerAccumulator with init args server5 = AccumulatorAsyncServer(CustomTriggerAccumulator, init_args=(3,)) self.assertIsNotNone(server5) def test_stream_sorter_functionality(self): """Test actual sorting functionality""" + async def _test_stream_sorter_functionality(): sorter = StreamSorterAccumulator() output = NonBlockingIterator() - + # Create datums with different event times (out of order) datums = [ Datum( @@ -336,25 +336,26 @@ async def _test_stream_sorter_functionality(): id_="2", ), ] - + async def datum_generator(): for datum in datums: yield datum - + # Process the datums await sorter.handler(datum_generator(), output) - + # Verify the buffer is cleared self.assertEqual(len(sorter.buffer), 0) - + asyncio.run(_test_stream_sorter_functionality()) def test_stream_joiner_functionality(self): """Test actual joining functionality""" + async def _test_stream_joiner_functionality(): joiner = StreamJoinerAccumulator() output = NonBlockingIterator() - + # Create datums from different sources datums = [ Datum( @@ -372,17 +373,17 @@ async def _test_stream_joiner_functionality(): id_="s2_1", ), ] - + async def datum_generator(): for datum in datums: yield datum - + # Process the datums await joiner.handler(datum_generator(), output) - + # Verify the streams are cleared self.assertEqual(len(joiner.streams), 0) - + asyncio.run(_test_stream_joiner_functionality()) def test_run_async_tests(self): diff --git a/tests/accumulator/utils.py b/tests/accumulator/utils.py index 4a32b9dd..07cd4703 100644 --- a/tests/accumulator/utils.py +++ b/tests/accumulator/utils.py @@ -14,7 +14,7 @@ def create_test_datum(keys, value, event_time=None, watermark=None, id_=None, he id_ = "test_id" if headers is None: headers = {} - + return Datum( keys=keys, value=value, @@ -32,14 +32,14 @@ def create_accumulator_request(operation, keys, value, event_time=None, watermar else: event_time_timestamp = event_time watermark_timestamp = watermark - + window = accumulator_pb2.KeyedWindow( start=event_time_timestamp, end=watermark_timestamp, slot="slot-0", keys=keys, ) - + payload = accumulator_pb2.Payload( keys=keys, value=value, @@ -47,12 +47,12 @@ def create_accumulator_request(operation, keys, value, event_time=None, watermar watermark=watermark_timestamp, id="test_id", ) - + operation_proto = accumulator_pb2.AccumulatorRequest.WindowOperation( event=operation, keyedWindow=window, ) - + return accumulator_pb2.AccumulatorRequest( payload=payload, operation=operation_proto, From 54c0224f53374664bceb1f145f1caabc44eba75f Mon Sep 17 00:00:00 2001 From: srao12 Date: Sun, 20 Jul 2025 18:24:01 -0700 Subject: [PATCH 14/30] fix: tests and lint Signed-off-by: srao12 --- examples/accumulator/streamsorter/Dockerfile | 1 - examples/accumulator/streamsorter/example.py | 10 +- examples/map/even_odd/Dockerfile | 1 - pynumaflow/accumulator/_dtypes.py | 10 +- pynumaflow/accumulator/async_server.py | 14 +- .../accumulator/servicer/async_servicer.py | 16 +- .../accumulator/servicer/task_manager.py | 46 +-- tests/accumulator/test_async_accumulator.py | 54 ++++ tests/accumulator/test_datatypes.py | 297 ++++++++++++++++++ tests/accumulator/test_use_cases.py | 244 ++++++++++++++ 10 files changed, 644 insertions(+), 49 deletions(-) diff --git a/examples/accumulator/streamsorter/Dockerfile b/examples/accumulator/streamsorter/Dockerfile index cbcbf214..dd2d605b 100644 --- a/examples/accumulator/streamsorter/Dockerfile +++ b/examples/accumulator/streamsorter/Dockerfile @@ -9,7 +9,6 @@ WORKDIR $PYSETUP_PATH # Copy only core dependency files first for better caching COPY pyproject.toml poetry.lock README.md ./ COPY pynumaflow/ ./pynumaflow/ -RUN echo "Simulating long build step..." && sleep 20 RUN apt-get update && apt-get install --no-install-recommends -y \ curl wget build-essential git \ && apt-get clean && rm -rf /var/lib/apt/lists/* \ diff --git a/examples/accumulator/streamsorter/example.py b/examples/accumulator/streamsorter/example.py index e699ce54..4ddc9170 100644 --- a/examples/accumulator/streamsorter/example.py +++ b/examples/accumulator/streamsorter/example.py @@ -29,9 +29,11 @@ async def handler( ): _LOGGER.info("StreamSorter handler started") async for datum in datums: - _LOGGER.info(f"Received datum with event time: {datum.event_time}") - _LOGGER.info(f"Current latest watermark: {self.latest_wm}") - _LOGGER.info(f"Datum watermark: {datum.watermark}") + _LOGGER.info( + f"Received datum with event time: {datum.event_time}, " + f"Current latest watermark: {self.latest_wm}, " + f"Datum watermark: {datum.watermark}" + ) # If watermark has moved forward if datum.watermark and datum.watermark > self.latest_wm: @@ -68,7 +70,7 @@ async def flush_buffer(self, output: NonBlockingIterator): invoke = os.getenv("INVOKE", "class") grpc_server = None if invoke == "class": - # Here we are using the class instance as the reducer_instance + # Here we are using the class instance as the accumulator_instance # which will be used to invoke the handler function. # We are passing the init_args for the class instance. grpc_server = AccumulatorAsyncServer(StreamSorter) diff --git a/examples/map/even_odd/Dockerfile b/examples/map/even_odd/Dockerfile index 1bf155ca..0e9be000 100644 --- a/examples/map/even_odd/Dockerfile +++ b/examples/map/even_odd/Dockerfile @@ -9,7 +9,6 @@ WORKDIR $PYSETUP_PATH # Copy only core dependency files first for better caching COPY pyproject.toml poetry.lock README.md ./ COPY pynumaflow/ ./pynumaflow/ -RUN echo "Simulating long build step..." && sleep 20 RUN apt-get update && apt-get install --no-install-recommends -y \ curl wget build-essential git \ && apt-get clean && rm -rf /var/lib/apt/lists/* \ diff --git a/pynumaflow/accumulator/_dtypes.py b/pynumaflow/accumulator/_dtypes.py index 68948b34..7b0d87cb 100644 --- a/pynumaflow/accumulator/_dtypes.py +++ b/pynumaflow/accumulator/_dtypes.py @@ -32,7 +32,7 @@ class Datum: event_time: the event time of the event. watermark: the watermark of the event. >>> # Example usage - >>> from pynumaflow.reducer import Datum + >>> from pynumaflow.accumulator import Datum >>> from datetime import datetime, timezone >>> payload = bytes("test_mock_message", encoding="utf-8") >>> t1 = datetime.fromtimestamp(1662998400, timezone.utc) @@ -419,8 +419,8 @@ class _AccumulatorBuilderClass: Args: accumulator_class: the Accumulator class to be used for Accumulator UDF - args: the arguments to be passed to the reducer class - kwargs: the keyword arguments to be passed to the reducer class + args: the arguments to be passed to the accumulator class + kwargs: the keyword arguments to be passed to the accumulator class """ def __init__(self, accumulator_class: type[Accumulator], args: tuple, kwargs: dict): @@ -430,10 +430,10 @@ def __init__(self, accumulator_class: type[Accumulator], args: tuple, kwargs: di def create(self) -> Accumulator: """ - Create a new ReduceStreamer instance. + Create a new Accumulator instance. """ return self._accumulator_class(*self._args, **self._kwargs) -# AccumulatorStreamCallable is a callable which can be used as a handler for the Reduce UDF. +# AccumulatorStreamCallable is a callable which can be used as a handler for the Accumulator UDF. AccumulatorStreamCallable = Union[AccumulatorAsyncCallable, type[Accumulator]] diff --git a/pynumaflow/accumulator/async_server.py b/pynumaflow/accumulator/async_server.py index 16569ad4..fc4da947 100644 --- a/pynumaflow/accumulator/async_server.py +++ b/pynumaflow/accumulator/async_server.py @@ -62,8 +62,8 @@ class AccumulatorAsyncServer(NumaflowServer): A new servicer instance is created and attached to the server. The server instance is returned. Args: - accumulator_instance: The reducer instance to be used for - Reduce Streaming UDF + accumulator_instance: The accumulator instance to be used for + Accumulator UDF init_args: The arguments to be passed to the accumulator_handler init_kwargs: The keyword arguments to be passed to the accumulator_handler @@ -78,7 +78,7 @@ class AccumulatorAsyncServer(NumaflowServer): from pynumaflow.accumulator import Messages, Message, Datum, Metadata, AccumulatorAsyncServer, Accumulator - class ReduceCounter(Accumulator): + class StreamSorter(Accumulator): def __init__(self, counter): self.counter = counter @@ -117,13 +117,13 @@ async def reduce_handler( if __name__ == "__main__": invoke = os.getenv("INVOKE", "func_handler") if invoke == "class": - # Here we are using the class instance as the reducer_instance + # Here we are using the class instance as the accumulator_instance # which will be used to invoke the handler function. # We are passing the init_args for the class instance. - grpc_server = AccumulatorAsyncServer(ReduceCounter, init_args=(0,)) + grpc_server = AccumulatorAsyncServer(StreamSorter, init_args=(0,)) else: - # Here we are using the handler function directly as the reducer_instance. - grpc_server = AccumulatorAsyncServer(reduce_handler) + # Here we are using the handler function directly as the accumulator_instance. + grpc_server = AccumulatorAsyncServer(accumulator_handler) grpc_server.start() """ diff --git a/pynumaflow/accumulator/servicer/async_servicer.py b/pynumaflow/accumulator/servicer/async_servicer.py index 71bb7c21..1891e5a9 100644 --- a/pynumaflow/accumulator/servicer/async_servicer.py +++ b/pynumaflow/accumulator/servicer/async_servicer.py @@ -48,7 +48,7 @@ async def datum_generator( class AsyncAccumulatorServicer(accumulator_pb2_grpc.AccumulatorServicer): """ - This class is used to create a new grpc Reduce servicer instance. + This class is used to create a new grpc Accumulator servicer instance. Provides the functionality for the required rpc methods. """ @@ -90,17 +90,17 @@ async def AccumulateFn( # Start the consumer task where we read from the result queue # and send the results to the client # The task manager can write the following to the result queue: - # 1. A accumulator_pb2.ReduceResponse message - # This is the result of the reduce function, it contains the window and the - # result of the reduce function - # The result of the reduce function is a accumulator_pb2.ReduceResponse message and can be - # directly sent to the client + # 1. A accumulator_pb2.AccumulatorResponse message + # This is the result of the accumulator function, it contains the window and the + # result of the accumulator function + # The result of the accumulator function is a accumulator_pb2.AccumulatorResponse message + # and can be directly sent to the client # # 2. An Exception - # Any exceptions that occur during the processing reduce function tasks are + # Any exceptions that occur during the processing accumulator function tasks are # sent to the result queue. We then forward these exception to the client # - # 3. A accumulator_pb2.ReduceResponse message with EOF=True + # 3. A accumulator_pb2.AccumulatorResponse message with EOF=True # This is a special message that indicates the end of the processing for a window # When we get this message, we send an EOF message to the client try: diff --git a/pynumaflow/accumulator/servicer/task_manager.py b/pynumaflow/accumulator/servicer/task_manager.py index ca0e330b..f8255e6e 100644 --- a/pynumaflow/accumulator/servicer/task_manager.py +++ b/pynumaflow/accumulator/servicer/task_manager.py @@ -23,7 +23,7 @@ def build_unique_key_name(keys): """ Builds a unique key name for the given keys and window. - The key name is used to identify the Reduce task. + The key name is used to identify the Accumulator task. The format is: start_time:end_time:key1:key2:... """ return f"{DELIMITER.join(keys)}" @@ -32,21 +32,21 @@ def build_unique_key_name(keys): def build_window_hash(window): """ Builds a hash for the given window. - The hash is used to identify the Reduce Window + The hash is used to identify the Accumulator Window The format is: start_time:end_time """ return f"{window.start.ToMilliseconds()}:{window.end.ToMilliseconds()}" def create_window_eof_response(window): - """Create a Reduce response with EOF=True for a given window""" - return accumulator_pb2.ReduceResponse(window=window, EOF=True) + """Create a Accumulator response with EOF=True for a given window""" + return accumulator_pb2.AccumulatorResponse(window=window, EOF=True) class TaskManager: """ - TaskManager is responsible for managing the Reduce tasks. - It is created whenever a new reduce operation is requested. + TaskManager is responsible for managing the Accumulator tasks. + It is created whenever a new accumulator operation is requested. """ def __init__(self, handler: Union[AccumulatorAsyncCallable, _AccumulatorBuilderClass]): @@ -56,13 +56,13 @@ def __init__(self, handler: Union[AccumulatorAsyncCallable, _AccumulatorBuilderC # Event loop only keeps a weak reference, which can cause it to # get lost during execution. self.background_tasks = set() - # Handler for the reduce operation + # Handler for the accumulator operation self.__accumulator_handler = handler - # Queue to store the results of the reduce operation + # Queue to store the results of the accumulator operation # This queue is used to send the results to the client - # once the reduce operation is completed. + # once the accumulator operation is completed. # This queue is also used to send the error/exceptions to the client - # if the reduce operation fails. + # if the accumulator operation fails. self.global_result_queue = NonBlockingIterator() # EOF response counting to ensure proper termination self._expected_eof_count = 0 @@ -87,14 +87,14 @@ def get_unique_windows(self): def get_tasks(self): """ - Returns the list of reduce tasks that are + Returns the list of accumulator tasks that are currently being processed """ return list(self.tasks.values()) async def stream_send_eof(self): """ - Sends EOF to input streams of all the Reduce + Sends EOF to input streams of all the Accumulator tasks that are currently being processed. This is called when the input grpc stream is closed. """ @@ -136,7 +136,7 @@ async def create_task(self, req): riter = niter.read_iterator() # Create a new result queue for the current task # We create a new result queue for each task, so that - # the results of the reduce operation can be sent to the + # the results of the accumulator operation can be sent to the # the global result queue, which in turn sends the results # to the client. res_queue = NonBlockingIterator() @@ -153,7 +153,7 @@ async def create_task(self, req): consumer.add_done_callback(self.clean_background) # Create a new task for the accumulator operation, this will invoke the - # Reduce handler with the given keys, request iterator, and window. + # Accumulator handler with the given keys, request iterator, and window. task = asyncio.create_task(self.__invoke_accumulator(riter, res_queue)) # Save a reference to the result of this function, to avoid a # task disappearing mid-execution. @@ -165,7 +165,7 @@ async def create_task(self, req): task, niter, keys, res_queue, consumer, datetime.fromtimestamp(-1) ) - # Save the result of the reduce operation to the task list + # Save the result of the accumulator operation to the task list self.tasks[unified_key] = curr_task # Increment expected EOF count since we created a new task @@ -195,15 +195,15 @@ async def __invoke_accumulator( output: NonBlockingIterator, ): """ - Invokes the UDF reduce handler with the given keys, + Invokes the UDF accumulator handler with the given keys, request iterator, and window. Returns the result of the - reduce operation. + accumulator operation. """ new_instance = self.__accumulator_handler # If the accumulator handler is a class instance, create a new instance of it. # It is required for a new key to be processed by a - # new instance of the reducer for a given window + # new instance of the accumulator for a given window # Otherwise the function handler can be called directly if isinstance(self.__accumulator_handler, _AccumulatorBuilderClass): new_instance = self.__accumulator_handler.create() @@ -211,7 +211,7 @@ async def __invoke_accumulator( _ = await new_instance(request_iterator, output) # send EOF to the output stream await output.put(STREAM_EOF) - # If there is an error in the reduce operation, log and + # If there is an error in the accumulator operation, log and # then send the error to the result queue except BaseException as err: _LOGGER.critical("panic inside accumulator handle", exc_info=True) @@ -242,7 +242,7 @@ async def process_input_stream( else: _LOGGER.debug(f"No operation matched for request: {request}", exc_info=True) - # If there is an error in the reduce operation, log and + # If there is an error in the accumulator operation, log and # then send the error to the result queue except BaseException as e: err_msg = f"Accumulator Error: {repr(e)}" @@ -257,7 +257,7 @@ async def process_input_stream( # respective iterators. await self.stream_send_eof() - # get the list of reduce tasks that are currently being processed + # get the list of accumulator tasks that are currently being processed # iterate through the tasks and wait for them to complete for task in self.get_tasks(): # Once this is done, we know that the task has written all the results @@ -281,7 +281,7 @@ async def process_input_stream( # Now send STREAM_EOF to terminate the global result queue iterator await self.global_result_queue.put(STREAM_EOF) except BaseException as e: - err_msg = f"Reduce Streaming Error: {repr(e)}" + err_msg = f"Accumulator Streaming Error: {repr(e)}" _LOGGER.critical(err_msg, exc_info=True) await self.global_result_queue.put(e) @@ -289,7 +289,7 @@ async def write_to_global_queue( self, input_queue: NonBlockingIterator, output_queue: NonBlockingIterator, unified_key: str ): """ - This task is for given Reduce task. + This task is for given Accumulator task. This would from the local result queue for the task and then write to the global result queue """ diff --git a/tests/accumulator/test_async_accumulator.py b/tests/accumulator/test_async_accumulator.py index a53b3938..2786acd7 100644 --- a/tests/accumulator/test_async_accumulator.py +++ b/tests/accumulator/test_async_accumulator.py @@ -270,6 +270,60 @@ def test_max_threads(self): server = AccumulatorAsyncServer(accumulator_instance=ExampleClass) self.assertEqual(server.max_threads, 4) + def test_server_info_file_path_handling(self): + """Test AccumulatorAsyncServer with custom server info file path.""" + + server = AccumulatorAsyncServer( + ExampleClass, init_args=(0,), server_info_file="/custom/path/server_info.json" + ) + + self.assertEqual(server.server_info_file, "/custom/path/server_info.json") + + def test_init_kwargs_none_handling(self): + """Test init_kwargs None handling in AccumulatorAsyncServer.""" + + server = AccumulatorAsyncServer( + ExampleClass, init_args=(0,), init_kwargs=None # This should be converted to {} + ) + + # Should not raise any errors and should work correctly + self.assertIsNotNone(server.accumulator_handler) + + def test_server_with_zero_max_threads(self): + """Test server creation with max_threads set to 0.""" + + server = AccumulatorAsyncServer(ExampleClass, max_threads=0) + + # Should be set to 0 (minimum) + self.assertEqual(server.max_threads, 0) + + def test_server_with_negative_max_threads(self): + """Test server creation with negative max_threads.""" + + server = AccumulatorAsyncServer(ExampleClass, max_threads=-5) + + # Should be set to -5 (the minimum function will handle this) + self.assertEqual(server.max_threads, -5) + + def test_server_start_method_logging(self): + """Test server start method includes proper logging.""" + from unittest.mock import patch + + server = AccumulatorAsyncServer(ExampleClass) + + # Mock aiorun.run to prevent actual server startup + with patch("pynumaflow.accumulator.async_server.aiorun") as mock_aiorun, patch( + "pynumaflow.accumulator.async_server._LOGGER" + ) as mock_logger: + server.start() + + # Verify logging was called + mock_logger.info.assert_called_once_with("Starting Async Accumulator Server") + + # Verify aiorun.run was called with correct parameters + mock_aiorun.run.assert_called_once() + self.assertTrue(mock_aiorun.run.call_args[1]["use_uvloop"]) + if __name__ == "__main__": logging.basicConfig(level=logging.DEBUG) diff --git a/tests/accumulator/test_datatypes.py b/tests/accumulator/test_datatypes.py index 671d2e29..2957941b 100644 --- a/tests/accumulator/test_datatypes.py +++ b/tests/accumulator/test_datatypes.py @@ -15,6 +15,7 @@ AccumulatorRequest, WindowOperation, Message, + _AccumulatorBuilderClass, ) from pynumaflow.shared.asynciter import NonBlockingIterator from tests.testing_utils import ( @@ -394,5 +395,301 @@ async def test_datums(): result.close() +class TestAccumulatorBuilderClass(unittest.TestCase): + """Test AccumulatorBuilderClass functionality.""" + + def test_builder_class_creation(self): + """Test AccumulatorBuilderClass creation and instantiation.""" + + class TestAccumulator(Accumulator): + def __init__(self, counter=0): + self.counter = counter + + async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterator): + pass + + builder = _AccumulatorBuilderClass(TestAccumulator, (15,), {}) + instance = builder.create() + + self.assertIsInstance(instance, TestAccumulator) + self.assertEqual(instance.counter, 15) + + def test_builder_class_with_kwargs(self): + """Test AccumulatorBuilderClass with keyword arguments.""" + + class KwargsAccumulator(Accumulator): + def __init__(self, param1, param2, param3=None): + self.param1 = param1 + self.param2 = param2 + self.param3 = param3 + + async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterator): + pass + + builder = _AccumulatorBuilderClass( + KwargsAccumulator, ("arg1", "arg2"), {"param3": "kwarg_value"} + ) + instance = builder.create() + + self.assertIsInstance(instance, KwargsAccumulator) + self.assertEqual(instance.param1, "arg1") + self.assertEqual(instance.param2, "arg2") + self.assertEqual(instance.param3, "kwarg_value") + + def test_builder_class_empty_args(self): + """Test AccumulatorBuilderClass with empty args and kwargs.""" + + class EmptyArgsAccumulator(Accumulator): + def __init__(self): + self.initialized = True + + async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterator): + pass + + builder = _AccumulatorBuilderClass(EmptyArgsAccumulator, (), {}) + instance = builder.create() + + self.assertIsInstance(instance, EmptyArgsAccumulator) + self.assertTrue(instance.initialized) + + +class TestAsyncServerHandlerCoverage(unittest.TestCase): + """Test async server handler function coverage.""" + + def test_get_handler_with_function_and_args_error(self): + """Test get_handler raises TypeError when function handler is passed with init args.""" + from pynumaflow.accumulator.async_server import get_handler + + async def test_func(datums, output): + pass + + with self.assertRaises(TypeError) as context: + get_handler(test_func, init_args=(1, 2)) + + self.assertIn( + "Cannot pass function handler with init args or kwargs", str(context.exception) + ) + + def test_get_handler_with_function_and_kwargs_error(self): + """Test get_handler raises TypeError when function handler is passed with init kwargs.""" + from pynumaflow.accumulator.async_server import get_handler + + async def test_func(datums, output): + pass + + with self.assertRaises(TypeError) as context: + get_handler(test_func, init_kwargs={"test": "value"}) + + self.assertIn( + "Cannot pass function handler with init args or kwargs", str(context.exception) + ) + + def test_get_handler_with_invalid_class(self): + """Test get_handler raises TypeError for invalid class type.""" + from pynumaflow.accumulator.async_server import get_handler + + class InvalidClass: + pass + + with self.assertRaises(TypeError) as context: + get_handler(InvalidClass()) + + # The actual error comes from issubclass() check since we're passing an instance + self.assertIn("issubclass() arg 1 must be a class", str(context.exception)) + + def test_get_handler_with_invalid_class_type(self): + """Test get_handler raises TypeError for invalid Accumulator class type.""" + from pynumaflow.accumulator.async_server import get_handler + + class NonAccumulatorClass: + pass + + with self.assertRaises(TypeError) as context: + get_handler(NonAccumulatorClass) + + # This will hit the 'Invalid Class Type' error path + self.assertIn("Invalid Class Type", str(context.exception)) + + def test_get_handler_with_valid_class(self): + """Test get_handler returns AccumulatorBuilderClass for valid Accumulator subclass.""" + from pynumaflow.accumulator.async_server import get_handler + + class ValidAccumulator(Accumulator): + def __init__(self, counter=0): + self.counter = counter + + async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterator): + pass + + result = get_handler(ValidAccumulator, init_args=(10,), init_kwargs={"counter": 5}) + + self.assertIsInstance(result, _AccumulatorBuilderClass) + + +class TestTaskManagerUtilities(unittest.TestCase): + """Test TaskManager utility functions.""" + + def test_build_window_hash(self): + """Test build_window_hash function.""" + from pynumaflow.accumulator.servicer.task_manager import build_window_hash + from unittest.mock import Mock + + # Create a mock window with ToMilliseconds method + mock_window = Mock() + mock_window.start.ToMilliseconds.return_value = 1000 + mock_window.end.ToMilliseconds.return_value = 2000 + + result = build_window_hash(mock_window) + self.assertEqual(result, "1000:2000") + + def test_build_unique_key_name(self): + """Test build_unique_key_name function.""" + from pynumaflow.accumulator.servicer.task_manager import build_unique_key_name + + keys = ["key1", "key2", "key3"] + result = build_unique_key_name(keys) + + self.assertEqual(result, "key1:key2:key3") + + def test_task_manager_initialization(self): + """Test TaskManager initialization.""" + from pynumaflow.accumulator.servicer.task_manager import TaskManager + from unittest.mock import Mock + + handler = Mock() + task_manager = TaskManager(handler) + + self.assertEqual(task_manager._TaskManager__accumulator_handler, handler) + self.assertEqual(len(task_manager.tasks), 0) + self.assertEqual(len(task_manager.background_tasks), 0) + self.assertEqual(task_manager._expected_eof_count, 0) + self.assertEqual(task_manager._received_eof_count, 0) + + def test_task_manager_get_unique_windows(self): + """Test TaskManager get_unique_windows with empty tasks.""" + from pynumaflow.accumulator.servicer.task_manager import TaskManager + from unittest.mock import Mock + + handler = Mock() + task_manager = TaskManager(handler) + + windows = task_manager.get_unique_windows() + self.assertEqual(len(windows), 0) + + def test_task_manager_get_tasks(self): + """Test TaskManager get_tasks method.""" + from pynumaflow.accumulator.servicer.task_manager import TaskManager + from unittest.mock import Mock + + handler = Mock() + task_manager = TaskManager(handler) + + tasks = task_manager.get_tasks() + self.assertEqual(len(tasks), 0) + + def test_task_manager_close_task_not_found(self): + """Test TaskManager close_task when task is not found.""" + import asyncio + from unittest.mock import patch, Mock + from pynumaflow.accumulator.servicer.task_manager import TaskManager + + with patch("pynumaflow.accumulator.servicer.task_manager._LOGGER") as mock_logger: + handler = Mock() + task_manager = TaskManager(handler) + + # Create a mock request with payload that has keys + mock_request = Mock() + mock_datum = Mock() + mock_datum.keys.return_value = ["test_key"] + mock_request.payload = mock_datum + + # Call close_task - should log error and put exception in queue + asyncio.run(task_manager.close_task(mock_request)) + + # Verify logger was called + mock_logger.critical.assert_called_once_with( + "accumulator task not found", exc_info=True + ) + + +class TestAsyncServicerCoverage(unittest.TestCase): + """Test AsyncAccumulatorServicer coverage.""" + + def test_servicer_is_ready_method(self): + """Test AsyncAccumulatorServicer IsReady method.""" + import asyncio + from unittest.mock import Mock + from pynumaflow.accumulator.servicer.async_servicer import AsyncAccumulatorServicer + from pynumaflow.proto.accumulator import accumulator_pb2 + from google.protobuf import empty_pb2 as _empty_pb2 + + async def run_test(): + mock_handler = Mock() + servicer = AsyncAccumulatorServicer(mock_handler) + + # Create mock context and empty request + mock_context = Mock() + empty_request = _empty_pb2.Empty() + + # Call IsReady + response = await servicer.IsReady(empty_request, mock_context) + + # Verify response + self.assertIsInstance(response, accumulator_pb2.ReadyResponse) + self.assertTrue(response.ready) + + asyncio.run(run_test()) + + def test_datum_generator(self): + """Test datum_generator function.""" + import asyncio + from unittest.mock import Mock + from pynumaflow.accumulator.servicer.async_servicer import datum_generator + from tests.testing_utils import ( + get_time_args, + mock_interval_window_start, + mock_interval_window_end, + ) + + async def run_test(): + # Create mock request + event_time_timestamp, watermark_timestamp = get_time_args() + + mock_request = Mock() + mock_request.operation.event = WindowOperation.OPEN.value + mock_request.operation.keyedWindow.start.ToDatetime.return_value = ( + mock_interval_window_start().ToDatetime() + ) + mock_request.operation.keyedWindow.end.ToDatetime.return_value = ( + mock_interval_window_end().ToDatetime() + ) + mock_request.operation.keyedWindow.slot = "slot-0" + mock_request.operation.keyedWindow.keys = ["test_key"] + + mock_request.payload.keys = ["test_key"] + mock_request.payload.value = b"test_value" + mock_request.payload.event_time.ToDatetime.return_value = ( + event_time_timestamp.ToDatetime() + ) + mock_request.payload.watermark.ToDatetime.return_value = ( + watermark_timestamp.ToDatetime() + ) + mock_request.payload.id = "test_id" + mock_request.payload.headers = {"header1": "value1"} + + async def mock_request_iterator(): + yield mock_request + + results = [] + async for result in datum_generator(mock_request_iterator()): + results.append(result) + + self.assertEqual(len(results), 1) + self.assertIsInstance(results[0], AccumulatorRequest) + self.assertEqual(results[0].operation, WindowOperation.OPEN.value) + + asyncio.run(run_test()) + + if __name__ == "__main__": unittest.main() diff --git a/tests/accumulator/test_use_cases.py b/tests/accumulator/test_use_cases.py index 75590513..f602d6d4 100644 --- a/tests/accumulator/test_use_cases.py +++ b/tests/accumulator/test_use_cases.py @@ -10,6 +10,7 @@ AccumulatorAsyncServer, ) from pynumaflow.shared.asynciter import NonBlockingIterator +from pynumaflow._constants import STREAM_EOF class StreamSorterAccumulator(Accumulator): @@ -392,6 +393,249 @@ def test_run_async_tests(self): # are now properly handled within their respective test methods pass + def test_error_handling_scenarios(self): + """Test error handling scenarios in accumulator processing.""" + + async def run_test(): + # Test 1: Function handler called directly (covers line 44 in async_server.py) + async def func_handler(datums: AsyncIterable[Datum], output: NonBlockingIterator): + async for datum in datums: + await output.put(Message(datum.value, keys=datum.keys())) + + from pynumaflow.accumulator.async_server import get_handler + + handler = get_handler(func_handler) + self.assertEqual(handler, func_handler) + + # Test 2: Task manager with function handler (covers lines 208->210 in task_manager.py) + from pynumaflow.accumulator.servicer.task_manager import TaskManager + + task_manager = TaskManager(func_handler) + # Create mock request iterator that simulates function handler path + mock_datum = Datum( + keys=["func_key"], + value=b"test_data", + event_time=datetime.now(), + watermark=datetime.now(), + id_="func_test", + ) + + output_queue = NonBlockingIterator() + + # Create an async iterator that yields our mock datum + async def datum_iter(): + yield mock_datum + + # This should cover the function handler path in __invoke_accumulator + await task_manager._TaskManager__invoke_accumulator(datum_iter(), output_queue) + + # Verify the function handler was called by checking for output + results = [] + try: + async for item in output_queue.read_iterator(): + results.append(item) + if item == STREAM_EOF: + break + # Only expect one message for this test + if len(results) >= 3: # Safety break to avoid infinite loop + break + except Exception: + pass # Expected behavior for this simplified test + + # Verify we got at least one result (message was processed) + self.assertGreaterEqual(len(results), 1) + + asyncio.run(run_test()) + + def test_task_manager_error_scenarios(self): + """Test various error scenarios in TaskManager.""" + + # Test 1: Unknown window operation (covers lines 239-243 in task_manager.py) + from pynumaflow.accumulator.servicer.task_manager import TaskManager + from unittest.mock import Mock + + handler = Mock() + task_manager = TaskManager(handler) + + # Test 2: Watermark update with AccumulatorResult (covers lines 311, 315) + from pynumaflow.accumulator._dtypes import AccumulatorResult + + # Create a task to test watermark updating + initial_watermark = datetime.fromtimestamp(500) + task = AccumulatorResult( + _future=Mock(), + _iterator=Mock(), + _key=["test_key"], + _result_queue=Mock(), + _consumer_future=Mock(), + _latest_watermark=initial_watermark, + ) + + # Test update_watermark method directly + new_watermark = datetime.fromtimestamp(2000) + task.update_watermark(new_watermark) + self.assertEqual(task.latest_watermark, new_watermark) + + # Test 3: Test direct instantiation and basic functionality + unified_key = "test_key" + task_manager.tasks[unified_key] = task + + # Verify task was added + self.assertIn(unified_key, task_manager.tasks) + self.assertEqual(task_manager.tasks[unified_key], task) + + def test_edge_case_scenarios(self): + """Test edge cases and error conditions.""" + + async def run_test(): + from pynumaflow.accumulator.servicer.task_manager import TaskManager + from unittest.mock import Mock + + # Test 1: Error handling in EOF counting (covers lines 361-363) + handler = Mock() + task_manager = TaskManager(handler) + + # Manually set up a scenario where EOF count exceeds expected + task_manager._expected_eof_count = 1 + task_manager._received_eof_count = 2 # More than expected + + # Create mock task for testing + from pynumaflow.accumulator._dtypes import AccumulatorResult + + mock_task = AccumulatorResult( + _future=Mock(), + _iterator=Mock(), + _key=["edge_test"], + _result_queue=Mock(), + _consumer_future=Mock(), + _latest_watermark=datetime.now(), + ) + + unified_key = "edge_test" + task_manager.tasks[unified_key] = mock_task + + # Test watermark handling with None values (covers lines 311, 315) + input_queue = NonBlockingIterator() + output_queue = NonBlockingIterator() + + # Create message with None watermark and event_time + edge_message = Message( + value=b"edge_test", + keys=["edge_test"], + watermark=None, # Test None watermark handling + event_time=None, # Test None event_time handling + ) + + await input_queue.put(edge_message) + await input_queue.put(STREAM_EOF) + + # This should handle None watermark and event_time without error + await task_manager.write_to_global_queue(input_queue, output_queue, unified_key) + + # Verify output was generated + results = [] + async for item in output_queue.read_iterator(): + results.append(item) + if len(results) >= 2: # Message + EOF response + break + + self.assertTrue(len(results) >= 2) + + asyncio.run(run_test()) + + def test_abstract_method_coverage(self): + """Test abstract method coverage (line 412 in _dtypes.py).""" + + # Test calling the abstract handler method directly + class DirectTestAccumulator(Accumulator): + pass # Don't implement handler to test abstract method + + # This should raise TypeError due to abstract method + with self.assertRaises(TypeError): + DirectTestAccumulator() + + def test_servicer_error_handling(self): + """Test error handling in AsyncAccumulatorServicer (lines 116-118, 122-124).""" + + async def run_test(): + from pynumaflow.accumulator.servicer.async_servicer import AsyncAccumulatorServicer + from unittest.mock import Mock, patch + + # Test exception in consumer loop (lines 116-118) + mock_handler = Mock() + servicer = AsyncAccumulatorServicer(mock_handler) + mock_context = Mock() + + async def failing_request_iterator(): + yield Mock() # Just one request + + # Mock TaskManager to simulate error in consumer + with patch("pynumaflow.accumulator.servicer.async_servicer.TaskManager") as mock_tm: + mock_task_manager = Mock() + mock_tm.return_value = mock_task_manager + + # Mock read_iterator to raise exception + async def failing_reader(): + raise RuntimeError("Consumer error") + + mock_result_queue = Mock() + mock_result_queue.read_iterator.return_value = failing_reader() + mock_task_manager.global_result_queue = mock_result_queue + + # Mock process_input_stream + async def mock_process(): + pass + + mock_task_manager.process_input_stream.return_value = mock_process() + + # This should handle the consumer exception (lines 116-118) + with patch( + "pynumaflow.accumulator.servicer.async_servicer.handle_async_error" + ) as mock_handle: + results = [] + async for result in servicer.AccumulateFn( + failing_request_iterator(), mock_context + ): + results.append(result) + + # Should have called error handler + mock_handle.assert_called() + + # Test exception in producer wait (lines 122-124) + with patch("pynumaflow.accumulator.servicer.async_servicer.TaskManager") as mock_tm2: + mock_task_manager2 = Mock() + mock_tm2.return_value = mock_task_manager2 + + # Mock read_iterator to work normally + async def normal_reader(): + return + yield # Empty generator + + mock_result_queue2 = Mock() + mock_result_queue2.read_iterator.return_value = normal_reader() + mock_task_manager2.global_result_queue = mock_result_queue2 + + # Mock process_input_stream to raise exception when awaited + async def failing_process(): + raise RuntimeError("Producer error") + + mock_task_manager2.process_input_stream.return_value = failing_process() + + # This should handle the producer exception (lines 122-124) + with patch( + "pynumaflow.accumulator.servicer.async_servicer.handle_async_error" + ) as mock_handle2: + results2 = [] + async for result in servicer.AccumulateFn( + failing_request_iterator(), mock_context + ): + results2.append(result) + + # Should have called error handler for producer error + mock_handle2.assert_called() + + asyncio.run(run_test()) + if __name__ == "__main__": unittest.main() From 1d2863c91e0dea2c7eaa97f52e72575a792292f7 Mon Sep 17 00:00:00 2001 From: srao12 Date: Sun, 20 Jul 2025 18:26:54 -0700 Subject: [PATCH 15/30] fix: update example Signed-off-by: srao12 --- examples/accumulator/streamsorter/example.py | 7 +------ examples/accumulator/streamsorter/pipeline.yaml | 2 -- 2 files changed, 1 insertion(+), 8 deletions(-) diff --git a/examples/accumulator/streamsorter/example.py b/examples/accumulator/streamsorter/example.py index 4ddc9170..8e0615ed 100644 --- a/examples/accumulator/streamsorter/example.py +++ b/examples/accumulator/streamsorter/example.py @@ -67,11 +67,6 @@ async def flush_buffer(self, output: NonBlockingIterator): if __name__ == "__main__": - invoke = os.getenv("INVOKE", "class") grpc_server = None - if invoke == "class": - # Here we are using the class instance as the accumulator_instance - # which will be used to invoke the handler function. - # We are passing the init_args for the class instance. - grpc_server = AccumulatorAsyncServer(StreamSorter) + grpc_server = AccumulatorAsyncServer(StreamSorter) grpc_server.start() diff --git a/examples/accumulator/streamsorter/pipeline.yaml b/examples/accumulator/streamsorter/pipeline.yaml index c13416f6..d4ccab96 100644 --- a/examples/accumulator/streamsorter/pipeline.yaml +++ b/examples/accumulator/streamsorter/pipeline.yaml @@ -26,8 +26,6 @@ spec: env: - name: PYTHONDEBUG value: "true" - - name: INVOKE - value: "class" groupBy: window: accumulator: From 82f83d54d7fb2caef6f191bd5fceb523c76e0bb8 Mon Sep 17 00:00:00 2001 From: srao12 Date: Sun, 20 Jul 2025 19:10:06 -0700 Subject: [PATCH 16/30] fix: tests Signed-off-by: srao12 --- .../streamsorter/Makefile.optimized | 2 +- .../accumulator/streamsorter/pyproject.toml | 2 +- tests/accumulator/test_datatypes.py | 42 ++++++---- tests/accumulator/test_use_cases.py | 78 ++++++++++--------- 4 files changed, 70 insertions(+), 54 deletions(-) diff --git a/examples/accumulator/streamsorter/Makefile.optimized b/examples/accumulator/streamsorter/Makefile.optimized index f0e4439c..136be046 100644 --- a/examples/accumulator/streamsorter/Makefile.optimized +++ b/examples/accumulator/streamsorter/Makefile.optimized @@ -1,7 +1,7 @@ TAG ?= stable PUSH ?= false IMAGE_REGISTRY = quay.io/numaio/numaflow-python/streamsorter:${TAG} -DOCKER_FILE_PATH = examples/map/even_odd/Dockerfile.optimized +DOCKER_FILE_PATH = examples/accumulator/streamsorter/Dockerfile BASE_IMAGE_NAME = numaflow-python-base .PHONY: base-image diff --git a/examples/accumulator/streamsorter/pyproject.toml b/examples/accumulator/streamsorter/pyproject.toml index 99d64cd1..9397268d 100644 --- a/examples/accumulator/streamsorter/pyproject.toml +++ b/examples/accumulator/streamsorter/pyproject.toml @@ -5,7 +5,7 @@ description = "" authors = ["Numaflow developers"] [tool.poetry.dependencies] -python = ">=3.11,<3.13" +python = "~3.10" pynumaflow = { path = "../../../"} [build-system] diff --git a/tests/accumulator/test_datatypes.py b/tests/accumulator/test_datatypes.py index 2957941b..75ca5cc0 100644 --- a/tests/accumulator/test_datatypes.py +++ b/tests/accumulator/test_datatypes.py @@ -553,39 +553,51 @@ def test_build_unique_key_name(self): def test_task_manager_initialization(self): """Test TaskManager initialization.""" + import asyncio from pynumaflow.accumulator.servicer.task_manager import TaskManager from unittest.mock import Mock - handler = Mock() - task_manager = TaskManager(handler) + async def run_test(): + handler = Mock() + task_manager = TaskManager(handler) - self.assertEqual(task_manager._TaskManager__accumulator_handler, handler) - self.assertEqual(len(task_manager.tasks), 0) - self.assertEqual(len(task_manager.background_tasks), 0) - self.assertEqual(task_manager._expected_eof_count, 0) - self.assertEqual(task_manager._received_eof_count, 0) + self.assertEqual(task_manager._TaskManager__accumulator_handler, handler) + self.assertEqual(len(task_manager.tasks), 0) + self.assertEqual(len(task_manager.background_tasks), 0) + self.assertEqual(task_manager._expected_eof_count, 0) + self.assertEqual(task_manager._received_eof_count, 0) + + asyncio.run(run_test()) def test_task_manager_get_unique_windows(self): """Test TaskManager get_unique_windows with empty tasks.""" + import asyncio from pynumaflow.accumulator.servicer.task_manager import TaskManager from unittest.mock import Mock - handler = Mock() - task_manager = TaskManager(handler) + async def run_test(): + handler = Mock() + task_manager = TaskManager(handler) + + windows = task_manager.get_unique_windows() + self.assertEqual(len(windows), 0) - windows = task_manager.get_unique_windows() - self.assertEqual(len(windows), 0) + asyncio.run(run_test()) def test_task_manager_get_tasks(self): """Test TaskManager get_tasks method.""" + import asyncio from pynumaflow.accumulator.servicer.task_manager import TaskManager from unittest.mock import Mock - handler = Mock() - task_manager = TaskManager(handler) + async def run_test(): + handler = Mock() + task_manager = TaskManager(handler) - tasks = task_manager.get_tasks() - self.assertEqual(len(tasks), 0) + tasks = task_manager.get_tasks() + self.assertEqual(len(tasks), 0) + + asyncio.run(run_test()) def test_task_manager_close_task_not_found(self): """Test TaskManager close_task when task is not found.""" diff --git a/tests/accumulator/test_use_cases.py b/tests/accumulator/test_use_cases.py index f602d6d4..29a76ab7 100644 --- a/tests/accumulator/test_use_cases.py +++ b/tests/accumulator/test_use_cases.py @@ -397,7 +397,7 @@ def test_error_handling_scenarios(self): """Test error handling scenarios in accumulator processing.""" async def run_test(): - # Test 1: Function handler called directly (covers line 44 in async_server.py) + # Test 1: Function handler called directly async def func_handler(datums: AsyncIterable[Datum], output: NonBlockingIterator): async for datum in datums: await output.put(Message(datum.value, keys=datum.keys())) @@ -407,7 +407,7 @@ async def func_handler(datums: AsyncIterable[Datum], output: NonBlockingIterator handler = get_handler(func_handler) self.assertEqual(handler, func_handler) - # Test 2: Task manager with function handler (covers lines 208->210 in task_manager.py) + # Test 2: Task manager with function handler from pynumaflow.accumulator.servicer.task_manager import TaskManager task_manager = TaskManager(func_handler) @@ -449,40 +449,44 @@ async def datum_iter(): def test_task_manager_error_scenarios(self): """Test various error scenarios in TaskManager.""" + import asyncio + + async def run_test(): + # Test 1: Unknown window operation + from pynumaflow.accumulator.servicer.task_manager import TaskManager + from unittest.mock import Mock - # Test 1: Unknown window operation (covers lines 239-243 in task_manager.py) - from pynumaflow.accumulator.servicer.task_manager import TaskManager - from unittest.mock import Mock + handler = Mock() + task_manager = TaskManager(handler) - handler = Mock() - task_manager = TaskManager(handler) + # Test 2: Watermark update with AccumulatorResult + from pynumaflow.accumulator._dtypes import AccumulatorResult - # Test 2: Watermark update with AccumulatorResult (covers lines 311, 315) - from pynumaflow.accumulator._dtypes import AccumulatorResult + # Create a task to test watermark updating + initial_watermark = datetime.fromtimestamp(500) + task = AccumulatorResult( + _future=Mock(), + _iterator=Mock(), + _key=["test_key"], + _result_queue=Mock(), + _consumer_future=Mock(), + _latest_watermark=initial_watermark, + ) - # Create a task to test watermark updating - initial_watermark = datetime.fromtimestamp(500) - task = AccumulatorResult( - _future=Mock(), - _iterator=Mock(), - _key=["test_key"], - _result_queue=Mock(), - _consumer_future=Mock(), - _latest_watermark=initial_watermark, - ) + # Test update_watermark method directly + new_watermark = datetime.fromtimestamp(2000) + task.update_watermark(new_watermark) + self.assertEqual(task.latest_watermark, new_watermark) - # Test update_watermark method directly - new_watermark = datetime.fromtimestamp(2000) - task.update_watermark(new_watermark) - self.assertEqual(task.latest_watermark, new_watermark) + # Test 3: Test direct instantiation and basic functionality + unified_key = "test_key" + task_manager.tasks[unified_key] = task - # Test 3: Test direct instantiation and basic functionality - unified_key = "test_key" - task_manager.tasks[unified_key] = task + # Verify task was added + self.assertIn(unified_key, task_manager.tasks) + self.assertEqual(task_manager.tasks[unified_key], task) - # Verify task was added - self.assertIn(unified_key, task_manager.tasks) - self.assertEqual(task_manager.tasks[unified_key], task) + asyncio.run(run_test()) def test_edge_case_scenarios(self): """Test edge cases and error conditions.""" @@ -491,7 +495,7 @@ async def run_test(): from pynumaflow.accumulator.servicer.task_manager import TaskManager from unittest.mock import Mock - # Test 1: Error handling in EOF counting (covers lines 361-363) + # Test 1: Error handling in EOF counting handler = Mock() task_manager = TaskManager(handler) @@ -514,7 +518,7 @@ async def run_test(): unified_key = "edge_test" task_manager.tasks[unified_key] = mock_task - # Test watermark handling with None values (covers lines 311, 315) + # Test watermark handling with None values input_queue = NonBlockingIterator() output_queue = NonBlockingIterator() @@ -544,7 +548,7 @@ async def run_test(): asyncio.run(run_test()) def test_abstract_method_coverage(self): - """Test abstract method coverage (line 412 in _dtypes.py).""" + """Test abstract method coverage.""" # Test calling the abstract handler method directly class DirectTestAccumulator(Accumulator): @@ -555,13 +559,13 @@ class DirectTestAccumulator(Accumulator): DirectTestAccumulator() def test_servicer_error_handling(self): - """Test error handling in AsyncAccumulatorServicer (lines 116-118, 122-124).""" + """Test error handling in AsyncAccumulatorServicer.""" async def run_test(): from pynumaflow.accumulator.servicer.async_servicer import AsyncAccumulatorServicer from unittest.mock import Mock, patch - # Test exception in consumer loop (lines 116-118) + # Test exception in consumer loop mock_handler = Mock() servicer = AsyncAccumulatorServicer(mock_handler) mock_context = Mock() @@ -588,7 +592,7 @@ async def mock_process(): mock_task_manager.process_input_stream.return_value = mock_process() - # This should handle the consumer exception (lines 116-118) + # This should handle the consumer exception with patch( "pynumaflow.accumulator.servicer.async_servicer.handle_async_error" ) as mock_handle: @@ -601,7 +605,7 @@ async def mock_process(): # Should have called error handler mock_handle.assert_called() - # Test exception in producer wait (lines 122-124) + # Test exception in producer wait with patch("pynumaflow.accumulator.servicer.async_servicer.TaskManager") as mock_tm2: mock_task_manager2 = Mock() mock_tm2.return_value = mock_task_manager2 @@ -621,7 +625,7 @@ async def failing_process(): mock_task_manager2.process_input_stream.return_value = failing_process() - # This should handle the producer exception (lines 122-124) + # This should handle the producer exception with patch( "pynumaflow.accumulator.servicer.async_servicer.handle_async_error" ) as mock_handle2: From ee3acf917a5824dc85468f6ada6675de0937f597 Mon Sep 17 00:00:00 2001 From: srao12 Date: Sun, 20 Jul 2025 19:14:02 -0700 Subject: [PATCH 17/30] fix: lint Signed-off-by: srao12 --- tests/accumulator/test_use_cases.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/accumulator/test_use_cases.py b/tests/accumulator/test_use_cases.py index 29a76ab7..5f377da6 100644 --- a/tests/accumulator/test_use_cases.py +++ b/tests/accumulator/test_use_cases.py @@ -450,7 +450,7 @@ async def datum_iter(): def test_task_manager_error_scenarios(self): """Test various error scenarios in TaskManager.""" import asyncio - + async def run_test(): # Test 1: Unknown window operation from pynumaflow.accumulator.servicer.task_manager import TaskManager From 6b76b9d9d28953535a4281d649de98264e798a73 Mon Sep 17 00:00:00 2001 From: srao12 Date: Sun, 20 Jul 2025 20:46:14 -0700 Subject: [PATCH 18/30] fix: update proto Signed-off-by: srao12 --- examples/accumulator/streamsorter/Makefile | 4 +- .../proto/accumulator/accumulator.proto | 29 +- .../proto/accumulator/accumulator_pb2.py | 55 ++-- .../proto/accumulator/accumulator_pb2.pyi | 65 +--- .../proto/accumulator/accumulator_pb2_grpc.py | 127 +++----- pynumaflow/proto/mapper/map_pb2.py | 53 ++- pynumaflow/proto/mapper/map_pb2.pyi | 46 +-- pynumaflow/proto/mapper/map_pb2_grpc.py | 127 +++----- pynumaflow/proto/reducer/reduce_pb2.py | 53 ++- pynumaflow/proto/reducer/reduce_pb2.pyi | 58 +--- pynumaflow/proto/reducer/reduce_pb2_grpc.py | 127 +++----- pynumaflow/proto/sideinput/sideinput_pb2.py | 21 +- .../proto/sideinput/sideinput_pb2_grpc.py | 127 +++----- pynumaflow/proto/sinker/sink_pb2.py | 53 ++- pynumaflow/proto/sinker/sink_pb2.pyi | 46 +-- pynumaflow/proto/sinker/sink_pb2_grpc.py | 127 +++----- pynumaflow/proto/sourcer/source_pb2.py | 93 +++--- pynumaflow/proto/sourcer/source_pb2.pyi | 91 +----- pynumaflow/proto/sourcer/source_pb2_grpc.py | 304 +++++++----------- .../proto/sourcetransformer/transform_pb2.py | 45 ++- .../proto/sourcetransformer/transform_pb2.pyi | 45 +-- .../sourcetransformer/transform_pb2_grpc.py | 124 +++---- 22 files changed, 674 insertions(+), 1146 deletions(-) diff --git a/examples/accumulator/streamsorter/Makefile b/examples/accumulator/streamsorter/Makefile index 5eb6a3e8..77cde191 100644 --- a/examples/accumulator/streamsorter/Makefile +++ b/examples/accumulator/streamsorter/Makefile @@ -1,6 +1,6 @@ -TAG ?= stable +TAG ?= test4 PUSH ?= false -IMAGE_REGISTRY = quay.io/numaio/numaflow-python/streamsorter:${TAG} +IMAGE_REGISTRY = docker.intuit.com/personal/srao12/py-accumulator-sorter:${TAG} DOCKER_FILE_PATH = examples/accumulator/streamsorter/Dockerfile .PHONY: update diff --git a/pynumaflow/proto/accumulator/accumulator.proto b/pynumaflow/proto/accumulator/accumulator.proto index f25691c1..acde986b 100644 --- a/pynumaflow/proto/accumulator/accumulator.proto +++ b/pynumaflow/proto/accumulator/accumulator.proto @@ -1,6 +1,23 @@ +/* +Copyright 2022 The Numaproj Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + syntax = "proto3"; option go_package = "github.com/numaproj/numaflow-go/pkg/apis/proto/accumulator/v1"; +option java_package = "io.numaproj.numaflow.accumulator.v1"; import "google/protobuf/empty.proto"; import "google/protobuf/timestamp.proto"; @@ -45,7 +62,6 @@ message AccumulatorRequest { Payload payload = 1; WindowOperation operation = 2; - optional Handshake handshake = 3; } @@ -63,19 +79,12 @@ message AccumulatorResponse { // window represents a window to which the result belongs. KeyedWindow window = 2; repeated string tags = 3; - optional Handshake handshake = 4; // EOF represents the end of the response for a window. - bool EOF = 5; + bool EOF = 4; } // ReadyResponse is the health check result. message ReadyResponse { bool ready = 1; -} - -// Handshake message between client and server to indicate the start of transmission. -message Handshake { - // Required field indicating the start of transmission. - bool sot = 1; -} +} \ No newline at end of file diff --git a/pynumaflow/proto/accumulator/accumulator_pb2.py b/pynumaflow/proto/accumulator/accumulator_pb2.py index 422aacc1..bc29e898 100644 --- a/pynumaflow/proto/accumulator/accumulator_pb2.py +++ b/pynumaflow/proto/accumulator/accumulator_pb2.py @@ -7,7 +7,6 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import symbol_database as _symbol_database from google.protobuf.internal import builder as _builder - # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() @@ -17,38 +16,32 @@ from google.protobuf import timestamp_pb2 as google_dot_protobuf_dot_timestamp__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x11\x61\x63\x63umulator.proto\x12\x0e\x61\x63\x63umulator.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto"\xf8\x01\n\x07Payload\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\n\n\x02id\x18\x05 \x01(\t\x12\x35\n\x07headers\x18\x06 \x03(\x0b\x32$.accumulator.v1.Payload.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01"\xff\x02\n\x12\x41\x63\x63umulatorRequest\x12(\n\x07payload\x18\x01 \x01(\x0b\x32\x17.accumulator.v1.Payload\x12\x45\n\toperation\x18\x02 \x01(\x0b\x32\x32.accumulator.v1.AccumulatorRequest.WindowOperation\x12\x31\n\thandshake\x18\x03 \x01(\x0b\x32\x19.accumulator.v1.HandshakeH\x00\x88\x01\x01\x1a\xb6\x01\n\x0fWindowOperation\x12G\n\x05\x65vent\x18\x01 \x01(\x0e\x32\x38.accumulator.v1.AccumulatorRequest.WindowOperation.Event\x12\x30\n\x0bkeyedWindow\x18\x02 \x01(\x0b\x32\x1b.accumulator.v1.KeyedWindow"(\n\x05\x45vent\x12\x08\n\x04OPEN\x10\x00\x12\t\n\x05\x43LOSE\x10\x01\x12\n\n\x06\x41PPEND\x10\x02\x42\x0c\n\n_handshake"}\n\x0bKeyedWindow\x12)\n\x05start\x18\x01 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\'\n\x03\x65nd\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04slot\x18\x03 \x01(\t\x12\x0c\n\x04keys\x18\x04 \x03(\t"\xc8\x01\n\x13\x41\x63\x63umulatorResponse\x12(\n\x07payload\x18\x01 \x01(\x0b\x32\x17.accumulator.v1.Payload\x12+\n\x06window\x18\x02 \x01(\x0b\x32\x1b.accumulator.v1.KeyedWindow\x12\x0c\n\x04tags\x18\x03 \x03(\t\x12\x31\n\thandshake\x18\x04 \x01(\x0b\x32\x19.accumulator.v1.HandshakeH\x00\x88\x01\x01\x12\x0b\n\x03\x45OF\x18\x05 \x01(\x08\x42\x0c\n\n_handshake"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08\x32\xac\x01\n\x0b\x41\x63\x63umulator\x12[\n\x0c\x41\x63\x63umulateFn\x12".accumulator.v1.AccumulatorRequest\x1a#.accumulator.v1.AccumulatorResponse(\x01\x30\x01\x12@\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x1d.accumulator.v1.ReadyResponseB?Z=github.com/numaproj/numaflow-go/pkg/apis/proto/accumulator/v1b\x06proto3' -) +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x11\x61\x63\x63umulator.proto\x12\x0e\x61\x63\x63umulator.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto\"\xf8\x01\n\x07Payload\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\n\n\x02id\x18\x05 \x01(\t\x12\x35\n\x07headers\x18\x06 \x03(\x0b\x32$.accumulator.v1.Payload.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\"\xbe\x02\n\x12\x41\x63\x63umulatorRequest\x12(\n\x07payload\x18\x01 \x01(\x0b\x32\x17.accumulator.v1.Payload\x12\x45\n\toperation\x18\x02 \x01(\x0b\x32\x32.accumulator.v1.AccumulatorRequest.WindowOperation\x1a\xb6\x01\n\x0fWindowOperation\x12G\n\x05\x65vent\x18\x01 \x01(\x0e\x32\x38.accumulator.v1.AccumulatorRequest.WindowOperation.Event\x12\x30\n\x0bkeyedWindow\x18\x02 \x01(\x0b\x32\x1b.accumulator.v1.KeyedWindow\"(\n\x05\x45vent\x12\x08\n\x04OPEN\x10\x00\x12\t\n\x05\x43LOSE\x10\x01\x12\n\n\x06\x41PPEND\x10\x02\"}\n\x0bKeyedWindow\x12)\n\x05start\x18\x01 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\'\n\x03\x65nd\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04slot\x18\x03 \x01(\t\x12\x0c\n\x04keys\x18\x04 \x03(\t\"\x87\x01\n\x13\x41\x63\x63umulatorResponse\x12(\n\x07payload\x18\x01 \x01(\x0b\x32\x17.accumulator.v1.Payload\x12+\n\x06window\x18\x02 \x01(\x0b\x32\x1b.accumulator.v1.KeyedWindow\x12\x0c\n\x04tags\x18\x03 \x03(\t\x12\x0b\n\x03\x45OF\x18\x04 \x01(\x08\"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\xac\x01\n\x0b\x41\x63\x63umulator\x12[\n\x0c\x41\x63\x63umulateFn\x12\".accumulator.v1.AccumulatorRequest\x1a#.accumulator.v1.AccumulatorResponse(\x01\x30\x01\x12@\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x1d.accumulator.v1.ReadyResponseBd\n#io.numaproj.numaflow.accumulator.v1Z=github.com/numaproj/numaflow-go/pkg/apis/proto/accumulator/v1b\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) -_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, "accumulator_pb2", _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'accumulator_pb2', _globals) if _descriptor._USE_C_DESCRIPTORS == False: - _globals["DESCRIPTOR"]._options = None - _globals[ - "DESCRIPTOR" - ]._serialized_options = b"Z=github.com/numaproj/numaflow-go/pkg/apis/proto/accumulator/v1" - _globals["_PAYLOAD_HEADERSENTRY"]._options = None - _globals["_PAYLOAD_HEADERSENTRY"]._serialized_options = b"8\001" - _globals["_PAYLOAD"]._serialized_start = 100 - _globals["_PAYLOAD"]._serialized_end = 348 - _globals["_PAYLOAD_HEADERSENTRY"]._serialized_start = 302 - _globals["_PAYLOAD_HEADERSENTRY"]._serialized_end = 348 - _globals["_ACCUMULATORREQUEST"]._serialized_start = 351 - _globals["_ACCUMULATORREQUEST"]._serialized_end = 734 - _globals["_ACCUMULATORREQUEST_WINDOWOPERATION"]._serialized_start = 538 - _globals["_ACCUMULATORREQUEST_WINDOWOPERATION"]._serialized_end = 720 - _globals["_ACCUMULATORREQUEST_WINDOWOPERATION_EVENT"]._serialized_start = 680 - _globals["_ACCUMULATORREQUEST_WINDOWOPERATION_EVENT"]._serialized_end = 720 - _globals["_KEYEDWINDOW"]._serialized_start = 736 - _globals["_KEYEDWINDOW"]._serialized_end = 861 - _globals["_ACCUMULATORRESPONSE"]._serialized_start = 864 - _globals["_ACCUMULATORRESPONSE"]._serialized_end = 1064 - _globals["_READYRESPONSE"]._serialized_start = 1066 - _globals["_READYRESPONSE"]._serialized_end = 1096 - _globals["_HANDSHAKE"]._serialized_start = 1098 - _globals["_HANDSHAKE"]._serialized_end = 1122 - _globals["_ACCUMULATOR"]._serialized_start = 1125 - _globals["_ACCUMULATOR"]._serialized_end = 1297 + _globals['DESCRIPTOR']._options = None + _globals['DESCRIPTOR']._serialized_options = b'\n#io.numaproj.numaflow.accumulator.v1Z=github.com/numaproj/numaflow-go/pkg/apis/proto/accumulator/v1' + _globals['_PAYLOAD_HEADERSENTRY']._options = None + _globals['_PAYLOAD_HEADERSENTRY']._serialized_options = b'8\001' + _globals['_PAYLOAD']._serialized_start=100 + _globals['_PAYLOAD']._serialized_end=348 + _globals['_PAYLOAD_HEADERSENTRY']._serialized_start=302 + _globals['_PAYLOAD_HEADERSENTRY']._serialized_end=348 + _globals['_ACCUMULATORREQUEST']._serialized_start=351 + _globals['_ACCUMULATORREQUEST']._serialized_end=669 + _globals['_ACCUMULATORREQUEST_WINDOWOPERATION']._serialized_start=487 + _globals['_ACCUMULATORREQUEST_WINDOWOPERATION']._serialized_end=669 + _globals['_ACCUMULATORREQUEST_WINDOWOPERATION_EVENT']._serialized_start=629 + _globals['_ACCUMULATORREQUEST_WINDOWOPERATION_EVENT']._serialized_end=669 + _globals['_KEYEDWINDOW']._serialized_start=671 + _globals['_KEYEDWINDOW']._serialized_end=796 + _globals['_ACCUMULATORRESPONSE']._serialized_start=799 + _globals['_ACCUMULATORRESPONSE']._serialized_end=934 + _globals['_READYRESPONSE']._serialized_start=936 + _globals['_READYRESPONSE']._serialized_end=966 + _globals['_ACCUMULATOR']._serialized_start=969 + _globals['_ACCUMULATOR']._serialized_end=1141 # @@protoc_insertion_point(module_scope) diff --git a/pynumaflow/proto/accumulator/accumulator_pb2.pyi b/pynumaflow/proto/accumulator/accumulator_pb2.pyi index 5c893f90..bfa0bbac 100644 --- a/pynumaflow/proto/accumulator/accumulator_pb2.pyi +++ b/pynumaflow/proto/accumulator/accumulator_pb2.pyi @@ -4,19 +4,12 @@ from google.protobuf.internal import containers as _containers from google.protobuf.internal import enum_type_wrapper as _enum_type_wrapper from google.protobuf import descriptor as _descriptor from google.protobuf import message as _message -from typing import ( - ClassVar as _ClassVar, - Iterable as _Iterable, - Mapping as _Mapping, - Optional as _Optional, - Union as _Union, -) +from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union DESCRIPTOR: _descriptor.FileDescriptor class Payload(_message.Message): __slots__ = ("keys", "value", "event_time", "watermark", "id", "headers") - class HeadersEntry(_message.Message): __slots__ = ("key", "value") KEY_FIELD_NUMBER: _ClassVar[int] @@ -36,22 +29,12 @@ class Payload(_message.Message): watermark: _timestamp_pb2.Timestamp id: str headers: _containers.ScalarMap[str, str] - def __init__( - self, - keys: _Optional[_Iterable[str]] = ..., - value: _Optional[bytes] = ..., - event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - id: _Optional[str] = ..., - headers: _Optional[_Mapping[str, str]] = ..., - ) -> None: ... + def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., id: _Optional[str] = ..., headers: _Optional[_Mapping[str, str]] = ...) -> None: ... class AccumulatorRequest(_message.Message): - __slots__ = ("payload", "operation", "handshake") - + __slots__ = ("payload", "operation") class WindowOperation(_message.Message): __slots__ = ("event", "keyedWindow") - class Event(int, metaclass=_enum_type_wrapper.EnumTypeWrapper): __slots__ = () OPEN: _ClassVar[AccumulatorRequest.WindowOperation.Event] @@ -64,23 +47,12 @@ class AccumulatorRequest(_message.Message): KEYEDWINDOW_FIELD_NUMBER: _ClassVar[int] event: AccumulatorRequest.WindowOperation.Event keyedWindow: KeyedWindow - def __init__( - self, - event: _Optional[_Union[AccumulatorRequest.WindowOperation.Event, str]] = ..., - keyedWindow: _Optional[_Union[KeyedWindow, _Mapping]] = ..., - ) -> None: ... + def __init__(self, event: _Optional[_Union[AccumulatorRequest.WindowOperation.Event, str]] = ..., keyedWindow: _Optional[_Union[KeyedWindow, _Mapping]] = ...) -> None: ... PAYLOAD_FIELD_NUMBER: _ClassVar[int] OPERATION_FIELD_NUMBER: _ClassVar[int] - HANDSHAKE_FIELD_NUMBER: _ClassVar[int] payload: Payload operation: AccumulatorRequest.WindowOperation - handshake: Handshake - def __init__( - self, - payload: _Optional[_Union[Payload, _Mapping]] = ..., - operation: _Optional[_Union[AccumulatorRequest.WindowOperation, _Mapping]] = ..., - handshake: _Optional[_Union[Handshake, _Mapping]] = ..., - ) -> None: ... + def __init__(self, payload: _Optional[_Union[Payload, _Mapping]] = ..., operation: _Optional[_Union[AccumulatorRequest.WindowOperation, _Mapping]] = ...) -> None: ... class KeyedWindow(_message.Message): __slots__ = ("start", "end", "slot", "keys") @@ -92,43 +64,22 @@ class KeyedWindow(_message.Message): end: _timestamp_pb2.Timestamp slot: str keys: _containers.RepeatedScalarFieldContainer[str] - def __init__( - self, - start: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - end: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - slot: _Optional[str] = ..., - keys: _Optional[_Iterable[str]] = ..., - ) -> None: ... + def __init__(self, start: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., end: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., slot: _Optional[str] = ..., keys: _Optional[_Iterable[str]] = ...) -> None: ... class AccumulatorResponse(_message.Message): - __slots__ = ("payload", "window", "tags", "handshake", "EOF") + __slots__ = ("payload", "window", "tags", "EOF") PAYLOAD_FIELD_NUMBER: _ClassVar[int] WINDOW_FIELD_NUMBER: _ClassVar[int] TAGS_FIELD_NUMBER: _ClassVar[int] - HANDSHAKE_FIELD_NUMBER: _ClassVar[int] EOF_FIELD_NUMBER: _ClassVar[int] payload: Payload window: KeyedWindow tags: _containers.RepeatedScalarFieldContainer[str] - handshake: Handshake EOF: bool - def __init__( - self, - payload: _Optional[_Union[Payload, _Mapping]] = ..., - window: _Optional[_Union[KeyedWindow, _Mapping]] = ..., - tags: _Optional[_Iterable[str]] = ..., - handshake: _Optional[_Union[Handshake, _Mapping]] = ..., - EOF: bool = ..., - ) -> None: ... + def __init__(self, payload: _Optional[_Union[Payload, _Mapping]] = ..., window: _Optional[_Union[KeyedWindow, _Mapping]] = ..., tags: _Optional[_Iterable[str]] = ..., EOF: bool = ...) -> None: ... class ReadyResponse(_message.Message): __slots__ = ("ready",) READY_FIELD_NUMBER: _ClassVar[int] ready: bool def __init__(self, ready: bool = ...) -> None: ... - -class Handshake(_message.Message): - __slots__ = ("sot",) - SOT_FIELD_NUMBER: _ClassVar[int] - sot: bool - def __init__(self, sot: bool = ...) -> None: ... diff --git a/pynumaflow/proto/accumulator/accumulator_pb2_grpc.py b/pynumaflow/proto/accumulator/accumulator_pb2_grpc.py index f41606dd..4f0f208f 100644 --- a/pynumaflow/proto/accumulator/accumulator_pb2_grpc.py +++ b/pynumaflow/proto/accumulator/accumulator_pb2_grpc.py @@ -19,15 +19,15 @@ def __init__(self, channel): channel: A grpc.Channel. """ self.AccumulateFn = channel.stream_stream( - "/accumulator.v1.Accumulator/AccumulateFn", - request_serializer=accumulator__pb2.AccumulatorRequest.SerializeToString, - response_deserializer=accumulator__pb2.AccumulatorResponse.FromString, - ) + '/accumulator.v1.Accumulator/AccumulateFn', + request_serializer=accumulator__pb2.AccumulatorRequest.SerializeToString, + response_deserializer=accumulator__pb2.AccumulatorResponse.FromString, + ) self.IsReady = channel.unary_unary( - "/accumulator.v1.Accumulator/IsReady", - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=accumulator__pb2.ReadyResponse.FromString, - ) + '/accumulator.v1.Accumulator/IsReady', + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=accumulator__pb2.ReadyResponse.FromString, + ) class AccumulatorServicer(object): @@ -37,38 +37,39 @@ class AccumulatorServicer(object): """ def AccumulateFn(self, request_iterator, context): - """AccumulateFn applies a accumulate function to a request stream.""" + """AccumulateFn applies a accumulate function to a request stream. + """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def IsReady(self, request, context): - """IsReady is the heartbeat endpoint for gRPC.""" + """IsReady is the heartbeat endpoint for gRPC. + """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def add_AccumulatorServicer_to_server(servicer, server): rpc_method_handlers = { - "AccumulateFn": grpc.stream_stream_rpc_method_handler( - servicer.AccumulateFn, - request_deserializer=accumulator__pb2.AccumulatorRequest.FromString, - response_serializer=accumulator__pb2.AccumulatorResponse.SerializeToString, - ), - "IsReady": grpc.unary_unary_rpc_method_handler( - servicer.IsReady, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=accumulator__pb2.ReadyResponse.SerializeToString, - ), + 'AccumulateFn': grpc.stream_stream_rpc_method_handler( + servicer.AccumulateFn, + request_deserializer=accumulator__pb2.AccumulatorRequest.FromString, + response_serializer=accumulator__pb2.AccumulatorResponse.SerializeToString, + ), + 'IsReady': grpc.unary_unary_rpc_method_handler( + servicer.IsReady, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=accumulator__pb2.ReadyResponse.SerializeToString, + ), } generic_handler = grpc.method_handlers_generic_handler( - "accumulator.v1.Accumulator", rpc_method_handlers - ) + 'accumulator.v1.Accumulator', rpc_method_handlers) server.add_generic_rpc_handlers((generic_handler,)) -# This class is part of an EXPERIMENTAL API. + # This class is part of an EXPERIMENTAL API. class Accumulator(object): """AccumulatorWindow describes a special kind of SessionWindow (similar to Global Window) where output should always have monotonically increasing WM but it can be manipulated through event-time by reordering the messages. @@ -76,59 +77,35 @@ class Accumulator(object): """ @staticmethod - def AccumulateFn( - request_iterator, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.stream_stream( - request_iterator, + def AccumulateFn(request_iterator, target, - "/accumulator.v1.Accumulator/AccumulateFn", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.stream_stream(request_iterator, target, '/accumulator.v1.Accumulator/AccumulateFn', accumulator__pb2.AccumulatorRequest.SerializeToString, accumulator__pb2.AccumulatorResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) @staticmethod - def IsReady( - request, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.unary_unary( - request, + def IsReady(request, target, - "/accumulator.v1.Accumulator/IsReady", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.unary_unary(request, target, '/accumulator.v1.Accumulator/IsReady', google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, accumulator__pb2.ReadyResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) diff --git a/pynumaflow/proto/mapper/map_pb2.py b/pynumaflow/proto/mapper/map_pb2.py index 80472786..78abbd69 100644 --- a/pynumaflow/proto/mapper/map_pb2.py +++ b/pynumaflow/proto/mapper/map_pb2.py @@ -7,7 +7,6 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import symbol_database as _symbol_database from google.protobuf.internal import builder as _builder - # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() @@ -17,36 +16,32 @@ from google.protobuf import timestamp_pb2 as google_dot_protobuf_dot_timestamp__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\tmap.proto\x12\x06map.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto"\xac\x03\n\nMapRequest\x12+\n\x07request\x18\x01 \x01(\x0b\x32\x1a.map.v1.MapRequest.Request\x12\n\n\x02id\x18\x02 \x01(\t\x12)\n\thandshake\x18\x03 \x01(\x0b\x32\x11.map.v1.HandshakeH\x00\x88\x01\x01\x12/\n\x06status\x18\x04 \x01(\x0b\x32\x1a.map.v1.TransmissionStatusH\x01\x88\x01\x01\x1a\xef\x01\n\x07Request\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x38\n\x07headers\x18\x05 \x03(\x0b\x32\'.map.v1.MapRequest.Request.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x42\x0c\n\n_handshakeB\t\n\x07_status"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08"!\n\x12TransmissionStatus\x12\x0b\n\x03\x65ot\x18\x01 \x01(\x08"\xf0\x01\n\x0bMapResponse\x12+\n\x07results\x18\x01 \x03(\x0b\x32\x1a.map.v1.MapResponse.Result\x12\n\n\x02id\x18\x02 \x01(\t\x12)\n\thandshake\x18\x03 \x01(\x0b\x32\x11.map.v1.HandshakeH\x00\x88\x01\x01\x12/\n\x06status\x18\x04 \x01(\x0b\x32\x1a.map.v1.TransmissionStatusH\x01\x88\x01\x01\x1a\x33\n\x06Result\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12\x0c\n\x04tags\x18\x03 \x03(\tB\x0c\n\n_handshakeB\t\n\x07_status"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32u\n\x03Map\x12\x34\n\x05MapFn\x12\x12.map.v1.MapRequest\x1a\x13.map.v1.MapResponse(\x01\x30\x01\x12\x38\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x15.map.v1.ReadyResponseB7Z5github.com/numaproj/numaflow-go/pkg/apis/proto/map/v1b\x06proto3' -) +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\tmap.proto\x12\x06map.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto\"\xac\x03\n\nMapRequest\x12+\n\x07request\x18\x01 \x01(\x0b\x32\x1a.map.v1.MapRequest.Request\x12\n\n\x02id\x18\x02 \x01(\t\x12)\n\thandshake\x18\x03 \x01(\x0b\x32\x11.map.v1.HandshakeH\x00\x88\x01\x01\x12/\n\x06status\x18\x04 \x01(\x0b\x32\x1a.map.v1.TransmissionStatusH\x01\x88\x01\x01\x1a\xef\x01\n\x07Request\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x38\n\x07headers\x18\x05 \x03(\x0b\x32\'.map.v1.MapRequest.Request.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x42\x0c\n\n_handshakeB\t\n\x07_status\"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08\"!\n\x12TransmissionStatus\x12\x0b\n\x03\x65ot\x18\x01 \x01(\x08\"\xf0\x01\n\x0bMapResponse\x12+\n\x07results\x18\x01 \x03(\x0b\x32\x1a.map.v1.MapResponse.Result\x12\n\n\x02id\x18\x02 \x01(\t\x12)\n\thandshake\x18\x03 \x01(\x0b\x32\x11.map.v1.HandshakeH\x00\x88\x01\x01\x12/\n\x06status\x18\x04 \x01(\x0b\x32\x1a.map.v1.TransmissionStatusH\x01\x88\x01\x01\x1a\x33\n\x06Result\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12\x0c\n\x04tags\x18\x03 \x03(\tB\x0c\n\n_handshakeB\t\n\x07_status\"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32u\n\x03Map\x12\x34\n\x05MapFn\x12\x12.map.v1.MapRequest\x1a\x13.map.v1.MapResponse(\x01\x30\x01\x12\x38\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x15.map.v1.ReadyResponseB7Z5github.com/numaproj/numaflow-go/pkg/apis/proto/map/v1b\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) -_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, "map_pb2", _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'map_pb2', _globals) if _descriptor._USE_C_DESCRIPTORS == False: - _globals["DESCRIPTOR"]._options = None - _globals[ - "DESCRIPTOR" - ]._serialized_options = b"Z5github.com/numaproj/numaflow-go/pkg/apis/proto/map/v1" - _globals["_MAPREQUEST_REQUEST_HEADERSENTRY"]._options = None - _globals["_MAPREQUEST_REQUEST_HEADERSENTRY"]._serialized_options = b"8\001" - _globals["_MAPREQUEST"]._serialized_start = 84 - _globals["_MAPREQUEST"]._serialized_end = 512 - _globals["_MAPREQUEST_REQUEST"]._serialized_start = 248 - _globals["_MAPREQUEST_REQUEST"]._serialized_end = 487 - _globals["_MAPREQUEST_REQUEST_HEADERSENTRY"]._serialized_start = 441 - _globals["_MAPREQUEST_REQUEST_HEADERSENTRY"]._serialized_end = 487 - _globals["_HANDSHAKE"]._serialized_start = 514 - _globals["_HANDSHAKE"]._serialized_end = 538 - _globals["_TRANSMISSIONSTATUS"]._serialized_start = 540 - _globals["_TRANSMISSIONSTATUS"]._serialized_end = 573 - _globals["_MAPRESPONSE"]._serialized_start = 576 - _globals["_MAPRESPONSE"]._serialized_end = 816 - _globals["_MAPRESPONSE_RESULT"]._serialized_start = 740 - _globals["_MAPRESPONSE_RESULT"]._serialized_end = 791 - _globals["_READYRESPONSE"]._serialized_start = 818 - _globals["_READYRESPONSE"]._serialized_end = 848 - _globals["_MAP"]._serialized_start = 850 - _globals["_MAP"]._serialized_end = 967 + _globals['DESCRIPTOR']._options = None + _globals['DESCRIPTOR']._serialized_options = b'Z5github.com/numaproj/numaflow-go/pkg/apis/proto/map/v1' + _globals['_MAPREQUEST_REQUEST_HEADERSENTRY']._options = None + _globals['_MAPREQUEST_REQUEST_HEADERSENTRY']._serialized_options = b'8\001' + _globals['_MAPREQUEST']._serialized_start=84 + _globals['_MAPREQUEST']._serialized_end=512 + _globals['_MAPREQUEST_REQUEST']._serialized_start=248 + _globals['_MAPREQUEST_REQUEST']._serialized_end=487 + _globals['_MAPREQUEST_REQUEST_HEADERSENTRY']._serialized_start=441 + _globals['_MAPREQUEST_REQUEST_HEADERSENTRY']._serialized_end=487 + _globals['_HANDSHAKE']._serialized_start=514 + _globals['_HANDSHAKE']._serialized_end=538 + _globals['_TRANSMISSIONSTATUS']._serialized_start=540 + _globals['_TRANSMISSIONSTATUS']._serialized_end=573 + _globals['_MAPRESPONSE']._serialized_start=576 + _globals['_MAPRESPONSE']._serialized_end=816 + _globals['_MAPRESPONSE_RESULT']._serialized_start=740 + _globals['_MAPRESPONSE_RESULT']._serialized_end=791 + _globals['_READYRESPONSE']._serialized_start=818 + _globals['_READYRESPONSE']._serialized_end=848 + _globals['_MAP']._serialized_start=850 + _globals['_MAP']._serialized_end=967 # @@protoc_insertion_point(module_scope) diff --git a/pynumaflow/proto/mapper/map_pb2.pyi b/pynumaflow/proto/mapper/map_pb2.pyi index e1279ff0..22eed7b9 100644 --- a/pynumaflow/proto/mapper/map_pb2.pyi +++ b/pynumaflow/proto/mapper/map_pb2.pyi @@ -3,22 +3,14 @@ from google.protobuf import timestamp_pb2 as _timestamp_pb2 from google.protobuf.internal import containers as _containers from google.protobuf import descriptor as _descriptor from google.protobuf import message as _message -from typing import ( - ClassVar as _ClassVar, - Iterable as _Iterable, - Mapping as _Mapping, - Optional as _Optional, - Union as _Union, -) +from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union DESCRIPTOR: _descriptor.FileDescriptor class MapRequest(_message.Message): __slots__ = ("request", "id", "handshake", "status") - class Request(_message.Message): __slots__ = ("keys", "value", "event_time", "watermark", "headers") - class HeadersEntry(_message.Message): __slots__ = ("key", "value") KEY_FIELD_NUMBER: _ClassVar[int] @@ -26,7 +18,6 @@ class MapRequest(_message.Message): key: str value: str def __init__(self, key: _Optional[str] = ..., value: _Optional[str] = ...) -> None: ... - KEYS_FIELD_NUMBER: _ClassVar[int] VALUE_FIELD_NUMBER: _ClassVar[int] EVENT_TIME_FIELD_NUMBER: _ClassVar[int] @@ -37,15 +28,7 @@ class MapRequest(_message.Message): event_time: _timestamp_pb2.Timestamp watermark: _timestamp_pb2.Timestamp headers: _containers.ScalarMap[str, str] - def __init__( - self, - keys: _Optional[_Iterable[str]] = ..., - value: _Optional[bytes] = ..., - event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - headers: _Optional[_Mapping[str, str]] = ..., - ) -> None: ... - + def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., headers: _Optional[_Mapping[str, str]] = ...) -> None: ... REQUEST_FIELD_NUMBER: _ClassVar[int] ID_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] @@ -54,13 +37,7 @@ class MapRequest(_message.Message): id: str handshake: Handshake status: TransmissionStatus - def __init__( - self, - request: _Optional[_Union[MapRequest.Request, _Mapping]] = ..., - id: _Optional[str] = ..., - handshake: _Optional[_Union[Handshake, _Mapping]] = ..., - status: _Optional[_Union[TransmissionStatus, _Mapping]] = ..., - ) -> None: ... + def __init__(self, request: _Optional[_Union[MapRequest.Request, _Mapping]] = ..., id: _Optional[str] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ..., status: _Optional[_Union[TransmissionStatus, _Mapping]] = ...) -> None: ... class Handshake(_message.Message): __slots__ = ("sot",) @@ -76,7 +53,6 @@ class TransmissionStatus(_message.Message): class MapResponse(_message.Message): __slots__ = ("results", "id", "handshake", "status") - class Result(_message.Message): __slots__ = ("keys", "value", "tags") KEYS_FIELD_NUMBER: _ClassVar[int] @@ -85,13 +61,7 @@ class MapResponse(_message.Message): keys: _containers.RepeatedScalarFieldContainer[str] value: bytes tags: _containers.RepeatedScalarFieldContainer[str] - def __init__( - self, - keys: _Optional[_Iterable[str]] = ..., - value: _Optional[bytes] = ..., - tags: _Optional[_Iterable[str]] = ..., - ) -> None: ... - + def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., tags: _Optional[_Iterable[str]] = ...) -> None: ... RESULTS_FIELD_NUMBER: _ClassVar[int] ID_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] @@ -100,13 +70,7 @@ class MapResponse(_message.Message): id: str handshake: Handshake status: TransmissionStatus - def __init__( - self, - results: _Optional[_Iterable[_Union[MapResponse.Result, _Mapping]]] = ..., - id: _Optional[str] = ..., - handshake: _Optional[_Union[Handshake, _Mapping]] = ..., - status: _Optional[_Union[TransmissionStatus, _Mapping]] = ..., - ) -> None: ... + def __init__(self, results: _Optional[_Iterable[_Union[MapResponse.Result, _Mapping]]] = ..., id: _Optional[str] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ..., status: _Optional[_Union[TransmissionStatus, _Mapping]] = ...) -> None: ... class ReadyResponse(_message.Message): __slots__ = ("ready",) diff --git a/pynumaflow/proto/mapper/map_pb2_grpc.py b/pynumaflow/proto/mapper/map_pb2_grpc.py index 97834865..87fa0b27 100644 --- a/pynumaflow/proto/mapper/map_pb2_grpc.py +++ b/pynumaflow/proto/mapper/map_pb2_grpc.py @@ -16,108 +16,87 @@ def __init__(self, channel): channel: A grpc.Channel. """ self.MapFn = channel.stream_stream( - "/map.v1.Map/MapFn", - request_serializer=map__pb2.MapRequest.SerializeToString, - response_deserializer=map__pb2.MapResponse.FromString, - ) + '/map.v1.Map/MapFn', + request_serializer=map__pb2.MapRequest.SerializeToString, + response_deserializer=map__pb2.MapResponse.FromString, + ) self.IsReady = channel.unary_unary( - "/map.v1.Map/IsReady", - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=map__pb2.ReadyResponse.FromString, - ) + '/map.v1.Map/IsReady', + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=map__pb2.ReadyResponse.FromString, + ) class MapServicer(object): """Missing associated documentation comment in .proto file.""" def MapFn(self, request_iterator, context): - """MapFn applies a function to each map request element.""" + """MapFn applies a function to each map request element. + """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def IsReady(self, request, context): - """IsReady is the heartbeat endpoint for gRPC.""" + """IsReady is the heartbeat endpoint for gRPC. + """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def add_MapServicer_to_server(servicer, server): rpc_method_handlers = { - "MapFn": grpc.stream_stream_rpc_method_handler( - servicer.MapFn, - request_deserializer=map__pb2.MapRequest.FromString, - response_serializer=map__pb2.MapResponse.SerializeToString, - ), - "IsReady": grpc.unary_unary_rpc_method_handler( - servicer.IsReady, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=map__pb2.ReadyResponse.SerializeToString, - ), + 'MapFn': grpc.stream_stream_rpc_method_handler( + servicer.MapFn, + request_deserializer=map__pb2.MapRequest.FromString, + response_serializer=map__pb2.MapResponse.SerializeToString, + ), + 'IsReady': grpc.unary_unary_rpc_method_handler( + servicer.IsReady, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=map__pb2.ReadyResponse.SerializeToString, + ), } - generic_handler = grpc.method_handlers_generic_handler("map.v1.Map", rpc_method_handlers) + generic_handler = grpc.method_handlers_generic_handler( + 'map.v1.Map', rpc_method_handlers) server.add_generic_rpc_handlers((generic_handler,)) -# This class is part of an EXPERIMENTAL API. + # This class is part of an EXPERIMENTAL API. class Map(object): """Missing associated documentation comment in .proto file.""" @staticmethod - def MapFn( - request_iterator, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.stream_stream( - request_iterator, + def MapFn(request_iterator, target, - "/map.v1.Map/MapFn", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.stream_stream(request_iterator, target, '/map.v1.Map/MapFn', map__pb2.MapRequest.SerializeToString, map__pb2.MapResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) @staticmethod - def IsReady( - request, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.unary_unary( - request, + def IsReady(request, target, - "/map.v1.Map/IsReady", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.unary_unary(request, target, '/map.v1.Map/IsReady', google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, map__pb2.ReadyResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) diff --git a/pynumaflow/proto/reducer/reduce_pb2.py b/pynumaflow/proto/reducer/reduce_pb2.py index f5c1ef83..ecb076e3 100644 --- a/pynumaflow/proto/reducer/reduce_pb2.py +++ b/pynumaflow/proto/reducer/reduce_pb2.py @@ -7,7 +7,6 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import symbol_database as _symbol_database from google.protobuf.internal import builder as _builder - # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() @@ -17,35 +16,33 @@ from google.protobuf import timestamp_pb2 as google_dot_protobuf_dot_timestamp__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x0creduce.proto\x12\treduce.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto"\x98\x04\n\rReduceRequest\x12\x31\n\x07payload\x18\x01 \x01(\x0b\x32 .reduce.v1.ReduceRequest.Payload\x12;\n\toperation\x18\x02 \x01(\x0b\x32(.reduce.v1.ReduceRequest.WindowOperation\x1a\x9e\x01\n\x0fWindowOperation\x12=\n\x05\x65vent\x18\x01 \x01(\x0e\x32..reduce.v1.ReduceRequest.WindowOperation.Event\x12"\n\x07windows\x18\x02 \x03(\x0b\x32\x11.reduce.v1.Window"(\n\x05\x45vent\x12\x08\n\x04OPEN\x10\x00\x12\t\n\x05\x43LOSE\x10\x01\x12\n\n\x06\x41PPEND\x10\x04\x1a\xf5\x01\n\x07Payload\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12>\n\x07headers\x18\x05 \x03(\x0b\x32-.reduce.v1.ReduceRequest.Payload.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01"j\n\x06Window\x12)\n\x05start\x18\x01 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\'\n\x03\x65nd\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04slot\x18\x03 \x01(\t"\xa7\x01\n\x0eReduceResponse\x12\x30\n\x06result\x18\x01 \x01(\x0b\x32 .reduce.v1.ReduceResponse.Result\x12!\n\x06window\x18\x02 \x01(\x0b\x32\x11.reduce.v1.Window\x12\x0b\n\x03\x45OF\x18\x03 \x01(\x08\x1a\x33\n\x06Result\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12\x0c\n\x04tags\x18\x03 \x03(\t"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\x8a\x01\n\x06Reduce\x12\x43\n\x08ReduceFn\x12\x18.reduce.v1.ReduceRequest\x1a\x19.reduce.v1.ReduceResponse(\x01\x30\x01\x12;\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x18.reduce.v1.ReadyResponseb\x06proto3' -) +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x0creduce.proto\x12\treduce.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto\"\x98\x04\n\rReduceRequest\x12\x31\n\x07payload\x18\x01 \x01(\x0b\x32 .reduce.v1.ReduceRequest.Payload\x12;\n\toperation\x18\x02 \x01(\x0b\x32(.reduce.v1.ReduceRequest.WindowOperation\x1a\x9e\x01\n\x0fWindowOperation\x12=\n\x05\x65vent\x18\x01 \x01(\x0e\x32..reduce.v1.ReduceRequest.WindowOperation.Event\x12\"\n\x07windows\x18\x02 \x03(\x0b\x32\x11.reduce.v1.Window\"(\n\x05\x45vent\x12\x08\n\x04OPEN\x10\x00\x12\t\n\x05\x43LOSE\x10\x01\x12\n\n\x06\x41PPEND\x10\x04\x1a\xf5\x01\n\x07Payload\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12>\n\x07headers\x18\x05 \x03(\x0b\x32-.reduce.v1.ReduceRequest.Payload.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\"j\n\x06Window\x12)\n\x05start\x18\x01 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\'\n\x03\x65nd\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04slot\x18\x03 \x01(\t\"\xa7\x01\n\x0eReduceResponse\x12\x30\n\x06result\x18\x01 \x01(\x0b\x32 .reduce.v1.ReduceResponse.Result\x12!\n\x06window\x18\x02 \x01(\x0b\x32\x11.reduce.v1.Window\x12\x0b\n\x03\x45OF\x18\x03 \x01(\x08\x1a\x33\n\x06Result\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12\x0c\n\x04tags\x18\x03 \x03(\t\"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\x8a\x01\n\x06Reduce\x12\x43\n\x08ReduceFn\x12\x18.reduce.v1.ReduceRequest\x1a\x19.reduce.v1.ReduceResponse(\x01\x30\x01\x12;\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x18.reduce.v1.ReadyResponseb\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) -_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, "reduce_pb2", _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'reduce_pb2', _globals) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None - _globals["_REDUCEREQUEST_PAYLOAD_HEADERSENTRY"]._options = None - _globals["_REDUCEREQUEST_PAYLOAD_HEADERSENTRY"]._serialized_options = b"8\001" - _globals["_REDUCEREQUEST"]._serialized_start = 90 - _globals["_REDUCEREQUEST"]._serialized_end = 626 - _globals["_REDUCEREQUEST_WINDOWOPERATION"]._serialized_start = 220 - _globals["_REDUCEREQUEST_WINDOWOPERATION"]._serialized_end = 378 - _globals["_REDUCEREQUEST_WINDOWOPERATION_EVENT"]._serialized_start = 338 - _globals["_REDUCEREQUEST_WINDOWOPERATION_EVENT"]._serialized_end = 378 - _globals["_REDUCEREQUEST_PAYLOAD"]._serialized_start = 381 - _globals["_REDUCEREQUEST_PAYLOAD"]._serialized_end = 626 - _globals["_REDUCEREQUEST_PAYLOAD_HEADERSENTRY"]._serialized_start = 580 - _globals["_REDUCEREQUEST_PAYLOAD_HEADERSENTRY"]._serialized_end = 626 - _globals["_WINDOW"]._serialized_start = 628 - _globals["_WINDOW"]._serialized_end = 734 - _globals["_REDUCERESPONSE"]._serialized_start = 737 - _globals["_REDUCERESPONSE"]._serialized_end = 904 - _globals["_REDUCERESPONSE_RESULT"]._serialized_start = 853 - _globals["_REDUCERESPONSE_RESULT"]._serialized_end = 904 - _globals["_READYRESPONSE"]._serialized_start = 906 - _globals["_READYRESPONSE"]._serialized_end = 936 - _globals["_REDUCE"]._serialized_start = 939 - _globals["_REDUCE"]._serialized_end = 1077 + DESCRIPTOR._options = None + _globals['_REDUCEREQUEST_PAYLOAD_HEADERSENTRY']._options = None + _globals['_REDUCEREQUEST_PAYLOAD_HEADERSENTRY']._serialized_options = b'8\001' + _globals['_REDUCEREQUEST']._serialized_start=90 + _globals['_REDUCEREQUEST']._serialized_end=626 + _globals['_REDUCEREQUEST_WINDOWOPERATION']._serialized_start=220 + _globals['_REDUCEREQUEST_WINDOWOPERATION']._serialized_end=378 + _globals['_REDUCEREQUEST_WINDOWOPERATION_EVENT']._serialized_start=338 + _globals['_REDUCEREQUEST_WINDOWOPERATION_EVENT']._serialized_end=378 + _globals['_REDUCEREQUEST_PAYLOAD']._serialized_start=381 + _globals['_REDUCEREQUEST_PAYLOAD']._serialized_end=626 + _globals['_REDUCEREQUEST_PAYLOAD_HEADERSENTRY']._serialized_start=580 + _globals['_REDUCEREQUEST_PAYLOAD_HEADERSENTRY']._serialized_end=626 + _globals['_WINDOW']._serialized_start=628 + _globals['_WINDOW']._serialized_end=734 + _globals['_REDUCERESPONSE']._serialized_start=737 + _globals['_REDUCERESPONSE']._serialized_end=904 + _globals['_REDUCERESPONSE_RESULT']._serialized_start=853 + _globals['_REDUCERESPONSE_RESULT']._serialized_end=904 + _globals['_READYRESPONSE']._serialized_start=906 + _globals['_READYRESPONSE']._serialized_end=936 + _globals['_REDUCE']._serialized_start=939 + _globals['_REDUCE']._serialized_end=1077 # @@protoc_insertion_point(module_scope) diff --git a/pynumaflow/proto/reducer/reduce_pb2.pyi b/pynumaflow/proto/reducer/reduce_pb2.pyi index 2c4b248c..80a974ff 100644 --- a/pynumaflow/proto/reducer/reduce_pb2.pyi +++ b/pynumaflow/proto/reducer/reduce_pb2.pyi @@ -4,22 +4,14 @@ from google.protobuf.internal import containers as _containers from google.protobuf.internal import enum_type_wrapper as _enum_type_wrapper from google.protobuf import descriptor as _descriptor from google.protobuf import message as _message -from typing import ( - ClassVar as _ClassVar, - Iterable as _Iterable, - Mapping as _Mapping, - Optional as _Optional, - Union as _Union, -) +from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union DESCRIPTOR: _descriptor.FileDescriptor class ReduceRequest(_message.Message): __slots__ = ("payload", "operation") - class WindowOperation(_message.Message): __slots__ = ("event", "windows") - class Event(int, metaclass=_enum_type_wrapper.EnumTypeWrapper): __slots__ = () OPEN: _ClassVar[ReduceRequest.WindowOperation.Event] @@ -32,15 +24,9 @@ class ReduceRequest(_message.Message): WINDOWS_FIELD_NUMBER: _ClassVar[int] event: ReduceRequest.WindowOperation.Event windows: _containers.RepeatedCompositeFieldContainer[Window] - def __init__( - self, - event: _Optional[_Union[ReduceRequest.WindowOperation.Event, str]] = ..., - windows: _Optional[_Iterable[_Union[Window, _Mapping]]] = ..., - ) -> None: ... - + def __init__(self, event: _Optional[_Union[ReduceRequest.WindowOperation.Event, str]] = ..., windows: _Optional[_Iterable[_Union[Window, _Mapping]]] = ...) -> None: ... class Payload(_message.Message): __slots__ = ("keys", "value", "event_time", "watermark", "headers") - class HeadersEntry(_message.Message): __slots__ = ("key", "value") KEY_FIELD_NUMBER: _ClassVar[int] @@ -48,7 +34,6 @@ class ReduceRequest(_message.Message): key: str value: str def __init__(self, key: _Optional[str] = ..., value: _Optional[str] = ...) -> None: ... - KEYS_FIELD_NUMBER: _ClassVar[int] VALUE_FIELD_NUMBER: _ClassVar[int] EVENT_TIME_FIELD_NUMBER: _ClassVar[int] @@ -59,24 +44,12 @@ class ReduceRequest(_message.Message): event_time: _timestamp_pb2.Timestamp watermark: _timestamp_pb2.Timestamp headers: _containers.ScalarMap[str, str] - def __init__( - self, - keys: _Optional[_Iterable[str]] = ..., - value: _Optional[bytes] = ..., - event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - headers: _Optional[_Mapping[str, str]] = ..., - ) -> None: ... - + def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., headers: _Optional[_Mapping[str, str]] = ...) -> None: ... PAYLOAD_FIELD_NUMBER: _ClassVar[int] OPERATION_FIELD_NUMBER: _ClassVar[int] payload: ReduceRequest.Payload operation: ReduceRequest.WindowOperation - def __init__( - self, - payload: _Optional[_Union[ReduceRequest.Payload, _Mapping]] = ..., - operation: _Optional[_Union[ReduceRequest.WindowOperation, _Mapping]] = ..., - ) -> None: ... + def __init__(self, payload: _Optional[_Union[ReduceRequest.Payload, _Mapping]] = ..., operation: _Optional[_Union[ReduceRequest.WindowOperation, _Mapping]] = ...) -> None: ... class Window(_message.Message): __slots__ = ("start", "end", "slot") @@ -86,16 +59,10 @@ class Window(_message.Message): start: _timestamp_pb2.Timestamp end: _timestamp_pb2.Timestamp slot: str - def __init__( - self, - start: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - end: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - slot: _Optional[str] = ..., - ) -> None: ... + def __init__(self, start: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., end: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., slot: _Optional[str] = ...) -> None: ... class ReduceResponse(_message.Message): __slots__ = ("result", "window", "EOF") - class Result(_message.Message): __slots__ = ("keys", "value", "tags") KEYS_FIELD_NUMBER: _ClassVar[int] @@ -104,25 +71,14 @@ class ReduceResponse(_message.Message): keys: _containers.RepeatedScalarFieldContainer[str] value: bytes tags: _containers.RepeatedScalarFieldContainer[str] - def __init__( - self, - keys: _Optional[_Iterable[str]] = ..., - value: _Optional[bytes] = ..., - tags: _Optional[_Iterable[str]] = ..., - ) -> None: ... - + def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., tags: _Optional[_Iterable[str]] = ...) -> None: ... RESULT_FIELD_NUMBER: _ClassVar[int] WINDOW_FIELD_NUMBER: _ClassVar[int] EOF_FIELD_NUMBER: _ClassVar[int] result: ReduceResponse.Result window: Window EOF: bool - def __init__( - self, - result: _Optional[_Union[ReduceResponse.Result, _Mapping]] = ..., - window: _Optional[_Union[Window, _Mapping]] = ..., - EOF: bool = ..., - ) -> None: ... + def __init__(self, result: _Optional[_Union[ReduceResponse.Result, _Mapping]] = ..., window: _Optional[_Union[Window, _Mapping]] = ..., EOF: bool = ...) -> None: ... class ReadyResponse(_message.Message): __slots__ = ("ready",) diff --git a/pynumaflow/proto/reducer/reduce_pb2_grpc.py b/pynumaflow/proto/reducer/reduce_pb2_grpc.py index 5a0a15f6..1fd860b4 100644 --- a/pynumaflow/proto/reducer/reduce_pb2_grpc.py +++ b/pynumaflow/proto/reducer/reduce_pb2_grpc.py @@ -16,108 +16,87 @@ def __init__(self, channel): channel: A grpc.Channel. """ self.ReduceFn = channel.stream_stream( - "/reduce.v1.Reduce/ReduceFn", - request_serializer=reduce__pb2.ReduceRequest.SerializeToString, - response_deserializer=reduce__pb2.ReduceResponse.FromString, - ) + '/reduce.v1.Reduce/ReduceFn', + request_serializer=reduce__pb2.ReduceRequest.SerializeToString, + response_deserializer=reduce__pb2.ReduceResponse.FromString, + ) self.IsReady = channel.unary_unary( - "/reduce.v1.Reduce/IsReady", - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=reduce__pb2.ReadyResponse.FromString, - ) + '/reduce.v1.Reduce/IsReady', + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=reduce__pb2.ReadyResponse.FromString, + ) class ReduceServicer(object): """Missing associated documentation comment in .proto file.""" def ReduceFn(self, request_iterator, context): - """ReduceFn applies a reduce function to a request stream.""" + """ReduceFn applies a reduce function to a request stream. + """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def IsReady(self, request, context): - """IsReady is the heartbeat endpoint for gRPC.""" + """IsReady is the heartbeat endpoint for gRPC. + """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def add_ReduceServicer_to_server(servicer, server): rpc_method_handlers = { - "ReduceFn": grpc.stream_stream_rpc_method_handler( - servicer.ReduceFn, - request_deserializer=reduce__pb2.ReduceRequest.FromString, - response_serializer=reduce__pb2.ReduceResponse.SerializeToString, - ), - "IsReady": grpc.unary_unary_rpc_method_handler( - servicer.IsReady, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=reduce__pb2.ReadyResponse.SerializeToString, - ), + 'ReduceFn': grpc.stream_stream_rpc_method_handler( + servicer.ReduceFn, + request_deserializer=reduce__pb2.ReduceRequest.FromString, + response_serializer=reduce__pb2.ReduceResponse.SerializeToString, + ), + 'IsReady': grpc.unary_unary_rpc_method_handler( + servicer.IsReady, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=reduce__pb2.ReadyResponse.SerializeToString, + ), } - generic_handler = grpc.method_handlers_generic_handler("reduce.v1.Reduce", rpc_method_handlers) + generic_handler = grpc.method_handlers_generic_handler( + 'reduce.v1.Reduce', rpc_method_handlers) server.add_generic_rpc_handlers((generic_handler,)) -# This class is part of an EXPERIMENTAL API. + # This class is part of an EXPERIMENTAL API. class Reduce(object): """Missing associated documentation comment in .proto file.""" @staticmethod - def ReduceFn( - request_iterator, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.stream_stream( - request_iterator, + def ReduceFn(request_iterator, target, - "/reduce.v1.Reduce/ReduceFn", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.stream_stream(request_iterator, target, '/reduce.v1.Reduce/ReduceFn', reduce__pb2.ReduceRequest.SerializeToString, reduce__pb2.ReduceResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) @staticmethod - def IsReady( - request, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.unary_unary( - request, + def IsReady(request, target, - "/reduce.v1.Reduce/IsReady", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.unary_unary(request, target, '/reduce.v1.Reduce/IsReady', google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, reduce__pb2.ReadyResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) diff --git a/pynumaflow/proto/sideinput/sideinput_pb2.py b/pynumaflow/proto/sideinput/sideinput_pb2.py index c0dab051..0160315f 100644 --- a/pynumaflow/proto/sideinput/sideinput_pb2.py +++ b/pynumaflow/proto/sideinput/sideinput_pb2.py @@ -7,7 +7,6 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import symbol_database as _symbol_database from google.protobuf.internal import builder as _builder - # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() @@ -16,19 +15,17 @@ from google.protobuf import empty_pb2 as google_dot_protobuf_dot_empty__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x0fsideinput.proto\x12\x0csideinput.v1\x1a\x1bgoogle/protobuf/empty.proto"8\n\x11SideInputResponse\x12\r\n\x05value\x18\x01 \x01(\x0c\x12\x14\n\x0cno_broadcast\x18\x02 \x01(\x08"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\x99\x01\n\tSideInput\x12L\n\x11RetrieveSideInput\x12\x16.google.protobuf.Empty\x1a\x1f.sideinput.v1.SideInputResponse\x12>\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x1b.sideinput.v1.ReadyResponseb\x06proto3' -) +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x0fsideinput.proto\x12\x0csideinput.v1\x1a\x1bgoogle/protobuf/empty.proto\"8\n\x11SideInputResponse\x12\r\n\x05value\x18\x01 \x01(\x0c\x12\x14\n\x0cno_broadcast\x18\x02 \x01(\x08\"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\x99\x01\n\tSideInput\x12L\n\x11RetrieveSideInput\x12\x16.google.protobuf.Empty\x1a\x1f.sideinput.v1.SideInputResponse\x12>\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x1b.sideinput.v1.ReadyResponseb\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) -_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, "sideinput_pb2", _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'sideinput_pb2', _globals) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None - _globals["_SIDEINPUTRESPONSE"]._serialized_start = 62 - _globals["_SIDEINPUTRESPONSE"]._serialized_end = 118 - _globals["_READYRESPONSE"]._serialized_start = 120 - _globals["_READYRESPONSE"]._serialized_end = 150 - _globals["_SIDEINPUT"]._serialized_start = 153 - _globals["_SIDEINPUT"]._serialized_end = 306 + DESCRIPTOR._options = None + _globals['_SIDEINPUTRESPONSE']._serialized_start=62 + _globals['_SIDEINPUTRESPONSE']._serialized_end=118 + _globals['_READYRESPONSE']._serialized_start=120 + _globals['_READYRESPONSE']._serialized_end=150 + _globals['_SIDEINPUT']._serialized_start=153 + _globals['_SIDEINPUT']._serialized_end=306 # @@protoc_insertion_point(module_scope) diff --git a/pynumaflow/proto/sideinput/sideinput_pb2_grpc.py b/pynumaflow/proto/sideinput/sideinput_pb2_grpc.py index 72ea87ed..8abe64d2 100644 --- a/pynumaflow/proto/sideinput/sideinput_pb2_grpc.py +++ b/pynumaflow/proto/sideinput/sideinput_pb2_grpc.py @@ -24,15 +24,15 @@ def __init__(self, channel): channel: A grpc.Channel. """ self.RetrieveSideInput = channel.unary_unary( - "/sideinput.v1.SideInput/RetrieveSideInput", - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=sideinput__pb2.SideInputResponse.FromString, - ) + '/sideinput.v1.SideInput/RetrieveSideInput', + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=sideinput__pb2.SideInputResponse.FromString, + ) self.IsReady = channel.unary_unary( - "/sideinput.v1.SideInput/IsReady", - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=sideinput__pb2.ReadyResponse.FromString, - ) + '/sideinput.v1.SideInput/IsReady', + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=sideinput__pb2.ReadyResponse.FromString, + ) class SideInputServicer(object): @@ -47,38 +47,39 @@ class SideInputServicer(object): """ def RetrieveSideInput(self, request, context): - """RetrieveSideInput is the endpoint to retrieve the latest value of a given Side Input.""" + """RetrieveSideInput is the endpoint to retrieve the latest value of a given Side Input. + """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def IsReady(self, request, context): - """IsReady is the health check endpoint to indicate whether the service is ready to be used.""" + """IsReady is the health check endpoint to indicate whether the service is ready to be used. + """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def add_SideInputServicer_to_server(servicer, server): rpc_method_handlers = { - "RetrieveSideInput": grpc.unary_unary_rpc_method_handler( - servicer.RetrieveSideInput, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=sideinput__pb2.SideInputResponse.SerializeToString, - ), - "IsReady": grpc.unary_unary_rpc_method_handler( - servicer.IsReady, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=sideinput__pb2.ReadyResponse.SerializeToString, - ), + 'RetrieveSideInput': grpc.unary_unary_rpc_method_handler( + servicer.RetrieveSideInput, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=sideinput__pb2.SideInputResponse.SerializeToString, + ), + 'IsReady': grpc.unary_unary_rpc_method_handler( + servicer.IsReady, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=sideinput__pb2.ReadyResponse.SerializeToString, + ), } generic_handler = grpc.method_handlers_generic_handler( - "sideinput.v1.SideInput", rpc_method_handlers - ) + 'sideinput.v1.SideInput', rpc_method_handlers) server.add_generic_rpc_handlers((generic_handler,)) -# This class is part of an EXPERIMENTAL API. + # This class is part of an EXPERIMENTAL API. class SideInput(object): """SideInput is the gRPC service for user-defined Side Inputs. It is used to propagate changes in the values of the provided Side Inputs @@ -91,59 +92,35 @@ class SideInput(object): """ @staticmethod - def RetrieveSideInput( - request, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.unary_unary( - request, + def RetrieveSideInput(request, target, - "/sideinput.v1.SideInput/RetrieveSideInput", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.unary_unary(request, target, '/sideinput.v1.SideInput/RetrieveSideInput', google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, sideinput__pb2.SideInputResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) @staticmethod - def IsReady( - request, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.unary_unary( - request, + def IsReady(request, target, - "/sideinput.v1.SideInput/IsReady", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.unary_unary(request, target, '/sideinput.v1.SideInput/IsReady', google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, sideinput__pb2.ReadyResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) diff --git a/pynumaflow/proto/sinker/sink_pb2.py b/pynumaflow/proto/sinker/sink_pb2.py index 27082a0e..67d905c5 100644 --- a/pynumaflow/proto/sinker/sink_pb2.py +++ b/pynumaflow/proto/sinker/sink_pb2.py @@ -7,7 +7,6 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import symbol_database as _symbol_database from google.protobuf.internal import builder as _builder - # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() @@ -17,35 +16,33 @@ from google.protobuf import timestamp_pb2 as google_dot_protobuf_dot_timestamp__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\nsink.proto\x12\x07sink.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto"\xa3\x03\n\x0bSinkRequest\x12-\n\x07request\x18\x01 \x01(\x0b\x32\x1c.sink.v1.SinkRequest.Request\x12+\n\x06status\x18\x02 \x01(\x0b\x32\x1b.sink.v1.TransmissionStatus\x12*\n\thandshake\x18\x03 \x01(\x0b\x32\x12.sink.v1.HandshakeH\x00\x88\x01\x01\x1a\xfd\x01\n\x07Request\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\n\n\x02id\x18\x05 \x01(\t\x12:\n\x07headers\x18\x06 \x03(\x0b\x32).sink.v1.SinkRequest.Request.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x42\x0c\n\n_handshake"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08"!\n\x12TransmissionStatus\x12\x0b\n\x03\x65ot\x18\x01 \x01(\x08"\xfc\x01\n\x0cSinkResponse\x12-\n\x07results\x18\x01 \x03(\x0b\x32\x1c.sink.v1.SinkResponse.Result\x12*\n\thandshake\x18\x02 \x01(\x0b\x32\x12.sink.v1.HandshakeH\x00\x88\x01\x01\x12\x30\n\x06status\x18\x03 \x01(\x0b\x32\x1b.sink.v1.TransmissionStatusH\x01\x88\x01\x01\x1a\x46\n\x06Result\x12\n\n\x02id\x18\x01 \x01(\t\x12\x1f\n\x06status\x18\x02 \x01(\x0e\x32\x0f.sink.v1.Status\x12\x0f\n\x07\x65rr_msg\x18\x03 \x01(\tB\x0c\n\n_handshakeB\t\n\x07_status*0\n\x06Status\x12\x0b\n\x07SUCCESS\x10\x00\x12\x0b\n\x07\x46\x41ILURE\x10\x01\x12\x0c\n\x08\x46\x41LLBACK\x10\x02\x32|\n\x04Sink\x12\x39\n\x06SinkFn\x12\x14.sink.v1.SinkRequest\x1a\x15.sink.v1.SinkResponse(\x01\x30\x01\x12\x39\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x16.sink.v1.ReadyResponseb\x06proto3' -) +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\nsink.proto\x12\x07sink.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto\"\xa3\x03\n\x0bSinkRequest\x12-\n\x07request\x18\x01 \x01(\x0b\x32\x1c.sink.v1.SinkRequest.Request\x12+\n\x06status\x18\x02 \x01(\x0b\x32\x1b.sink.v1.TransmissionStatus\x12*\n\thandshake\x18\x03 \x01(\x0b\x32\x12.sink.v1.HandshakeH\x00\x88\x01\x01\x1a\xfd\x01\n\x07Request\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\n\n\x02id\x18\x05 \x01(\t\x12:\n\x07headers\x18\x06 \x03(\x0b\x32).sink.v1.SinkRequest.Request.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x42\x0c\n\n_handshake\"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08\"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\"!\n\x12TransmissionStatus\x12\x0b\n\x03\x65ot\x18\x01 \x01(\x08\"\xfc\x01\n\x0cSinkResponse\x12-\n\x07results\x18\x01 \x03(\x0b\x32\x1c.sink.v1.SinkResponse.Result\x12*\n\thandshake\x18\x02 \x01(\x0b\x32\x12.sink.v1.HandshakeH\x00\x88\x01\x01\x12\x30\n\x06status\x18\x03 \x01(\x0b\x32\x1b.sink.v1.TransmissionStatusH\x01\x88\x01\x01\x1a\x46\n\x06Result\x12\n\n\x02id\x18\x01 \x01(\t\x12\x1f\n\x06status\x18\x02 \x01(\x0e\x32\x0f.sink.v1.Status\x12\x0f\n\x07\x65rr_msg\x18\x03 \x01(\tB\x0c\n\n_handshakeB\t\n\x07_status*0\n\x06Status\x12\x0b\n\x07SUCCESS\x10\x00\x12\x0b\n\x07\x46\x41ILURE\x10\x01\x12\x0c\n\x08\x46\x41LLBACK\x10\x02\x32|\n\x04Sink\x12\x39\n\x06SinkFn\x12\x14.sink.v1.SinkRequest\x1a\x15.sink.v1.SinkResponse(\x01\x30\x01\x12\x39\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x16.sink.v1.ReadyResponseb\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) -_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, "sink_pb2", _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'sink_pb2', _globals) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None - _globals["_SINKREQUEST_REQUEST_HEADERSENTRY"]._options = None - _globals["_SINKREQUEST_REQUEST_HEADERSENTRY"]._serialized_options = b"8\001" - _globals["_STATUS"]._serialized_start = 855 - _globals["_STATUS"]._serialized_end = 903 - _globals["_SINKREQUEST"]._serialized_start = 86 - _globals["_SINKREQUEST"]._serialized_end = 505 - _globals["_SINKREQUEST_REQUEST"]._serialized_start = 238 - _globals["_SINKREQUEST_REQUEST"]._serialized_end = 491 - _globals["_SINKREQUEST_REQUEST_HEADERSENTRY"]._serialized_start = 445 - _globals["_SINKREQUEST_REQUEST_HEADERSENTRY"]._serialized_end = 491 - _globals["_HANDSHAKE"]._serialized_start = 507 - _globals["_HANDSHAKE"]._serialized_end = 531 - _globals["_READYRESPONSE"]._serialized_start = 533 - _globals["_READYRESPONSE"]._serialized_end = 563 - _globals["_TRANSMISSIONSTATUS"]._serialized_start = 565 - _globals["_TRANSMISSIONSTATUS"]._serialized_end = 598 - _globals["_SINKRESPONSE"]._serialized_start = 601 - _globals["_SINKRESPONSE"]._serialized_end = 853 - _globals["_SINKRESPONSE_RESULT"]._serialized_start = 758 - _globals["_SINKRESPONSE_RESULT"]._serialized_end = 828 - _globals["_SINK"]._serialized_start = 905 - _globals["_SINK"]._serialized_end = 1029 + DESCRIPTOR._options = None + _globals['_SINKREQUEST_REQUEST_HEADERSENTRY']._options = None + _globals['_SINKREQUEST_REQUEST_HEADERSENTRY']._serialized_options = b'8\001' + _globals['_STATUS']._serialized_start=855 + _globals['_STATUS']._serialized_end=903 + _globals['_SINKREQUEST']._serialized_start=86 + _globals['_SINKREQUEST']._serialized_end=505 + _globals['_SINKREQUEST_REQUEST']._serialized_start=238 + _globals['_SINKREQUEST_REQUEST']._serialized_end=491 + _globals['_SINKREQUEST_REQUEST_HEADERSENTRY']._serialized_start=445 + _globals['_SINKREQUEST_REQUEST_HEADERSENTRY']._serialized_end=491 + _globals['_HANDSHAKE']._serialized_start=507 + _globals['_HANDSHAKE']._serialized_end=531 + _globals['_READYRESPONSE']._serialized_start=533 + _globals['_READYRESPONSE']._serialized_end=563 + _globals['_TRANSMISSIONSTATUS']._serialized_start=565 + _globals['_TRANSMISSIONSTATUS']._serialized_end=598 + _globals['_SINKRESPONSE']._serialized_start=601 + _globals['_SINKRESPONSE']._serialized_end=853 + _globals['_SINKRESPONSE_RESULT']._serialized_start=758 + _globals['_SINKRESPONSE_RESULT']._serialized_end=828 + _globals['_SINK']._serialized_start=905 + _globals['_SINK']._serialized_end=1029 # @@protoc_insertion_point(module_scope) diff --git a/pynumaflow/proto/sinker/sink_pb2.pyi b/pynumaflow/proto/sinker/sink_pb2.pyi index 18d4d3b6..6c0292ca 100644 --- a/pynumaflow/proto/sinker/sink_pb2.pyi +++ b/pynumaflow/proto/sinker/sink_pb2.pyi @@ -4,13 +4,7 @@ from google.protobuf.internal import containers as _containers from google.protobuf.internal import enum_type_wrapper as _enum_type_wrapper from google.protobuf import descriptor as _descriptor from google.protobuf import message as _message -from typing import ( - ClassVar as _ClassVar, - Iterable as _Iterable, - Mapping as _Mapping, - Optional as _Optional, - Union as _Union, -) +from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union DESCRIPTOR: _descriptor.FileDescriptor @@ -19,17 +13,14 @@ class Status(int, metaclass=_enum_type_wrapper.EnumTypeWrapper): SUCCESS: _ClassVar[Status] FAILURE: _ClassVar[Status] FALLBACK: _ClassVar[Status] - SUCCESS: Status FAILURE: Status FALLBACK: Status class SinkRequest(_message.Message): __slots__ = ("request", "status", "handshake") - class Request(_message.Message): __slots__ = ("keys", "value", "event_time", "watermark", "id", "headers") - class HeadersEntry(_message.Message): __slots__ = ("key", "value") KEY_FIELD_NUMBER: _ClassVar[int] @@ -37,7 +28,6 @@ class SinkRequest(_message.Message): key: str value: str def __init__(self, key: _Optional[str] = ..., value: _Optional[str] = ...) -> None: ... - KEYS_FIELD_NUMBER: _ClassVar[int] VALUE_FIELD_NUMBER: _ClassVar[int] EVENT_TIME_FIELD_NUMBER: _ClassVar[int] @@ -50,28 +40,14 @@ class SinkRequest(_message.Message): watermark: _timestamp_pb2.Timestamp id: str headers: _containers.ScalarMap[str, str] - def __init__( - self, - keys: _Optional[_Iterable[str]] = ..., - value: _Optional[bytes] = ..., - event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - id: _Optional[str] = ..., - headers: _Optional[_Mapping[str, str]] = ..., - ) -> None: ... - + def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., id: _Optional[str] = ..., headers: _Optional[_Mapping[str, str]] = ...) -> None: ... REQUEST_FIELD_NUMBER: _ClassVar[int] STATUS_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] request: SinkRequest.Request status: TransmissionStatus handshake: Handshake - def __init__( - self, - request: _Optional[_Union[SinkRequest.Request, _Mapping]] = ..., - status: _Optional[_Union[TransmissionStatus, _Mapping]] = ..., - handshake: _Optional[_Union[Handshake, _Mapping]] = ..., - ) -> None: ... + def __init__(self, request: _Optional[_Union[SinkRequest.Request, _Mapping]] = ..., status: _Optional[_Union[TransmissionStatus, _Mapping]] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ...) -> None: ... class Handshake(_message.Message): __slots__ = ("sot",) @@ -93,7 +69,6 @@ class TransmissionStatus(_message.Message): class SinkResponse(_message.Message): __slots__ = ("results", "handshake", "status") - class Result(_message.Message): __slots__ = ("id", "status", "err_msg") ID_FIELD_NUMBER: _ClassVar[int] @@ -102,22 +77,11 @@ class SinkResponse(_message.Message): id: str status: Status err_msg: str - def __init__( - self, - id: _Optional[str] = ..., - status: _Optional[_Union[Status, str]] = ..., - err_msg: _Optional[str] = ..., - ) -> None: ... - + def __init__(self, id: _Optional[str] = ..., status: _Optional[_Union[Status, str]] = ..., err_msg: _Optional[str] = ...) -> None: ... RESULTS_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] STATUS_FIELD_NUMBER: _ClassVar[int] results: _containers.RepeatedCompositeFieldContainer[SinkResponse.Result] handshake: Handshake status: TransmissionStatus - def __init__( - self, - results: _Optional[_Iterable[_Union[SinkResponse.Result, _Mapping]]] = ..., - handshake: _Optional[_Union[Handshake, _Mapping]] = ..., - status: _Optional[_Union[TransmissionStatus, _Mapping]] = ..., - ) -> None: ... + def __init__(self, results: _Optional[_Iterable[_Union[SinkResponse.Result, _Mapping]]] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ..., status: _Optional[_Union[TransmissionStatus, _Mapping]] = ...) -> None: ... diff --git a/pynumaflow/proto/sinker/sink_pb2_grpc.py b/pynumaflow/proto/sinker/sink_pb2_grpc.py index 9609c76e..4f53a34e 100644 --- a/pynumaflow/proto/sinker/sink_pb2_grpc.py +++ b/pynumaflow/proto/sinker/sink_pb2_grpc.py @@ -16,108 +16,87 @@ def __init__(self, channel): channel: A grpc.Channel. """ self.SinkFn = channel.stream_stream( - "/sink.v1.Sink/SinkFn", - request_serializer=sink__pb2.SinkRequest.SerializeToString, - response_deserializer=sink__pb2.SinkResponse.FromString, - ) + '/sink.v1.Sink/SinkFn', + request_serializer=sink__pb2.SinkRequest.SerializeToString, + response_deserializer=sink__pb2.SinkResponse.FromString, + ) self.IsReady = channel.unary_unary( - "/sink.v1.Sink/IsReady", - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=sink__pb2.ReadyResponse.FromString, - ) + '/sink.v1.Sink/IsReady', + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=sink__pb2.ReadyResponse.FromString, + ) class SinkServicer(object): """Missing associated documentation comment in .proto file.""" def SinkFn(self, request_iterator, context): - """SinkFn writes the request to a user defined sink.""" + """SinkFn writes the request to a user defined sink. + """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def IsReady(self, request, context): - """IsReady is the heartbeat endpoint for gRPC.""" + """IsReady is the heartbeat endpoint for gRPC. + """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def add_SinkServicer_to_server(servicer, server): rpc_method_handlers = { - "SinkFn": grpc.stream_stream_rpc_method_handler( - servicer.SinkFn, - request_deserializer=sink__pb2.SinkRequest.FromString, - response_serializer=sink__pb2.SinkResponse.SerializeToString, - ), - "IsReady": grpc.unary_unary_rpc_method_handler( - servicer.IsReady, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=sink__pb2.ReadyResponse.SerializeToString, - ), + 'SinkFn': grpc.stream_stream_rpc_method_handler( + servicer.SinkFn, + request_deserializer=sink__pb2.SinkRequest.FromString, + response_serializer=sink__pb2.SinkResponse.SerializeToString, + ), + 'IsReady': grpc.unary_unary_rpc_method_handler( + servicer.IsReady, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=sink__pb2.ReadyResponse.SerializeToString, + ), } - generic_handler = grpc.method_handlers_generic_handler("sink.v1.Sink", rpc_method_handlers) + generic_handler = grpc.method_handlers_generic_handler( + 'sink.v1.Sink', rpc_method_handlers) server.add_generic_rpc_handlers((generic_handler,)) -# This class is part of an EXPERIMENTAL API. + # This class is part of an EXPERIMENTAL API. class Sink(object): """Missing associated documentation comment in .proto file.""" @staticmethod - def SinkFn( - request_iterator, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.stream_stream( - request_iterator, + def SinkFn(request_iterator, target, - "/sink.v1.Sink/SinkFn", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.stream_stream(request_iterator, target, '/sink.v1.Sink/SinkFn', sink__pb2.SinkRequest.SerializeToString, sink__pb2.SinkResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) @staticmethod - def IsReady( - request, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.unary_unary( - request, + def IsReady(request, target, - "/sink.v1.Sink/IsReady", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.unary_unary(request, target, '/sink.v1.Sink/IsReady', google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, sink__pb2.ReadyResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) diff --git a/pynumaflow/proto/sourcer/source_pb2.py b/pynumaflow/proto/sourcer/source_pb2.py index 6cd6f0b2..e8d62515 100644 --- a/pynumaflow/proto/sourcer/source_pb2.py +++ b/pynumaflow/proto/sourcer/source_pb2.py @@ -7,7 +7,6 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import symbol_database as _symbol_database from google.protobuf.internal import builder as _builder - # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() @@ -17,55 +16,53 @@ from google.protobuf import empty_pb2 as google_dot_protobuf_dot_empty__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x0csource.proto\x12\tsource.v1\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1bgoogle/protobuf/empty.proto"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08"\xb1\x01\n\x0bReadRequest\x12/\n\x07request\x18\x01 \x01(\x0b\x32\x1e.source.v1.ReadRequest.Request\x12,\n\thandshake\x18\x02 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a\x35\n\x07Request\x12\x13\n\x0bnum_records\x18\x01 \x01(\x04\x12\x15\n\rtimeout_in_ms\x18\x02 \x01(\rB\x0c\n\n_handshake"\x81\x05\n\x0cReadResponse\x12.\n\x06result\x18\x01 \x01(\x0b\x32\x1e.source.v1.ReadResponse.Result\x12.\n\x06status\x18\x02 \x01(\x0b\x32\x1e.source.v1.ReadResponse.Status\x12,\n\thandshake\x18\x03 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a\xe8\x01\n\x06Result\x12\x0f\n\x07payload\x18\x01 \x01(\x0c\x12!\n\x06offset\x18\x02 \x01(\x0b\x32\x11.source.v1.Offset\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04keys\x18\x04 \x03(\t\x12<\n\x07headers\x18\x05 \x03(\x0b\x32+.source.v1.ReadResponse.Result.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x1a\xe9\x01\n\x06Status\x12\x0b\n\x03\x65ot\x18\x01 \x01(\x08\x12\x31\n\x04\x63ode\x18\x02 \x01(\x0e\x32#.source.v1.ReadResponse.Status.Code\x12\x38\n\x05\x65rror\x18\x03 \x01(\x0e\x32$.source.v1.ReadResponse.Status.ErrorH\x00\x88\x01\x01\x12\x10\n\x03msg\x18\x04 \x01(\tH\x01\x88\x01\x01" \n\x04\x43ode\x12\x0b\n\x07SUCCESS\x10\x00\x12\x0b\n\x07\x46\x41ILURE\x10\x01"\x1f\n\x05\x45rror\x12\x0b\n\x07UNACKED\x10\x00\x12\t\n\x05OTHER\x10\x01\x42\x08\n\x06_errorB\x06\n\x04_msgB\x0c\n\n_handshake"\xa7\x01\n\nAckRequest\x12.\n\x07request\x18\x01 \x01(\x0b\x32\x1d.source.v1.AckRequest.Request\x12,\n\thandshake\x18\x02 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a-\n\x07Request\x12"\n\x07offsets\x18\x01 \x03(\x0b\x32\x11.source.v1.OffsetB\x0c\n\n_handshake"\xab\x01\n\x0b\x41\x63kResponse\x12-\n\x06result\x18\x01 \x01(\x0b\x32\x1d.source.v1.AckResponse.Result\x12,\n\thandshake\x18\x02 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a\x31\n\x06Result\x12\'\n\x07success\x18\x01 \x01(\x0b\x32\x16.google.protobuf.EmptyB\x0c\n\n_handshake"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08"]\n\x0fPendingResponse\x12\x31\n\x06result\x18\x01 \x01(\x0b\x32!.source.v1.PendingResponse.Result\x1a\x17\n\x06Result\x12\r\n\x05\x63ount\x18\x01 \x01(\x03"h\n\x12PartitionsResponse\x12\x34\n\x06result\x18\x01 \x01(\x0b\x32$.source.v1.PartitionsResponse.Result\x1a\x1c\n\x06Result\x12\x12\n\npartitions\x18\x01 \x03(\x05".\n\x06Offset\x12\x0e\n\x06offset\x18\x01 \x01(\x0c\x12\x14\n\x0cpartition_id\x18\x02 \x01(\x05\x32\xc8\x02\n\x06Source\x12=\n\x06ReadFn\x12\x16.source.v1.ReadRequest\x1a\x17.source.v1.ReadResponse(\x01\x30\x01\x12:\n\x05\x41\x63kFn\x12\x15.source.v1.AckRequest\x1a\x16.source.v1.AckResponse(\x01\x30\x01\x12?\n\tPendingFn\x12\x16.google.protobuf.Empty\x1a\x1a.source.v1.PendingResponse\x12\x45\n\x0cPartitionsFn\x12\x16.google.protobuf.Empty\x1a\x1d.source.v1.PartitionsResponse\x12;\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x18.source.v1.ReadyResponseb\x06proto3' -) +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x0csource.proto\x12\tsource.v1\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1bgoogle/protobuf/empty.proto\"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08\"\xb1\x01\n\x0bReadRequest\x12/\n\x07request\x18\x01 \x01(\x0b\x32\x1e.source.v1.ReadRequest.Request\x12,\n\thandshake\x18\x02 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a\x35\n\x07Request\x12\x13\n\x0bnum_records\x18\x01 \x01(\x04\x12\x15\n\rtimeout_in_ms\x18\x02 \x01(\rB\x0c\n\n_handshake\"\x81\x05\n\x0cReadResponse\x12.\n\x06result\x18\x01 \x01(\x0b\x32\x1e.source.v1.ReadResponse.Result\x12.\n\x06status\x18\x02 \x01(\x0b\x32\x1e.source.v1.ReadResponse.Status\x12,\n\thandshake\x18\x03 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a\xe8\x01\n\x06Result\x12\x0f\n\x07payload\x18\x01 \x01(\x0c\x12!\n\x06offset\x18\x02 \x01(\x0b\x32\x11.source.v1.Offset\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04keys\x18\x04 \x03(\t\x12<\n\x07headers\x18\x05 \x03(\x0b\x32+.source.v1.ReadResponse.Result.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x1a\xe9\x01\n\x06Status\x12\x0b\n\x03\x65ot\x18\x01 \x01(\x08\x12\x31\n\x04\x63ode\x18\x02 \x01(\x0e\x32#.source.v1.ReadResponse.Status.Code\x12\x38\n\x05\x65rror\x18\x03 \x01(\x0e\x32$.source.v1.ReadResponse.Status.ErrorH\x00\x88\x01\x01\x12\x10\n\x03msg\x18\x04 \x01(\tH\x01\x88\x01\x01\" \n\x04\x43ode\x12\x0b\n\x07SUCCESS\x10\x00\x12\x0b\n\x07\x46\x41ILURE\x10\x01\"\x1f\n\x05\x45rror\x12\x0b\n\x07UNACKED\x10\x00\x12\t\n\x05OTHER\x10\x01\x42\x08\n\x06_errorB\x06\n\x04_msgB\x0c\n\n_handshake\"\xa7\x01\n\nAckRequest\x12.\n\x07request\x18\x01 \x01(\x0b\x32\x1d.source.v1.AckRequest.Request\x12,\n\thandshake\x18\x02 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a-\n\x07Request\x12\"\n\x07offsets\x18\x01 \x03(\x0b\x32\x11.source.v1.OffsetB\x0c\n\n_handshake\"\xab\x01\n\x0b\x41\x63kResponse\x12-\n\x06result\x18\x01 \x01(\x0b\x32\x1d.source.v1.AckResponse.Result\x12,\n\thandshake\x18\x02 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a\x31\n\x06Result\x12\'\n\x07success\x18\x01 \x01(\x0b\x32\x16.google.protobuf.EmptyB\x0c\n\n_handshake\"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\"]\n\x0fPendingResponse\x12\x31\n\x06result\x18\x01 \x01(\x0b\x32!.source.v1.PendingResponse.Result\x1a\x17\n\x06Result\x12\r\n\x05\x63ount\x18\x01 \x01(\x03\"h\n\x12PartitionsResponse\x12\x34\n\x06result\x18\x01 \x01(\x0b\x32$.source.v1.PartitionsResponse.Result\x1a\x1c\n\x06Result\x12\x12\n\npartitions\x18\x01 \x03(\x05\".\n\x06Offset\x12\x0e\n\x06offset\x18\x01 \x01(\x0c\x12\x14\n\x0cpartition_id\x18\x02 \x01(\x05\x32\xc8\x02\n\x06Source\x12=\n\x06ReadFn\x12\x16.source.v1.ReadRequest\x1a\x17.source.v1.ReadResponse(\x01\x30\x01\x12:\n\x05\x41\x63kFn\x12\x15.source.v1.AckRequest\x1a\x16.source.v1.AckResponse(\x01\x30\x01\x12?\n\tPendingFn\x12\x16.google.protobuf.Empty\x1a\x1a.source.v1.PendingResponse\x12\x45\n\x0cPartitionsFn\x12\x16.google.protobuf.Empty\x1a\x1d.source.v1.PartitionsResponse\x12;\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x18.source.v1.ReadyResponseb\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) -_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, "source_pb2", _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'source_pb2', _globals) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None - _globals["_READRESPONSE_RESULT_HEADERSENTRY"]._options = None - _globals["_READRESPONSE_RESULT_HEADERSENTRY"]._serialized_options = b"8\001" - _globals["_HANDSHAKE"]._serialized_start = 89 - _globals["_HANDSHAKE"]._serialized_end = 113 - _globals["_READREQUEST"]._serialized_start = 116 - _globals["_READREQUEST"]._serialized_end = 293 - _globals["_READREQUEST_REQUEST"]._serialized_start = 226 - _globals["_READREQUEST_REQUEST"]._serialized_end = 279 - _globals["_READRESPONSE"]._serialized_start = 296 - _globals["_READRESPONSE"]._serialized_end = 937 - _globals["_READRESPONSE_RESULT"]._serialized_start = 455 - _globals["_READRESPONSE_RESULT"]._serialized_end = 687 - _globals["_READRESPONSE_RESULT_HEADERSENTRY"]._serialized_start = 641 - _globals["_READRESPONSE_RESULT_HEADERSENTRY"]._serialized_end = 687 - _globals["_READRESPONSE_STATUS"]._serialized_start = 690 - _globals["_READRESPONSE_STATUS"]._serialized_end = 923 - _globals["_READRESPONSE_STATUS_CODE"]._serialized_start = 840 - _globals["_READRESPONSE_STATUS_CODE"]._serialized_end = 872 - _globals["_READRESPONSE_STATUS_ERROR"]._serialized_start = 874 - _globals["_READRESPONSE_STATUS_ERROR"]._serialized_end = 905 - _globals["_ACKREQUEST"]._serialized_start = 940 - _globals["_ACKREQUEST"]._serialized_end = 1107 - _globals["_ACKREQUEST_REQUEST"]._serialized_start = 1048 - _globals["_ACKREQUEST_REQUEST"]._serialized_end = 1093 - _globals["_ACKRESPONSE"]._serialized_start = 1110 - _globals["_ACKRESPONSE"]._serialized_end = 1281 - _globals["_ACKRESPONSE_RESULT"]._serialized_start = 1218 - _globals["_ACKRESPONSE_RESULT"]._serialized_end = 1267 - _globals["_READYRESPONSE"]._serialized_start = 1283 - _globals["_READYRESPONSE"]._serialized_end = 1313 - _globals["_PENDINGRESPONSE"]._serialized_start = 1315 - _globals["_PENDINGRESPONSE"]._serialized_end = 1408 - _globals["_PENDINGRESPONSE_RESULT"]._serialized_start = 1385 - _globals["_PENDINGRESPONSE_RESULT"]._serialized_end = 1408 - _globals["_PARTITIONSRESPONSE"]._serialized_start = 1410 - _globals["_PARTITIONSRESPONSE"]._serialized_end = 1514 - _globals["_PARTITIONSRESPONSE_RESULT"]._serialized_start = 1486 - _globals["_PARTITIONSRESPONSE_RESULT"]._serialized_end = 1514 - _globals["_OFFSET"]._serialized_start = 1516 - _globals["_OFFSET"]._serialized_end = 1562 - _globals["_SOURCE"]._serialized_start = 1565 - _globals["_SOURCE"]._serialized_end = 1893 + DESCRIPTOR._options = None + _globals['_READRESPONSE_RESULT_HEADERSENTRY']._options = None + _globals['_READRESPONSE_RESULT_HEADERSENTRY']._serialized_options = b'8\001' + _globals['_HANDSHAKE']._serialized_start=89 + _globals['_HANDSHAKE']._serialized_end=113 + _globals['_READREQUEST']._serialized_start=116 + _globals['_READREQUEST']._serialized_end=293 + _globals['_READREQUEST_REQUEST']._serialized_start=226 + _globals['_READREQUEST_REQUEST']._serialized_end=279 + _globals['_READRESPONSE']._serialized_start=296 + _globals['_READRESPONSE']._serialized_end=937 + _globals['_READRESPONSE_RESULT']._serialized_start=455 + _globals['_READRESPONSE_RESULT']._serialized_end=687 + _globals['_READRESPONSE_RESULT_HEADERSENTRY']._serialized_start=641 + _globals['_READRESPONSE_RESULT_HEADERSENTRY']._serialized_end=687 + _globals['_READRESPONSE_STATUS']._serialized_start=690 + _globals['_READRESPONSE_STATUS']._serialized_end=923 + _globals['_READRESPONSE_STATUS_CODE']._serialized_start=840 + _globals['_READRESPONSE_STATUS_CODE']._serialized_end=872 + _globals['_READRESPONSE_STATUS_ERROR']._serialized_start=874 + _globals['_READRESPONSE_STATUS_ERROR']._serialized_end=905 + _globals['_ACKREQUEST']._serialized_start=940 + _globals['_ACKREQUEST']._serialized_end=1107 + _globals['_ACKREQUEST_REQUEST']._serialized_start=1048 + _globals['_ACKREQUEST_REQUEST']._serialized_end=1093 + _globals['_ACKRESPONSE']._serialized_start=1110 + _globals['_ACKRESPONSE']._serialized_end=1281 + _globals['_ACKRESPONSE_RESULT']._serialized_start=1218 + _globals['_ACKRESPONSE_RESULT']._serialized_end=1267 + _globals['_READYRESPONSE']._serialized_start=1283 + _globals['_READYRESPONSE']._serialized_end=1313 + _globals['_PENDINGRESPONSE']._serialized_start=1315 + _globals['_PENDINGRESPONSE']._serialized_end=1408 + _globals['_PENDINGRESPONSE_RESULT']._serialized_start=1385 + _globals['_PENDINGRESPONSE_RESULT']._serialized_end=1408 + _globals['_PARTITIONSRESPONSE']._serialized_start=1410 + _globals['_PARTITIONSRESPONSE']._serialized_end=1514 + _globals['_PARTITIONSRESPONSE_RESULT']._serialized_start=1486 + _globals['_PARTITIONSRESPONSE_RESULT']._serialized_end=1514 + _globals['_OFFSET']._serialized_start=1516 + _globals['_OFFSET']._serialized_end=1562 + _globals['_SOURCE']._serialized_start=1565 + _globals['_SOURCE']._serialized_end=1893 # @@protoc_insertion_point(module_scope) diff --git a/pynumaflow/proto/sourcer/source_pb2.pyi b/pynumaflow/proto/sourcer/source_pb2.pyi index 8f588410..52c20928 100644 --- a/pynumaflow/proto/sourcer/source_pb2.pyi +++ b/pynumaflow/proto/sourcer/source_pb2.pyi @@ -4,13 +4,7 @@ from google.protobuf.internal import containers as _containers from google.protobuf.internal import enum_type_wrapper as _enum_type_wrapper from google.protobuf import descriptor as _descriptor from google.protobuf import message as _message -from typing import ( - ClassVar as _ClassVar, - Iterable as _Iterable, - Mapping as _Mapping, - Optional as _Optional, - Union as _Union, -) +from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union DESCRIPTOR: _descriptor.FileDescriptor @@ -22,33 +16,23 @@ class Handshake(_message.Message): class ReadRequest(_message.Message): __slots__ = ("request", "handshake") - class Request(_message.Message): __slots__ = ("num_records", "timeout_in_ms") NUM_RECORDS_FIELD_NUMBER: _ClassVar[int] TIMEOUT_IN_MS_FIELD_NUMBER: _ClassVar[int] num_records: int timeout_in_ms: int - def __init__( - self, num_records: _Optional[int] = ..., timeout_in_ms: _Optional[int] = ... - ) -> None: ... - + def __init__(self, num_records: _Optional[int] = ..., timeout_in_ms: _Optional[int] = ...) -> None: ... REQUEST_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] request: ReadRequest.Request handshake: Handshake - def __init__( - self, - request: _Optional[_Union[ReadRequest.Request, _Mapping]] = ..., - handshake: _Optional[_Union[Handshake, _Mapping]] = ..., - ) -> None: ... + def __init__(self, request: _Optional[_Union[ReadRequest.Request, _Mapping]] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ...) -> None: ... class ReadResponse(_message.Message): __slots__ = ("result", "status", "handshake") - class Result(_message.Message): __slots__ = ("payload", "offset", "event_time", "keys", "headers") - class HeadersEntry(_message.Message): __slots__ = ("key", "value") KEY_FIELD_NUMBER: _ClassVar[int] @@ -56,7 +40,6 @@ class ReadResponse(_message.Message): key: str value: str def __init__(self, key: _Optional[str] = ..., value: _Optional[str] = ...) -> None: ... - PAYLOAD_FIELD_NUMBER: _ClassVar[int] OFFSET_FIELD_NUMBER: _ClassVar[int] EVENT_TIME_FIELD_NUMBER: _ClassVar[int] @@ -67,25 +50,15 @@ class ReadResponse(_message.Message): event_time: _timestamp_pb2.Timestamp keys: _containers.RepeatedScalarFieldContainer[str] headers: _containers.ScalarMap[str, str] - def __init__( - self, - payload: _Optional[bytes] = ..., - offset: _Optional[_Union[Offset, _Mapping]] = ..., - event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - keys: _Optional[_Iterable[str]] = ..., - headers: _Optional[_Mapping[str, str]] = ..., - ) -> None: ... - + def __init__(self, payload: _Optional[bytes] = ..., offset: _Optional[_Union[Offset, _Mapping]] = ..., event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., keys: _Optional[_Iterable[str]] = ..., headers: _Optional[_Mapping[str, str]] = ...) -> None: ... class Status(_message.Message): __slots__ = ("eot", "code", "error", "msg") - class Code(int, metaclass=_enum_type_wrapper.EnumTypeWrapper): __slots__ = () SUCCESS: _ClassVar[ReadResponse.Status.Code] FAILURE: _ClassVar[ReadResponse.Status.Code] SUCCESS: ReadResponse.Status.Code FAILURE: ReadResponse.Status.Code - class Error(int, metaclass=_enum_type_wrapper.EnumTypeWrapper): __slots__ = () UNACKED: _ClassVar[ReadResponse.Status.Error] @@ -100,68 +73,40 @@ class ReadResponse(_message.Message): code: ReadResponse.Status.Code error: ReadResponse.Status.Error msg: str - def __init__( - self, - eot: bool = ..., - code: _Optional[_Union[ReadResponse.Status.Code, str]] = ..., - error: _Optional[_Union[ReadResponse.Status.Error, str]] = ..., - msg: _Optional[str] = ..., - ) -> None: ... - + def __init__(self, eot: bool = ..., code: _Optional[_Union[ReadResponse.Status.Code, str]] = ..., error: _Optional[_Union[ReadResponse.Status.Error, str]] = ..., msg: _Optional[str] = ...) -> None: ... RESULT_FIELD_NUMBER: _ClassVar[int] STATUS_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] result: ReadResponse.Result status: ReadResponse.Status handshake: Handshake - def __init__( - self, - result: _Optional[_Union[ReadResponse.Result, _Mapping]] = ..., - status: _Optional[_Union[ReadResponse.Status, _Mapping]] = ..., - handshake: _Optional[_Union[Handshake, _Mapping]] = ..., - ) -> None: ... + def __init__(self, result: _Optional[_Union[ReadResponse.Result, _Mapping]] = ..., status: _Optional[_Union[ReadResponse.Status, _Mapping]] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ...) -> None: ... class AckRequest(_message.Message): __slots__ = ("request", "handshake") - class Request(_message.Message): __slots__ = ("offsets",) OFFSETS_FIELD_NUMBER: _ClassVar[int] offsets: _containers.RepeatedCompositeFieldContainer[Offset] - def __init__( - self, offsets: _Optional[_Iterable[_Union[Offset, _Mapping]]] = ... - ) -> None: ... - + def __init__(self, offsets: _Optional[_Iterable[_Union[Offset, _Mapping]]] = ...) -> None: ... REQUEST_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] request: AckRequest.Request handshake: Handshake - def __init__( - self, - request: _Optional[_Union[AckRequest.Request, _Mapping]] = ..., - handshake: _Optional[_Union[Handshake, _Mapping]] = ..., - ) -> None: ... + def __init__(self, request: _Optional[_Union[AckRequest.Request, _Mapping]] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ...) -> None: ... class AckResponse(_message.Message): __slots__ = ("result", "handshake") - class Result(_message.Message): __slots__ = ("success",) SUCCESS_FIELD_NUMBER: _ClassVar[int] success: _empty_pb2.Empty - def __init__( - self, success: _Optional[_Union[_empty_pb2.Empty, _Mapping]] = ... - ) -> None: ... - + def __init__(self, success: _Optional[_Union[_empty_pb2.Empty, _Mapping]] = ...) -> None: ... RESULT_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] result: AckResponse.Result handshake: Handshake - def __init__( - self, - result: _Optional[_Union[AckResponse.Result, _Mapping]] = ..., - handshake: _Optional[_Union[Handshake, _Mapping]] = ..., - ) -> None: ... + def __init__(self, result: _Optional[_Union[AckResponse.Result, _Mapping]] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ...) -> None: ... class ReadyResponse(_message.Message): __slots__ = ("ready",) @@ -171,33 +116,25 @@ class ReadyResponse(_message.Message): class PendingResponse(_message.Message): __slots__ = ("result",) - class Result(_message.Message): __slots__ = ("count",) COUNT_FIELD_NUMBER: _ClassVar[int] count: int def __init__(self, count: _Optional[int] = ...) -> None: ... - RESULT_FIELD_NUMBER: _ClassVar[int] result: PendingResponse.Result - def __init__( - self, result: _Optional[_Union[PendingResponse.Result, _Mapping]] = ... - ) -> None: ... + def __init__(self, result: _Optional[_Union[PendingResponse.Result, _Mapping]] = ...) -> None: ... class PartitionsResponse(_message.Message): __slots__ = ("result",) - class Result(_message.Message): __slots__ = ("partitions",) PARTITIONS_FIELD_NUMBER: _ClassVar[int] partitions: _containers.RepeatedScalarFieldContainer[int] def __init__(self, partitions: _Optional[_Iterable[int]] = ...) -> None: ... - RESULT_FIELD_NUMBER: _ClassVar[int] result: PartitionsResponse.Result - def __init__( - self, result: _Optional[_Union[PartitionsResponse.Result, _Mapping]] = ... - ) -> None: ... + def __init__(self, result: _Optional[_Union[PartitionsResponse.Result, _Mapping]] = ...) -> None: ... class Offset(_message.Message): __slots__ = ("offset", "partition_id") @@ -205,6 +142,4 @@ class Offset(_message.Message): PARTITION_ID_FIELD_NUMBER: _ClassVar[int] offset: bytes partition_id: int - def __init__( - self, offset: _Optional[bytes] = ..., partition_id: _Optional[int] = ... - ) -> None: ... + def __init__(self, offset: _Optional[bytes] = ..., partition_id: _Optional[int] = ...) -> None: ... diff --git a/pynumaflow/proto/sourcer/source_pb2_grpc.py b/pynumaflow/proto/sourcer/source_pb2_grpc.py index d5e09f1b..14ad1d5a 100644 --- a/pynumaflow/proto/sourcer/source_pb2_grpc.py +++ b/pynumaflow/proto/sourcer/source_pb2_grpc.py @@ -16,30 +16,30 @@ def __init__(self, channel): channel: A grpc.Channel. """ self.ReadFn = channel.stream_stream( - "/source.v1.Source/ReadFn", - request_serializer=source__pb2.ReadRequest.SerializeToString, - response_deserializer=source__pb2.ReadResponse.FromString, - ) + '/source.v1.Source/ReadFn', + request_serializer=source__pb2.ReadRequest.SerializeToString, + response_deserializer=source__pb2.ReadResponse.FromString, + ) self.AckFn = channel.stream_stream( - "/source.v1.Source/AckFn", - request_serializer=source__pb2.AckRequest.SerializeToString, - response_deserializer=source__pb2.AckResponse.FromString, - ) + '/source.v1.Source/AckFn', + request_serializer=source__pb2.AckRequest.SerializeToString, + response_deserializer=source__pb2.AckResponse.FromString, + ) self.PendingFn = channel.unary_unary( - "/source.v1.Source/PendingFn", - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=source__pb2.PendingResponse.FromString, - ) + '/source.v1.Source/PendingFn', + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=source__pb2.PendingResponse.FromString, + ) self.PartitionsFn = channel.unary_unary( - "/source.v1.Source/PartitionsFn", - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=source__pb2.PartitionsResponse.FromString, - ) + '/source.v1.Source/PartitionsFn', + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=source__pb2.PartitionsResponse.FromString, + ) self.IsReady = channel.unary_unary( - "/source.v1.Source/IsReady", - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=source__pb2.ReadyResponse.FromString, - ) + '/source.v1.Source/IsReady', + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=source__pb2.ReadyResponse.FromString, + ) class SourceServicer(object): @@ -53,8 +53,8 @@ def ReadFn(self, request_iterator, context): Once it has sent all the datum, the server will send a ReadResponse with the end of transmission flag set to true. """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def AckFn(self, request_iterator, context): """AckFn acknowledges a stream of datum offsets. @@ -65,205 +65,149 @@ def AckFn(self, request_iterator, context): Clients sends n requests and expects n responses. """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def PendingFn(self, request, context): - """PendingFn returns the number of pending records at the user defined source.""" + """PendingFn returns the number of pending records at the user defined source. + """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def PartitionsFn(self, request, context): - """PartitionsFn returns the list of partitions for the user defined source.""" + """PartitionsFn returns the list of partitions for the user defined source. + """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def IsReady(self, request, context): - """IsReady is the heartbeat endpoint for user defined source gRPC.""" + """IsReady is the heartbeat endpoint for user defined source gRPC. + """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def add_SourceServicer_to_server(servicer, server): rpc_method_handlers = { - "ReadFn": grpc.stream_stream_rpc_method_handler( - servicer.ReadFn, - request_deserializer=source__pb2.ReadRequest.FromString, - response_serializer=source__pb2.ReadResponse.SerializeToString, - ), - "AckFn": grpc.stream_stream_rpc_method_handler( - servicer.AckFn, - request_deserializer=source__pb2.AckRequest.FromString, - response_serializer=source__pb2.AckResponse.SerializeToString, - ), - "PendingFn": grpc.unary_unary_rpc_method_handler( - servicer.PendingFn, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=source__pb2.PendingResponse.SerializeToString, - ), - "PartitionsFn": grpc.unary_unary_rpc_method_handler( - servicer.PartitionsFn, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=source__pb2.PartitionsResponse.SerializeToString, - ), - "IsReady": grpc.unary_unary_rpc_method_handler( - servicer.IsReady, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=source__pb2.ReadyResponse.SerializeToString, - ), + 'ReadFn': grpc.stream_stream_rpc_method_handler( + servicer.ReadFn, + request_deserializer=source__pb2.ReadRequest.FromString, + response_serializer=source__pb2.ReadResponse.SerializeToString, + ), + 'AckFn': grpc.stream_stream_rpc_method_handler( + servicer.AckFn, + request_deserializer=source__pb2.AckRequest.FromString, + response_serializer=source__pb2.AckResponse.SerializeToString, + ), + 'PendingFn': grpc.unary_unary_rpc_method_handler( + servicer.PendingFn, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=source__pb2.PendingResponse.SerializeToString, + ), + 'PartitionsFn': grpc.unary_unary_rpc_method_handler( + servicer.PartitionsFn, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=source__pb2.PartitionsResponse.SerializeToString, + ), + 'IsReady': grpc.unary_unary_rpc_method_handler( + servicer.IsReady, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=source__pb2.ReadyResponse.SerializeToString, + ), } - generic_handler = grpc.method_handlers_generic_handler("source.v1.Source", rpc_method_handlers) + generic_handler = grpc.method_handlers_generic_handler( + 'source.v1.Source', rpc_method_handlers) server.add_generic_rpc_handlers((generic_handler,)) -# This class is part of an EXPERIMENTAL API. + # This class is part of an EXPERIMENTAL API. class Source(object): """Missing associated documentation comment in .proto file.""" @staticmethod - def ReadFn( - request_iterator, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.stream_stream( - request_iterator, + def ReadFn(request_iterator, target, - "/source.v1.Source/ReadFn", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.stream_stream(request_iterator, target, '/source.v1.Source/ReadFn', source__pb2.ReadRequest.SerializeToString, source__pb2.ReadResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) @staticmethod - def AckFn( - request_iterator, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.stream_stream( - request_iterator, + def AckFn(request_iterator, target, - "/source.v1.Source/AckFn", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.stream_stream(request_iterator, target, '/source.v1.Source/AckFn', source__pb2.AckRequest.SerializeToString, source__pb2.AckResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) @staticmethod - def PendingFn( - request, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.unary_unary( - request, + def PendingFn(request, target, - "/source.v1.Source/PendingFn", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.unary_unary(request, target, '/source.v1.Source/PendingFn', google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, source__pb2.PendingResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) @staticmethod - def PartitionsFn( - request, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.unary_unary( - request, + def PartitionsFn(request, target, - "/source.v1.Source/PartitionsFn", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.unary_unary(request, target, '/source.v1.Source/PartitionsFn', google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, source__pb2.PartitionsResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) @staticmethod - def IsReady( - request, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.unary_unary( - request, + def IsReady(request, target, - "/source.v1.Source/IsReady", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.unary_unary(request, target, '/source.v1.Source/IsReady', google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, source__pb2.ReadyResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) diff --git a/pynumaflow/proto/sourcetransformer/transform_pb2.py b/pynumaflow/proto/sourcetransformer/transform_pb2.py index a7b08491..8309b7ca 100644 --- a/pynumaflow/proto/sourcetransformer/transform_pb2.py +++ b/pynumaflow/proto/sourcetransformer/transform_pb2.py @@ -7,7 +7,6 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import symbol_database as _symbol_database from google.protobuf.internal import builder as _builder - # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() @@ -17,31 +16,29 @@ from google.protobuf import empty_pb2 as google_dot_protobuf_dot_empty__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x0ftransform.proto\x12\x14sourcetransformer.v1\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1bgoogle/protobuf/empty.proto"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08"\xbe\x03\n\x16SourceTransformRequest\x12\x45\n\x07request\x18\x01 \x01(\x0b\x32\x34.sourcetransformer.v1.SourceTransformRequest.Request\x12\x37\n\thandshake\x18\x02 \x01(\x0b\x32\x1f.sourcetransformer.v1.HandshakeH\x00\x88\x01\x01\x1a\x95\x02\n\x07Request\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12R\n\x07headers\x18\x05 \x03(\x0b\x32\x41.sourcetransformer.v1.SourceTransformRequest.Request.HeadersEntry\x12\n\n\x02id\x18\x06 \x01(\t\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x42\x0c\n\n_handshake"\x98\x02\n\x17SourceTransformResponse\x12\x45\n\x07results\x18\x01 \x03(\x0b\x32\x34.sourcetransformer.v1.SourceTransformResponse.Result\x12\n\n\x02id\x18\x02 \x01(\t\x12\x37\n\thandshake\x18\x03 \x01(\x0b\x32\x1f.sourcetransformer.v1.HandshakeH\x00\x88\x01\x01\x1a\x63\n\x06Result\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04tags\x18\x04 \x03(\tB\x0c\n\n_handshake"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\xcf\x01\n\x0fSourceTransform\x12t\n\x11SourceTransformFn\x12,.sourcetransformer.v1.SourceTransformRequest\x1a-.sourcetransformer.v1.SourceTransformResponse(\x01\x30\x01\x12\x46\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a#.sourcetransformer.v1.ReadyResponseb\x06proto3' -) +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x0ftransform.proto\x12\x14sourcetransformer.v1\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1bgoogle/protobuf/empty.proto\"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08\"\xbe\x03\n\x16SourceTransformRequest\x12\x45\n\x07request\x18\x01 \x01(\x0b\x32\x34.sourcetransformer.v1.SourceTransformRequest.Request\x12\x37\n\thandshake\x18\x02 \x01(\x0b\x32\x1f.sourcetransformer.v1.HandshakeH\x00\x88\x01\x01\x1a\x95\x02\n\x07Request\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12R\n\x07headers\x18\x05 \x03(\x0b\x32\x41.sourcetransformer.v1.SourceTransformRequest.Request.HeadersEntry\x12\n\n\x02id\x18\x06 \x01(\t\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x42\x0c\n\n_handshake\"\x98\x02\n\x17SourceTransformResponse\x12\x45\n\x07results\x18\x01 \x03(\x0b\x32\x34.sourcetransformer.v1.SourceTransformResponse.Result\x12\n\n\x02id\x18\x02 \x01(\t\x12\x37\n\thandshake\x18\x03 \x01(\x0b\x32\x1f.sourcetransformer.v1.HandshakeH\x00\x88\x01\x01\x1a\x63\n\x06Result\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04tags\x18\x04 \x03(\tB\x0c\n\n_handshake\"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\xcf\x01\n\x0fSourceTransform\x12t\n\x11SourceTransformFn\x12,.sourcetransformer.v1.SourceTransformRequest\x1a-.sourcetransformer.v1.SourceTransformResponse(\x01\x30\x01\x12\x46\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a#.sourcetransformer.v1.ReadyResponseb\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) -_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, "transform_pb2", _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'transform_pb2', _globals) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None - _globals["_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY"]._options = None - _globals["_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY"]._serialized_options = b"8\001" - _globals["_HANDSHAKE"]._serialized_start = 103 - _globals["_HANDSHAKE"]._serialized_end = 127 - _globals["_SOURCETRANSFORMREQUEST"]._serialized_start = 130 - _globals["_SOURCETRANSFORMREQUEST"]._serialized_end = 576 - _globals["_SOURCETRANSFORMREQUEST_REQUEST"]._serialized_start = 285 - _globals["_SOURCETRANSFORMREQUEST_REQUEST"]._serialized_end = 562 - _globals["_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY"]._serialized_start = 516 - _globals["_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY"]._serialized_end = 562 - _globals["_SOURCETRANSFORMRESPONSE"]._serialized_start = 579 - _globals["_SOURCETRANSFORMRESPONSE"]._serialized_end = 859 - _globals["_SOURCETRANSFORMRESPONSE_RESULT"]._serialized_start = 746 - _globals["_SOURCETRANSFORMRESPONSE_RESULT"]._serialized_end = 845 - _globals["_READYRESPONSE"]._serialized_start = 861 - _globals["_READYRESPONSE"]._serialized_end = 891 - _globals["_SOURCETRANSFORM"]._serialized_start = 894 - _globals["_SOURCETRANSFORM"]._serialized_end = 1101 + DESCRIPTOR._options = None + _globals['_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY']._options = None + _globals['_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY']._serialized_options = b'8\001' + _globals['_HANDSHAKE']._serialized_start=103 + _globals['_HANDSHAKE']._serialized_end=127 + _globals['_SOURCETRANSFORMREQUEST']._serialized_start=130 + _globals['_SOURCETRANSFORMREQUEST']._serialized_end=576 + _globals['_SOURCETRANSFORMREQUEST_REQUEST']._serialized_start=285 + _globals['_SOURCETRANSFORMREQUEST_REQUEST']._serialized_end=562 + _globals['_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY']._serialized_start=516 + _globals['_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY']._serialized_end=562 + _globals['_SOURCETRANSFORMRESPONSE']._serialized_start=579 + _globals['_SOURCETRANSFORMRESPONSE']._serialized_end=859 + _globals['_SOURCETRANSFORMRESPONSE_RESULT']._serialized_start=746 + _globals['_SOURCETRANSFORMRESPONSE_RESULT']._serialized_end=845 + _globals['_READYRESPONSE']._serialized_start=861 + _globals['_READYRESPONSE']._serialized_end=891 + _globals['_SOURCETRANSFORM']._serialized_start=894 + _globals['_SOURCETRANSFORM']._serialized_end=1101 # @@protoc_insertion_point(module_scope) diff --git a/pynumaflow/proto/sourcetransformer/transform_pb2.pyi b/pynumaflow/proto/sourcetransformer/transform_pb2.pyi index 1fe8cb08..c7c15079 100644 --- a/pynumaflow/proto/sourcetransformer/transform_pb2.pyi +++ b/pynumaflow/proto/sourcetransformer/transform_pb2.pyi @@ -3,13 +3,7 @@ from google.protobuf import empty_pb2 as _empty_pb2 from google.protobuf.internal import containers as _containers from google.protobuf import descriptor as _descriptor from google.protobuf import message as _message -from typing import ( - ClassVar as _ClassVar, - Iterable as _Iterable, - Mapping as _Mapping, - Optional as _Optional, - Union as _Union, -) +from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union DESCRIPTOR: _descriptor.FileDescriptor @@ -21,10 +15,8 @@ class Handshake(_message.Message): class SourceTransformRequest(_message.Message): __slots__ = ("request", "handshake") - class Request(_message.Message): __slots__ = ("keys", "value", "event_time", "watermark", "headers", "id") - class HeadersEntry(_message.Message): __slots__ = ("key", "value") KEY_FIELD_NUMBER: _ClassVar[int] @@ -32,7 +24,6 @@ class SourceTransformRequest(_message.Message): key: str value: str def __init__(self, key: _Optional[str] = ..., value: _Optional[str] = ...) -> None: ... - KEYS_FIELD_NUMBER: _ClassVar[int] VALUE_FIELD_NUMBER: _ClassVar[int] EVENT_TIME_FIELD_NUMBER: _ClassVar[int] @@ -45,29 +36,15 @@ class SourceTransformRequest(_message.Message): watermark: _timestamp_pb2.Timestamp headers: _containers.ScalarMap[str, str] id: str - def __init__( - self, - keys: _Optional[_Iterable[str]] = ..., - value: _Optional[bytes] = ..., - event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - headers: _Optional[_Mapping[str, str]] = ..., - id: _Optional[str] = ..., - ) -> None: ... - + def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., headers: _Optional[_Mapping[str, str]] = ..., id: _Optional[str] = ...) -> None: ... REQUEST_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] request: SourceTransformRequest.Request handshake: Handshake - def __init__( - self, - request: _Optional[_Union[SourceTransformRequest.Request, _Mapping]] = ..., - handshake: _Optional[_Union[Handshake, _Mapping]] = ..., - ) -> None: ... + def __init__(self, request: _Optional[_Union[SourceTransformRequest.Request, _Mapping]] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ...) -> None: ... class SourceTransformResponse(_message.Message): __slots__ = ("results", "id", "handshake") - class Result(_message.Message): __slots__ = ("keys", "value", "event_time", "tags") KEYS_FIELD_NUMBER: _ClassVar[int] @@ -78,26 +55,14 @@ class SourceTransformResponse(_message.Message): value: bytes event_time: _timestamp_pb2.Timestamp tags: _containers.RepeatedScalarFieldContainer[str] - def __init__( - self, - keys: _Optional[_Iterable[str]] = ..., - value: _Optional[bytes] = ..., - event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - tags: _Optional[_Iterable[str]] = ..., - ) -> None: ... - + def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., tags: _Optional[_Iterable[str]] = ...) -> None: ... RESULTS_FIELD_NUMBER: _ClassVar[int] ID_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] results: _containers.RepeatedCompositeFieldContainer[SourceTransformResponse.Result] id: str handshake: Handshake - def __init__( - self, - results: _Optional[_Iterable[_Union[SourceTransformResponse.Result, _Mapping]]] = ..., - id: _Optional[str] = ..., - handshake: _Optional[_Union[Handshake, _Mapping]] = ..., - ) -> None: ... + def __init__(self, results: _Optional[_Iterable[_Union[SourceTransformResponse.Result, _Mapping]]] = ..., id: _Optional[str] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ...) -> None: ... class ReadyResponse(_message.Message): __slots__ = ("ready",) diff --git a/pynumaflow/proto/sourcetransformer/transform_pb2_grpc.py b/pynumaflow/proto/sourcetransformer/transform_pb2_grpc.py index c6820d39..1a1ea499 100644 --- a/pynumaflow/proto/sourcetransformer/transform_pb2_grpc.py +++ b/pynumaflow/proto/sourcetransformer/transform_pb2_grpc.py @@ -16,15 +16,15 @@ def __init__(self, channel): channel: A grpc.Channel. """ self.SourceTransformFn = channel.stream_stream( - "/sourcetransformer.v1.SourceTransform/SourceTransformFn", - request_serializer=transform__pb2.SourceTransformRequest.SerializeToString, - response_deserializer=transform__pb2.SourceTransformResponse.FromString, - ) + '/sourcetransformer.v1.SourceTransform/SourceTransformFn', + request_serializer=transform__pb2.SourceTransformRequest.SerializeToString, + response_deserializer=transform__pb2.SourceTransformResponse.FromString, + ) self.IsReady = channel.unary_unary( - "/sourcetransformer.v1.SourceTransform/IsReady", - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=transform__pb2.ReadyResponse.FromString, - ) + '/sourcetransformer.v1.SourceTransform/IsReady', + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=transform__pb2.ReadyResponse.FromString, + ) class SourceTransformServicer(object): @@ -36,93 +36,69 @@ def SourceTransformFn(self, request_iterator, context): SourceTransformFn can be used only at source vertex by source data transformer. """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def IsReady(self, request, context): - """IsReady is the heartbeat endpoint for gRPC.""" + """IsReady is the heartbeat endpoint for gRPC. + """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def add_SourceTransformServicer_to_server(servicer, server): rpc_method_handlers = { - "SourceTransformFn": grpc.stream_stream_rpc_method_handler( - servicer.SourceTransformFn, - request_deserializer=transform__pb2.SourceTransformRequest.FromString, - response_serializer=transform__pb2.SourceTransformResponse.SerializeToString, - ), - "IsReady": grpc.unary_unary_rpc_method_handler( - servicer.IsReady, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=transform__pb2.ReadyResponse.SerializeToString, - ), + 'SourceTransformFn': grpc.stream_stream_rpc_method_handler( + servicer.SourceTransformFn, + request_deserializer=transform__pb2.SourceTransformRequest.FromString, + response_serializer=transform__pb2.SourceTransformResponse.SerializeToString, + ), + 'IsReady': grpc.unary_unary_rpc_method_handler( + servicer.IsReady, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=transform__pb2.ReadyResponse.SerializeToString, + ), } generic_handler = grpc.method_handlers_generic_handler( - "sourcetransformer.v1.SourceTransform", rpc_method_handlers - ) + 'sourcetransformer.v1.SourceTransform', rpc_method_handlers) server.add_generic_rpc_handlers((generic_handler,)) -# This class is part of an EXPERIMENTAL API. + # This class is part of an EXPERIMENTAL API. class SourceTransform(object): """Missing associated documentation comment in .proto file.""" @staticmethod - def SourceTransformFn( - request_iterator, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.stream_stream( - request_iterator, + def SourceTransformFn(request_iterator, target, - "/sourcetransformer.v1.SourceTransform/SourceTransformFn", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.stream_stream(request_iterator, target, '/sourcetransformer.v1.SourceTransform/SourceTransformFn', transform__pb2.SourceTransformRequest.SerializeToString, transform__pb2.SourceTransformResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) @staticmethod - def IsReady( - request, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.unary_unary( - request, + def IsReady(request, target, - "/sourcetransformer.v1.SourceTransform/IsReady", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.unary_unary(request, target, '/sourcetransformer.v1.SourceTransform/IsReady', google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, transform__pb2.ReadyResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) From 5774e806f1431b68bfd748e0b08d8f4a55838d9b Mon Sep 17 00:00:00 2001 From: srao12 Date: Sun, 20 Jul 2025 21:03:48 -0700 Subject: [PATCH 19/30] fix: update docstring Signed-off-by: srao12 --- pynumaflow/accumulator/_dtypes.py | 216 ++++++++++--- .../proto/accumulator/accumulator_pb2.py | 53 +-- .../proto/accumulator/accumulator_pb2.pyi | 49 ++- .../proto/accumulator/accumulator_pb2_grpc.py | 127 +++++--- pynumaflow/proto/mapper/map_pb2.py | 53 +-- pynumaflow/proto/mapper/map_pb2.pyi | 43 ++- pynumaflow/proto/mapper/map_pb2_grpc.py | 127 +++++--- pynumaflow/proto/reducer/reduce_pb2.py | 53 +-- pynumaflow/proto/reducer/reduce_pb2.pyi | 55 +++- pynumaflow/proto/reducer/reduce_pb2_grpc.py | 127 +++++--- pynumaflow/proto/sideinput/sideinput_pb2.py | 21 +- .../proto/sideinput/sideinput_pb2_grpc.py | 127 +++++--- pynumaflow/proto/sinker/sink_pb2.py | 53 +-- pynumaflow/proto/sinker/sink_pb2.pyi | 43 ++- pynumaflow/proto/sinker/sink_pb2_grpc.py | 127 +++++--- pynumaflow/proto/sourcer/source_pb2.py | 93 +++--- pynumaflow/proto/sourcer/source_pb2.pyi | 84 ++++- pynumaflow/proto/sourcer/source_pb2_grpc.py | 304 +++++++++++------- .../proto/sourcetransformer/transform_pb2.py | 45 +-- .../proto/sourcetransformer/transform_pb2.pyi | 42 ++- .../sourcetransformer/transform_pb2_grpc.py | 124 ++++--- 21 files changed, 1276 insertions(+), 690 deletions(-) diff --git a/pynumaflow/accumulator/_dtypes.py b/pynumaflow/accumulator/_dtypes.py index 7b0d87cb..8a8922d6 100644 --- a/pynumaflow/accumulator/_dtypes.py +++ b/pynumaflow/accumulator/_dtypes.py @@ -77,32 +77,56 @@ def __init__( self._id = id_ def keys(self) -> list[str]: - """Returns the keys of the event""" + """Returns the keys of the event. + + Returns: + list[str]: A list of string keys associated with this event. + """ return self._keys @property def value(self) -> bytes: - """Returns the value of the event.""" + """Returns the value of the event. + + Returns: + bytes: The payload data of the event as bytes. + """ return self._value @property def event_time(self) -> datetime: - """Returns the event time of the event.""" + """Returns the event time of the event. + + Returns: + datetime: The timestamp when the event occurred. + """ return self._event_time @property def watermark(self) -> datetime: - """Returns the watermark of the event.""" + """Returns the watermark of the event. + + Returns: + datetime: The watermark timestamp indicating the progress of event time. + """ return self._watermark @property def headers(self) -> dict[str, str]: - """Returns the headers of the event.""" + """Returns the headers of the event. + + Returns: + dict[str, str]: A dictionary containing header key-value pairs for this event. + """ return self._headers @property def id(self) -> str: - """Returns the id of the event.""" + """Returns the id of the event. + + Returns: + str: The unique identifier for this event. + """ return self._id @@ -120,13 +144,21 @@ def __init__(self, start: datetime, end: datetime): self._end = end @property - def start(self): - """Returns the start point of the interval window.""" + def start(self) -> datetime: + """Returns the start point of the interval window. + + Returns: + datetime: The start timestamp of the interval window. + """ return self._start @property - def end(self): - """Returns the end point of the interval window.""" + def end(self) -> datetime: + """Returns the end point of the interval window. + + Returns: + datetime: The end timestamp of the interval window. + """ return self._end @@ -149,28 +181,48 @@ def __init__(self, start: datetime, end: datetime, slot: str = "", keys: list[st self._keys = keys @property - def start(self): - """Returns the start point of the interval window.""" + def start(self) -> datetime: + """Returns the start point of the interval window. + + Returns: + datetime: The start timestamp of the interval window. + """ return self._window.start @property - def end(self): - """Returns the end point of the interval window.""" + def end(self) -> datetime: + """Returns the end point of the interval window. + + Returns: + datetime: The end timestamp of the interval window. + """ return self._window.end @property - def slot(self): - """Returns the slot from the window""" + def slot(self) -> str: + """Returns the slot from the window. + + Returns: + str: The slot identifier for this window. + """ return self._slot @property - def window(self): - """Return the interval window""" + def window(self) -> IntervalWindow: + """Returns the interval window. + + Returns: + IntervalWindow: The underlying interval window object. + """ return self._window @property - def keys(self): - """Return the keys for window""" + def keys(self) -> list[str]: + """Returns the keys for window. + + Returns: + list[str]: A list of keys associated with this window. + """ return self._keys @@ -186,8 +238,12 @@ def __init__(self, interval_window: IntervalWindow): self._interval_window = interval_window @property - def interval_window(self): - """Returns the interval window for the event.""" + def interval_window(self) -> IntervalWindow: + """Returns the interval window for the event. + + Returns: + IntervalWindow: The interval window associated with this event. + """ return self._interval_window @@ -212,37 +268,68 @@ class AccumulatorResult: _latest_watermark: datetime @property - def future(self): - """Returns the future result of computation.""" + def future(self) -> Task: + """Returns the future result of computation. + + Returns: + Task: The asyncio Task representing the computation future. + """ return self._future @property - def iterator(self): - """Returns the handle to the producer queue.""" + def iterator(self) -> NonBlockingIterator: + """Returns the handle to the producer queue. + + Returns: + NonBlockingIterator: The iterator for producing data to the queue. + """ return self._iterator @property def keys(self) -> list[str]: - """Returns the keys of the partition.""" + """Returns the keys of the partition. + + Returns: + list[str]: The keys associated with this partition. + """ return self._key @property - def result_queue(self): - """Returns the async queue used to write the output for the tasks""" + def result_queue(self) -> NonBlockingIterator: + """Returns the async queue used to write the output for the tasks. + + Returns: + NonBlockingIterator: The queue for writing task output. + """ return self._result_queue @property - def consumer_future(self): - """Returns the async consumer task for the result queue""" + def consumer_future(self) -> Task: + """Returns the async consumer task for the result queue. + + Returns: + Task: The asyncio Task for consuming from the result queue. + """ return self._consumer_future @property - def latest_watermark(self): - """Returns the latest watermark for task""" + def latest_watermark(self) -> datetime: + """Returns the latest watermark for task. + + Returns: + datetime: The latest watermark timestamp for this task. + """ return self._latest_watermark def update_watermark(self, new_watermark: datetime): - """Updates the latest watermark value.""" + """Updates the latest watermark value. + + Args: + new_watermark (datetime): The new watermark timestamp to set. + + Raises: + TypeError: If new_watermark is not a datetime object. + """ if not isinstance(new_watermark, datetime): raise TypeError("new_watermark must be a datetime object") self._latest_watermark = new_watermark @@ -265,17 +352,29 @@ def __init__(self, operation: WindowOperation, keyed_window: KeyedWindow, payloa @property def operation(self) -> WindowOperation: - """Returns the operation type.""" + """Returns the operation type. + + Returns: + WindowOperation: The type of window operation (OPEN, CLOSE, or APPEND). + """ return self._operation @property def keyed_window(self) -> KeyedWindow: - """Returns the keyed window.""" + """Returns the keyed window. + + Returns: + KeyedWindow: The keyed window associated with this request. + """ return self._keyed_window @property def payload(self) -> Datum: - """Returns the payload of the window.""" + """Returns the payload of the window. + + Returns: + Datum: The data payload for this accumulator request. + """ return self._payload @@ -324,39 +423,78 @@ def __init__( self._event_time = event_time self._headers = headers or {} self._id = id or "" - # self._window = window or None - # returns the Message Object which will be dropped @classmethod def to_drop(cls: type[M]) -> M: + """Creates a Message instance that indicates the message should be dropped. + + Returns: + M: A Message instance with empty value and DROP tag indicating + the message should be dropped. + """ return cls(b"", None, [DROP]) @property def value(self) -> bytes: + """Returns the message payload value. + + Returns: + bytes: The message payload data as bytes. + """ return self._value @property def keys(self) -> list[str]: + """Returns the message keys. + + Returns: + list[str]: A list of string keys associated with this message. + """ return self._keys @property def tags(self) -> list[str]: + """Returns the message tags for conditional forwarding. + + Returns: + list[str]: A list of string tags used for conditional forwarding. + """ return self._tags @property def watermark(self) -> datetime: + """Returns the watermark timestamp for this message. + + Returns: + datetime: The watermark timestamp, or None if not set. + """ return self._watermark @property def event_time(self) -> datetime: + """Returns the event time for this message. + + Returns: + datetime: The event time timestamp, or None if not set. + """ return self._event_time @property def headers(self) -> dict[str, str]: + """Returns the message headers. + + Returns: + dict[str, str]: A dictionary containing header key-value pairs for this message. + """ return self._headers @property def id(self) -> str: + """Returns the message ID. + + Returns: + str: The unique identifier for this message. + """ return self._id @classmethod diff --git a/pynumaflow/proto/accumulator/accumulator_pb2.py b/pynumaflow/proto/accumulator/accumulator_pb2.py index bc29e898..f1e8ec8d 100644 --- a/pynumaflow/proto/accumulator/accumulator_pb2.py +++ b/pynumaflow/proto/accumulator/accumulator_pb2.py @@ -7,6 +7,7 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import symbol_database as _symbol_database from google.protobuf.internal import builder as _builder + # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() @@ -16,32 +17,36 @@ from google.protobuf import timestamp_pb2 as google_dot_protobuf_dot_timestamp__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x11\x61\x63\x63umulator.proto\x12\x0e\x61\x63\x63umulator.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto\"\xf8\x01\n\x07Payload\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\n\n\x02id\x18\x05 \x01(\t\x12\x35\n\x07headers\x18\x06 \x03(\x0b\x32$.accumulator.v1.Payload.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\"\xbe\x02\n\x12\x41\x63\x63umulatorRequest\x12(\n\x07payload\x18\x01 \x01(\x0b\x32\x17.accumulator.v1.Payload\x12\x45\n\toperation\x18\x02 \x01(\x0b\x32\x32.accumulator.v1.AccumulatorRequest.WindowOperation\x1a\xb6\x01\n\x0fWindowOperation\x12G\n\x05\x65vent\x18\x01 \x01(\x0e\x32\x38.accumulator.v1.AccumulatorRequest.WindowOperation.Event\x12\x30\n\x0bkeyedWindow\x18\x02 \x01(\x0b\x32\x1b.accumulator.v1.KeyedWindow\"(\n\x05\x45vent\x12\x08\n\x04OPEN\x10\x00\x12\t\n\x05\x43LOSE\x10\x01\x12\n\n\x06\x41PPEND\x10\x02\"}\n\x0bKeyedWindow\x12)\n\x05start\x18\x01 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\'\n\x03\x65nd\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04slot\x18\x03 \x01(\t\x12\x0c\n\x04keys\x18\x04 \x03(\t\"\x87\x01\n\x13\x41\x63\x63umulatorResponse\x12(\n\x07payload\x18\x01 \x01(\x0b\x32\x17.accumulator.v1.Payload\x12+\n\x06window\x18\x02 \x01(\x0b\x32\x1b.accumulator.v1.KeyedWindow\x12\x0c\n\x04tags\x18\x03 \x03(\t\x12\x0b\n\x03\x45OF\x18\x04 \x01(\x08\"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\xac\x01\n\x0b\x41\x63\x63umulator\x12[\n\x0c\x41\x63\x63umulateFn\x12\".accumulator.v1.AccumulatorRequest\x1a#.accumulator.v1.AccumulatorResponse(\x01\x30\x01\x12@\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x1d.accumulator.v1.ReadyResponseBd\n#io.numaproj.numaflow.accumulator.v1Z=github.com/numaproj/numaflow-go/pkg/apis/proto/accumulator/v1b\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( + b'\n\x11\x61\x63\x63umulator.proto\x12\x0e\x61\x63\x63umulator.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto"\xf8\x01\n\x07Payload\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\n\n\x02id\x18\x05 \x01(\t\x12\x35\n\x07headers\x18\x06 \x03(\x0b\x32$.accumulator.v1.Payload.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01"\xbe\x02\n\x12\x41\x63\x63umulatorRequest\x12(\n\x07payload\x18\x01 \x01(\x0b\x32\x17.accumulator.v1.Payload\x12\x45\n\toperation\x18\x02 \x01(\x0b\x32\x32.accumulator.v1.AccumulatorRequest.WindowOperation\x1a\xb6\x01\n\x0fWindowOperation\x12G\n\x05\x65vent\x18\x01 \x01(\x0e\x32\x38.accumulator.v1.AccumulatorRequest.WindowOperation.Event\x12\x30\n\x0bkeyedWindow\x18\x02 \x01(\x0b\x32\x1b.accumulator.v1.KeyedWindow"(\n\x05\x45vent\x12\x08\n\x04OPEN\x10\x00\x12\t\n\x05\x43LOSE\x10\x01\x12\n\n\x06\x41PPEND\x10\x02"}\n\x0bKeyedWindow\x12)\n\x05start\x18\x01 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\'\n\x03\x65nd\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04slot\x18\x03 \x01(\t\x12\x0c\n\x04keys\x18\x04 \x03(\t"\x87\x01\n\x13\x41\x63\x63umulatorResponse\x12(\n\x07payload\x18\x01 \x01(\x0b\x32\x17.accumulator.v1.Payload\x12+\n\x06window\x18\x02 \x01(\x0b\x32\x1b.accumulator.v1.KeyedWindow\x12\x0c\n\x04tags\x18\x03 \x03(\t\x12\x0b\n\x03\x45OF\x18\x04 \x01(\x08"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\xac\x01\n\x0b\x41\x63\x63umulator\x12[\n\x0c\x41\x63\x63umulateFn\x12".accumulator.v1.AccumulatorRequest\x1a#.accumulator.v1.AccumulatorResponse(\x01\x30\x01\x12@\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x1d.accumulator.v1.ReadyResponseBd\n#io.numaproj.numaflow.accumulator.v1Z=github.com/numaproj/numaflow-go/pkg/apis/proto/accumulator/v1b\x06proto3' +) _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) -_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'accumulator_pb2', _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, "accumulator_pb2", _globals) if _descriptor._USE_C_DESCRIPTORS == False: - _globals['DESCRIPTOR']._options = None - _globals['DESCRIPTOR']._serialized_options = b'\n#io.numaproj.numaflow.accumulator.v1Z=github.com/numaproj/numaflow-go/pkg/apis/proto/accumulator/v1' - _globals['_PAYLOAD_HEADERSENTRY']._options = None - _globals['_PAYLOAD_HEADERSENTRY']._serialized_options = b'8\001' - _globals['_PAYLOAD']._serialized_start=100 - _globals['_PAYLOAD']._serialized_end=348 - _globals['_PAYLOAD_HEADERSENTRY']._serialized_start=302 - _globals['_PAYLOAD_HEADERSENTRY']._serialized_end=348 - _globals['_ACCUMULATORREQUEST']._serialized_start=351 - _globals['_ACCUMULATORREQUEST']._serialized_end=669 - _globals['_ACCUMULATORREQUEST_WINDOWOPERATION']._serialized_start=487 - _globals['_ACCUMULATORREQUEST_WINDOWOPERATION']._serialized_end=669 - _globals['_ACCUMULATORREQUEST_WINDOWOPERATION_EVENT']._serialized_start=629 - _globals['_ACCUMULATORREQUEST_WINDOWOPERATION_EVENT']._serialized_end=669 - _globals['_KEYEDWINDOW']._serialized_start=671 - _globals['_KEYEDWINDOW']._serialized_end=796 - _globals['_ACCUMULATORRESPONSE']._serialized_start=799 - _globals['_ACCUMULATORRESPONSE']._serialized_end=934 - _globals['_READYRESPONSE']._serialized_start=936 - _globals['_READYRESPONSE']._serialized_end=966 - _globals['_ACCUMULATOR']._serialized_start=969 - _globals['_ACCUMULATOR']._serialized_end=1141 + _globals["DESCRIPTOR"]._options = None + _globals[ + "DESCRIPTOR" + ]._serialized_options = b"\n#io.numaproj.numaflow.accumulator.v1Z=github.com/numaproj/numaflow-go/pkg/apis/proto/accumulator/v1" + _globals["_PAYLOAD_HEADERSENTRY"]._options = None + _globals["_PAYLOAD_HEADERSENTRY"]._serialized_options = b"8\001" + _globals["_PAYLOAD"]._serialized_start = 100 + _globals["_PAYLOAD"]._serialized_end = 348 + _globals["_PAYLOAD_HEADERSENTRY"]._serialized_start = 302 + _globals["_PAYLOAD_HEADERSENTRY"]._serialized_end = 348 + _globals["_ACCUMULATORREQUEST"]._serialized_start = 351 + _globals["_ACCUMULATORREQUEST"]._serialized_end = 669 + _globals["_ACCUMULATORREQUEST_WINDOWOPERATION"]._serialized_start = 487 + _globals["_ACCUMULATORREQUEST_WINDOWOPERATION"]._serialized_end = 669 + _globals["_ACCUMULATORREQUEST_WINDOWOPERATION_EVENT"]._serialized_start = 629 + _globals["_ACCUMULATORREQUEST_WINDOWOPERATION_EVENT"]._serialized_end = 669 + _globals["_KEYEDWINDOW"]._serialized_start = 671 + _globals["_KEYEDWINDOW"]._serialized_end = 796 + _globals["_ACCUMULATORRESPONSE"]._serialized_start = 799 + _globals["_ACCUMULATORRESPONSE"]._serialized_end = 934 + _globals["_READYRESPONSE"]._serialized_start = 936 + _globals["_READYRESPONSE"]._serialized_end = 966 + _globals["_ACCUMULATOR"]._serialized_start = 969 + _globals["_ACCUMULATOR"]._serialized_end = 1141 # @@protoc_insertion_point(module_scope) diff --git a/pynumaflow/proto/accumulator/accumulator_pb2.pyi b/pynumaflow/proto/accumulator/accumulator_pb2.pyi index bfa0bbac..d9f0f7a5 100644 --- a/pynumaflow/proto/accumulator/accumulator_pb2.pyi +++ b/pynumaflow/proto/accumulator/accumulator_pb2.pyi @@ -4,12 +4,19 @@ from google.protobuf.internal import containers as _containers from google.protobuf.internal import enum_type_wrapper as _enum_type_wrapper from google.protobuf import descriptor as _descriptor from google.protobuf import message as _message -from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union +from typing import ( + ClassVar as _ClassVar, + Iterable as _Iterable, + Mapping as _Mapping, + Optional as _Optional, + Union as _Union, +) DESCRIPTOR: _descriptor.FileDescriptor class Payload(_message.Message): __slots__ = ("keys", "value", "event_time", "watermark", "id", "headers") + class HeadersEntry(_message.Message): __slots__ = ("key", "value") KEY_FIELD_NUMBER: _ClassVar[int] @@ -29,12 +36,22 @@ class Payload(_message.Message): watermark: _timestamp_pb2.Timestamp id: str headers: _containers.ScalarMap[str, str] - def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., id: _Optional[str] = ..., headers: _Optional[_Mapping[str, str]] = ...) -> None: ... + def __init__( + self, + keys: _Optional[_Iterable[str]] = ..., + value: _Optional[bytes] = ..., + event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + id: _Optional[str] = ..., + headers: _Optional[_Mapping[str, str]] = ..., + ) -> None: ... class AccumulatorRequest(_message.Message): __slots__ = ("payload", "operation") + class WindowOperation(_message.Message): __slots__ = ("event", "keyedWindow") + class Event(int, metaclass=_enum_type_wrapper.EnumTypeWrapper): __slots__ = () OPEN: _ClassVar[AccumulatorRequest.WindowOperation.Event] @@ -47,12 +64,20 @@ class AccumulatorRequest(_message.Message): KEYEDWINDOW_FIELD_NUMBER: _ClassVar[int] event: AccumulatorRequest.WindowOperation.Event keyedWindow: KeyedWindow - def __init__(self, event: _Optional[_Union[AccumulatorRequest.WindowOperation.Event, str]] = ..., keyedWindow: _Optional[_Union[KeyedWindow, _Mapping]] = ...) -> None: ... + def __init__( + self, + event: _Optional[_Union[AccumulatorRequest.WindowOperation.Event, str]] = ..., + keyedWindow: _Optional[_Union[KeyedWindow, _Mapping]] = ..., + ) -> None: ... PAYLOAD_FIELD_NUMBER: _ClassVar[int] OPERATION_FIELD_NUMBER: _ClassVar[int] payload: Payload operation: AccumulatorRequest.WindowOperation - def __init__(self, payload: _Optional[_Union[Payload, _Mapping]] = ..., operation: _Optional[_Union[AccumulatorRequest.WindowOperation, _Mapping]] = ...) -> None: ... + def __init__( + self, + payload: _Optional[_Union[Payload, _Mapping]] = ..., + operation: _Optional[_Union[AccumulatorRequest.WindowOperation, _Mapping]] = ..., + ) -> None: ... class KeyedWindow(_message.Message): __slots__ = ("start", "end", "slot", "keys") @@ -64,7 +89,13 @@ class KeyedWindow(_message.Message): end: _timestamp_pb2.Timestamp slot: str keys: _containers.RepeatedScalarFieldContainer[str] - def __init__(self, start: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., end: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., slot: _Optional[str] = ..., keys: _Optional[_Iterable[str]] = ...) -> None: ... + def __init__( + self, + start: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + end: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + slot: _Optional[str] = ..., + keys: _Optional[_Iterable[str]] = ..., + ) -> None: ... class AccumulatorResponse(_message.Message): __slots__ = ("payload", "window", "tags", "EOF") @@ -76,7 +107,13 @@ class AccumulatorResponse(_message.Message): window: KeyedWindow tags: _containers.RepeatedScalarFieldContainer[str] EOF: bool - def __init__(self, payload: _Optional[_Union[Payload, _Mapping]] = ..., window: _Optional[_Union[KeyedWindow, _Mapping]] = ..., tags: _Optional[_Iterable[str]] = ..., EOF: bool = ...) -> None: ... + def __init__( + self, + payload: _Optional[_Union[Payload, _Mapping]] = ..., + window: _Optional[_Union[KeyedWindow, _Mapping]] = ..., + tags: _Optional[_Iterable[str]] = ..., + EOF: bool = ..., + ) -> None: ... class ReadyResponse(_message.Message): __slots__ = ("ready",) diff --git a/pynumaflow/proto/accumulator/accumulator_pb2_grpc.py b/pynumaflow/proto/accumulator/accumulator_pb2_grpc.py index 4f0f208f..f41606dd 100644 --- a/pynumaflow/proto/accumulator/accumulator_pb2_grpc.py +++ b/pynumaflow/proto/accumulator/accumulator_pb2_grpc.py @@ -19,15 +19,15 @@ def __init__(self, channel): channel: A grpc.Channel. """ self.AccumulateFn = channel.stream_stream( - '/accumulator.v1.Accumulator/AccumulateFn', - request_serializer=accumulator__pb2.AccumulatorRequest.SerializeToString, - response_deserializer=accumulator__pb2.AccumulatorResponse.FromString, - ) + "/accumulator.v1.Accumulator/AccumulateFn", + request_serializer=accumulator__pb2.AccumulatorRequest.SerializeToString, + response_deserializer=accumulator__pb2.AccumulatorResponse.FromString, + ) self.IsReady = channel.unary_unary( - '/accumulator.v1.Accumulator/IsReady', - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=accumulator__pb2.ReadyResponse.FromString, - ) + "/accumulator.v1.Accumulator/IsReady", + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=accumulator__pb2.ReadyResponse.FromString, + ) class AccumulatorServicer(object): @@ -37,39 +37,38 @@ class AccumulatorServicer(object): """ def AccumulateFn(self, request_iterator, context): - """AccumulateFn applies a accumulate function to a request stream. - """ + """AccumulateFn applies a accumulate function to a request stream.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def IsReady(self, request, context): - """IsReady is the heartbeat endpoint for gRPC. - """ + """IsReady is the heartbeat endpoint for gRPC.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def add_AccumulatorServicer_to_server(servicer, server): rpc_method_handlers = { - 'AccumulateFn': grpc.stream_stream_rpc_method_handler( - servicer.AccumulateFn, - request_deserializer=accumulator__pb2.AccumulatorRequest.FromString, - response_serializer=accumulator__pb2.AccumulatorResponse.SerializeToString, - ), - 'IsReady': grpc.unary_unary_rpc_method_handler( - servicer.IsReady, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=accumulator__pb2.ReadyResponse.SerializeToString, - ), + "AccumulateFn": grpc.stream_stream_rpc_method_handler( + servicer.AccumulateFn, + request_deserializer=accumulator__pb2.AccumulatorRequest.FromString, + response_serializer=accumulator__pb2.AccumulatorResponse.SerializeToString, + ), + "IsReady": grpc.unary_unary_rpc_method_handler( + servicer.IsReady, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=accumulator__pb2.ReadyResponse.SerializeToString, + ), } generic_handler = grpc.method_handlers_generic_handler( - 'accumulator.v1.Accumulator', rpc_method_handlers) + "accumulator.v1.Accumulator", rpc_method_handlers + ) server.add_generic_rpc_handlers((generic_handler,)) - # This class is part of an EXPERIMENTAL API. +# This class is part of an EXPERIMENTAL API. class Accumulator(object): """AccumulatorWindow describes a special kind of SessionWindow (similar to Global Window) where output should always have monotonically increasing WM but it can be manipulated through event-time by reordering the messages. @@ -77,35 +76,59 @@ class Accumulator(object): """ @staticmethod - def AccumulateFn(request_iterator, + def AccumulateFn( + request_iterator, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.stream_stream( + request_iterator, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.stream_stream(request_iterator, target, '/accumulator.v1.Accumulator/AccumulateFn', + "/accumulator.v1.Accumulator/AccumulateFn", accumulator__pb2.AccumulatorRequest.SerializeToString, accumulator__pb2.AccumulatorResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) @staticmethod - def IsReady(request, + def IsReady( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.unary_unary(request, target, '/accumulator.v1.Accumulator/IsReady', + "/accumulator.v1.Accumulator/IsReady", google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, accumulator__pb2.ReadyResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) diff --git a/pynumaflow/proto/mapper/map_pb2.py b/pynumaflow/proto/mapper/map_pb2.py index 78abbd69..80472786 100644 --- a/pynumaflow/proto/mapper/map_pb2.py +++ b/pynumaflow/proto/mapper/map_pb2.py @@ -7,6 +7,7 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import symbol_database as _symbol_database from google.protobuf.internal import builder as _builder + # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() @@ -16,32 +17,36 @@ from google.protobuf import timestamp_pb2 as google_dot_protobuf_dot_timestamp__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\tmap.proto\x12\x06map.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto\"\xac\x03\n\nMapRequest\x12+\n\x07request\x18\x01 \x01(\x0b\x32\x1a.map.v1.MapRequest.Request\x12\n\n\x02id\x18\x02 \x01(\t\x12)\n\thandshake\x18\x03 \x01(\x0b\x32\x11.map.v1.HandshakeH\x00\x88\x01\x01\x12/\n\x06status\x18\x04 \x01(\x0b\x32\x1a.map.v1.TransmissionStatusH\x01\x88\x01\x01\x1a\xef\x01\n\x07Request\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x38\n\x07headers\x18\x05 \x03(\x0b\x32\'.map.v1.MapRequest.Request.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x42\x0c\n\n_handshakeB\t\n\x07_status\"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08\"!\n\x12TransmissionStatus\x12\x0b\n\x03\x65ot\x18\x01 \x01(\x08\"\xf0\x01\n\x0bMapResponse\x12+\n\x07results\x18\x01 \x03(\x0b\x32\x1a.map.v1.MapResponse.Result\x12\n\n\x02id\x18\x02 \x01(\t\x12)\n\thandshake\x18\x03 \x01(\x0b\x32\x11.map.v1.HandshakeH\x00\x88\x01\x01\x12/\n\x06status\x18\x04 \x01(\x0b\x32\x1a.map.v1.TransmissionStatusH\x01\x88\x01\x01\x1a\x33\n\x06Result\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12\x0c\n\x04tags\x18\x03 \x03(\tB\x0c\n\n_handshakeB\t\n\x07_status\"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32u\n\x03Map\x12\x34\n\x05MapFn\x12\x12.map.v1.MapRequest\x1a\x13.map.v1.MapResponse(\x01\x30\x01\x12\x38\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x15.map.v1.ReadyResponseB7Z5github.com/numaproj/numaflow-go/pkg/apis/proto/map/v1b\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( + b'\n\tmap.proto\x12\x06map.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto"\xac\x03\n\nMapRequest\x12+\n\x07request\x18\x01 \x01(\x0b\x32\x1a.map.v1.MapRequest.Request\x12\n\n\x02id\x18\x02 \x01(\t\x12)\n\thandshake\x18\x03 \x01(\x0b\x32\x11.map.v1.HandshakeH\x00\x88\x01\x01\x12/\n\x06status\x18\x04 \x01(\x0b\x32\x1a.map.v1.TransmissionStatusH\x01\x88\x01\x01\x1a\xef\x01\n\x07Request\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x38\n\x07headers\x18\x05 \x03(\x0b\x32\'.map.v1.MapRequest.Request.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x42\x0c\n\n_handshakeB\t\n\x07_status"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08"!\n\x12TransmissionStatus\x12\x0b\n\x03\x65ot\x18\x01 \x01(\x08"\xf0\x01\n\x0bMapResponse\x12+\n\x07results\x18\x01 \x03(\x0b\x32\x1a.map.v1.MapResponse.Result\x12\n\n\x02id\x18\x02 \x01(\t\x12)\n\thandshake\x18\x03 \x01(\x0b\x32\x11.map.v1.HandshakeH\x00\x88\x01\x01\x12/\n\x06status\x18\x04 \x01(\x0b\x32\x1a.map.v1.TransmissionStatusH\x01\x88\x01\x01\x1a\x33\n\x06Result\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12\x0c\n\x04tags\x18\x03 \x03(\tB\x0c\n\n_handshakeB\t\n\x07_status"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32u\n\x03Map\x12\x34\n\x05MapFn\x12\x12.map.v1.MapRequest\x1a\x13.map.v1.MapResponse(\x01\x30\x01\x12\x38\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x15.map.v1.ReadyResponseB7Z5github.com/numaproj/numaflow-go/pkg/apis/proto/map/v1b\x06proto3' +) _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) -_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'map_pb2', _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, "map_pb2", _globals) if _descriptor._USE_C_DESCRIPTORS == False: - _globals['DESCRIPTOR']._options = None - _globals['DESCRIPTOR']._serialized_options = b'Z5github.com/numaproj/numaflow-go/pkg/apis/proto/map/v1' - _globals['_MAPREQUEST_REQUEST_HEADERSENTRY']._options = None - _globals['_MAPREQUEST_REQUEST_HEADERSENTRY']._serialized_options = b'8\001' - _globals['_MAPREQUEST']._serialized_start=84 - _globals['_MAPREQUEST']._serialized_end=512 - _globals['_MAPREQUEST_REQUEST']._serialized_start=248 - _globals['_MAPREQUEST_REQUEST']._serialized_end=487 - _globals['_MAPREQUEST_REQUEST_HEADERSENTRY']._serialized_start=441 - _globals['_MAPREQUEST_REQUEST_HEADERSENTRY']._serialized_end=487 - _globals['_HANDSHAKE']._serialized_start=514 - _globals['_HANDSHAKE']._serialized_end=538 - _globals['_TRANSMISSIONSTATUS']._serialized_start=540 - _globals['_TRANSMISSIONSTATUS']._serialized_end=573 - _globals['_MAPRESPONSE']._serialized_start=576 - _globals['_MAPRESPONSE']._serialized_end=816 - _globals['_MAPRESPONSE_RESULT']._serialized_start=740 - _globals['_MAPRESPONSE_RESULT']._serialized_end=791 - _globals['_READYRESPONSE']._serialized_start=818 - _globals['_READYRESPONSE']._serialized_end=848 - _globals['_MAP']._serialized_start=850 - _globals['_MAP']._serialized_end=967 + _globals["DESCRIPTOR"]._options = None + _globals[ + "DESCRIPTOR" + ]._serialized_options = b"Z5github.com/numaproj/numaflow-go/pkg/apis/proto/map/v1" + _globals["_MAPREQUEST_REQUEST_HEADERSENTRY"]._options = None + _globals["_MAPREQUEST_REQUEST_HEADERSENTRY"]._serialized_options = b"8\001" + _globals["_MAPREQUEST"]._serialized_start = 84 + _globals["_MAPREQUEST"]._serialized_end = 512 + _globals["_MAPREQUEST_REQUEST"]._serialized_start = 248 + _globals["_MAPREQUEST_REQUEST"]._serialized_end = 487 + _globals["_MAPREQUEST_REQUEST_HEADERSENTRY"]._serialized_start = 441 + _globals["_MAPREQUEST_REQUEST_HEADERSENTRY"]._serialized_end = 487 + _globals["_HANDSHAKE"]._serialized_start = 514 + _globals["_HANDSHAKE"]._serialized_end = 538 + _globals["_TRANSMISSIONSTATUS"]._serialized_start = 540 + _globals["_TRANSMISSIONSTATUS"]._serialized_end = 573 + _globals["_MAPRESPONSE"]._serialized_start = 576 + _globals["_MAPRESPONSE"]._serialized_end = 816 + _globals["_MAPRESPONSE_RESULT"]._serialized_start = 740 + _globals["_MAPRESPONSE_RESULT"]._serialized_end = 791 + _globals["_READYRESPONSE"]._serialized_start = 818 + _globals["_READYRESPONSE"]._serialized_end = 848 + _globals["_MAP"]._serialized_start = 850 + _globals["_MAP"]._serialized_end = 967 # @@protoc_insertion_point(module_scope) diff --git a/pynumaflow/proto/mapper/map_pb2.pyi b/pynumaflow/proto/mapper/map_pb2.pyi index 22eed7b9..9832bc3e 100644 --- a/pynumaflow/proto/mapper/map_pb2.pyi +++ b/pynumaflow/proto/mapper/map_pb2.pyi @@ -3,14 +3,22 @@ from google.protobuf import timestamp_pb2 as _timestamp_pb2 from google.protobuf.internal import containers as _containers from google.protobuf import descriptor as _descriptor from google.protobuf import message as _message -from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union +from typing import ( + ClassVar as _ClassVar, + Iterable as _Iterable, + Mapping as _Mapping, + Optional as _Optional, + Union as _Union, +) DESCRIPTOR: _descriptor.FileDescriptor class MapRequest(_message.Message): __slots__ = ("request", "id", "handshake", "status") + class Request(_message.Message): __slots__ = ("keys", "value", "event_time", "watermark", "headers") + class HeadersEntry(_message.Message): __slots__ = ("key", "value") KEY_FIELD_NUMBER: _ClassVar[int] @@ -28,7 +36,14 @@ class MapRequest(_message.Message): event_time: _timestamp_pb2.Timestamp watermark: _timestamp_pb2.Timestamp headers: _containers.ScalarMap[str, str] - def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., headers: _Optional[_Mapping[str, str]] = ...) -> None: ... + def __init__( + self, + keys: _Optional[_Iterable[str]] = ..., + value: _Optional[bytes] = ..., + event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + headers: _Optional[_Mapping[str, str]] = ..., + ) -> None: ... REQUEST_FIELD_NUMBER: _ClassVar[int] ID_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] @@ -37,7 +52,13 @@ class MapRequest(_message.Message): id: str handshake: Handshake status: TransmissionStatus - def __init__(self, request: _Optional[_Union[MapRequest.Request, _Mapping]] = ..., id: _Optional[str] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ..., status: _Optional[_Union[TransmissionStatus, _Mapping]] = ...) -> None: ... + def __init__( + self, + request: _Optional[_Union[MapRequest.Request, _Mapping]] = ..., + id: _Optional[str] = ..., + handshake: _Optional[_Union[Handshake, _Mapping]] = ..., + status: _Optional[_Union[TransmissionStatus, _Mapping]] = ..., + ) -> None: ... class Handshake(_message.Message): __slots__ = ("sot",) @@ -53,6 +74,7 @@ class TransmissionStatus(_message.Message): class MapResponse(_message.Message): __slots__ = ("results", "id", "handshake", "status") + class Result(_message.Message): __slots__ = ("keys", "value", "tags") KEYS_FIELD_NUMBER: _ClassVar[int] @@ -61,7 +83,12 @@ class MapResponse(_message.Message): keys: _containers.RepeatedScalarFieldContainer[str] value: bytes tags: _containers.RepeatedScalarFieldContainer[str] - def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., tags: _Optional[_Iterable[str]] = ...) -> None: ... + def __init__( + self, + keys: _Optional[_Iterable[str]] = ..., + value: _Optional[bytes] = ..., + tags: _Optional[_Iterable[str]] = ..., + ) -> None: ... RESULTS_FIELD_NUMBER: _ClassVar[int] ID_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] @@ -70,7 +97,13 @@ class MapResponse(_message.Message): id: str handshake: Handshake status: TransmissionStatus - def __init__(self, results: _Optional[_Iterable[_Union[MapResponse.Result, _Mapping]]] = ..., id: _Optional[str] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ..., status: _Optional[_Union[TransmissionStatus, _Mapping]] = ...) -> None: ... + def __init__( + self, + results: _Optional[_Iterable[_Union[MapResponse.Result, _Mapping]]] = ..., + id: _Optional[str] = ..., + handshake: _Optional[_Union[Handshake, _Mapping]] = ..., + status: _Optional[_Union[TransmissionStatus, _Mapping]] = ..., + ) -> None: ... class ReadyResponse(_message.Message): __slots__ = ("ready",) diff --git a/pynumaflow/proto/mapper/map_pb2_grpc.py b/pynumaflow/proto/mapper/map_pb2_grpc.py index 87fa0b27..97834865 100644 --- a/pynumaflow/proto/mapper/map_pb2_grpc.py +++ b/pynumaflow/proto/mapper/map_pb2_grpc.py @@ -16,87 +16,108 @@ def __init__(self, channel): channel: A grpc.Channel. """ self.MapFn = channel.stream_stream( - '/map.v1.Map/MapFn', - request_serializer=map__pb2.MapRequest.SerializeToString, - response_deserializer=map__pb2.MapResponse.FromString, - ) + "/map.v1.Map/MapFn", + request_serializer=map__pb2.MapRequest.SerializeToString, + response_deserializer=map__pb2.MapResponse.FromString, + ) self.IsReady = channel.unary_unary( - '/map.v1.Map/IsReady', - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=map__pb2.ReadyResponse.FromString, - ) + "/map.v1.Map/IsReady", + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=map__pb2.ReadyResponse.FromString, + ) class MapServicer(object): """Missing associated documentation comment in .proto file.""" def MapFn(self, request_iterator, context): - """MapFn applies a function to each map request element. - """ + """MapFn applies a function to each map request element.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def IsReady(self, request, context): - """IsReady is the heartbeat endpoint for gRPC. - """ + """IsReady is the heartbeat endpoint for gRPC.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def add_MapServicer_to_server(servicer, server): rpc_method_handlers = { - 'MapFn': grpc.stream_stream_rpc_method_handler( - servicer.MapFn, - request_deserializer=map__pb2.MapRequest.FromString, - response_serializer=map__pb2.MapResponse.SerializeToString, - ), - 'IsReady': grpc.unary_unary_rpc_method_handler( - servicer.IsReady, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=map__pb2.ReadyResponse.SerializeToString, - ), + "MapFn": grpc.stream_stream_rpc_method_handler( + servicer.MapFn, + request_deserializer=map__pb2.MapRequest.FromString, + response_serializer=map__pb2.MapResponse.SerializeToString, + ), + "IsReady": grpc.unary_unary_rpc_method_handler( + servicer.IsReady, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=map__pb2.ReadyResponse.SerializeToString, + ), } - generic_handler = grpc.method_handlers_generic_handler( - 'map.v1.Map', rpc_method_handlers) + generic_handler = grpc.method_handlers_generic_handler("map.v1.Map", rpc_method_handlers) server.add_generic_rpc_handlers((generic_handler,)) - # This class is part of an EXPERIMENTAL API. +# This class is part of an EXPERIMENTAL API. class Map(object): """Missing associated documentation comment in .proto file.""" @staticmethod - def MapFn(request_iterator, + def MapFn( + request_iterator, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.stream_stream( + request_iterator, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.stream_stream(request_iterator, target, '/map.v1.Map/MapFn', + "/map.v1.Map/MapFn", map__pb2.MapRequest.SerializeToString, map__pb2.MapResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) @staticmethod - def IsReady(request, + def IsReady( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.unary_unary(request, target, '/map.v1.Map/IsReady', + "/map.v1.Map/IsReady", google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, map__pb2.ReadyResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) diff --git a/pynumaflow/proto/reducer/reduce_pb2.py b/pynumaflow/proto/reducer/reduce_pb2.py index ecb076e3..f5c1ef83 100644 --- a/pynumaflow/proto/reducer/reduce_pb2.py +++ b/pynumaflow/proto/reducer/reduce_pb2.py @@ -7,6 +7,7 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import symbol_database as _symbol_database from google.protobuf.internal import builder as _builder + # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() @@ -16,33 +17,35 @@ from google.protobuf import timestamp_pb2 as google_dot_protobuf_dot_timestamp__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x0creduce.proto\x12\treduce.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto\"\x98\x04\n\rReduceRequest\x12\x31\n\x07payload\x18\x01 \x01(\x0b\x32 .reduce.v1.ReduceRequest.Payload\x12;\n\toperation\x18\x02 \x01(\x0b\x32(.reduce.v1.ReduceRequest.WindowOperation\x1a\x9e\x01\n\x0fWindowOperation\x12=\n\x05\x65vent\x18\x01 \x01(\x0e\x32..reduce.v1.ReduceRequest.WindowOperation.Event\x12\"\n\x07windows\x18\x02 \x03(\x0b\x32\x11.reduce.v1.Window\"(\n\x05\x45vent\x12\x08\n\x04OPEN\x10\x00\x12\t\n\x05\x43LOSE\x10\x01\x12\n\n\x06\x41PPEND\x10\x04\x1a\xf5\x01\n\x07Payload\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12>\n\x07headers\x18\x05 \x03(\x0b\x32-.reduce.v1.ReduceRequest.Payload.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\"j\n\x06Window\x12)\n\x05start\x18\x01 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\'\n\x03\x65nd\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04slot\x18\x03 \x01(\t\"\xa7\x01\n\x0eReduceResponse\x12\x30\n\x06result\x18\x01 \x01(\x0b\x32 .reduce.v1.ReduceResponse.Result\x12!\n\x06window\x18\x02 \x01(\x0b\x32\x11.reduce.v1.Window\x12\x0b\n\x03\x45OF\x18\x03 \x01(\x08\x1a\x33\n\x06Result\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12\x0c\n\x04tags\x18\x03 \x03(\t\"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\x8a\x01\n\x06Reduce\x12\x43\n\x08ReduceFn\x12\x18.reduce.v1.ReduceRequest\x1a\x19.reduce.v1.ReduceResponse(\x01\x30\x01\x12;\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x18.reduce.v1.ReadyResponseb\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( + b'\n\x0creduce.proto\x12\treduce.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto"\x98\x04\n\rReduceRequest\x12\x31\n\x07payload\x18\x01 \x01(\x0b\x32 .reduce.v1.ReduceRequest.Payload\x12;\n\toperation\x18\x02 \x01(\x0b\x32(.reduce.v1.ReduceRequest.WindowOperation\x1a\x9e\x01\n\x0fWindowOperation\x12=\n\x05\x65vent\x18\x01 \x01(\x0e\x32..reduce.v1.ReduceRequest.WindowOperation.Event\x12"\n\x07windows\x18\x02 \x03(\x0b\x32\x11.reduce.v1.Window"(\n\x05\x45vent\x12\x08\n\x04OPEN\x10\x00\x12\t\n\x05\x43LOSE\x10\x01\x12\n\n\x06\x41PPEND\x10\x04\x1a\xf5\x01\n\x07Payload\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12>\n\x07headers\x18\x05 \x03(\x0b\x32-.reduce.v1.ReduceRequest.Payload.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01"j\n\x06Window\x12)\n\x05start\x18\x01 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\'\n\x03\x65nd\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04slot\x18\x03 \x01(\t"\xa7\x01\n\x0eReduceResponse\x12\x30\n\x06result\x18\x01 \x01(\x0b\x32 .reduce.v1.ReduceResponse.Result\x12!\n\x06window\x18\x02 \x01(\x0b\x32\x11.reduce.v1.Window\x12\x0b\n\x03\x45OF\x18\x03 \x01(\x08\x1a\x33\n\x06Result\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12\x0c\n\x04tags\x18\x03 \x03(\t"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\x8a\x01\n\x06Reduce\x12\x43\n\x08ReduceFn\x12\x18.reduce.v1.ReduceRequest\x1a\x19.reduce.v1.ReduceResponse(\x01\x30\x01\x12;\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x18.reduce.v1.ReadyResponseb\x06proto3' +) _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) -_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'reduce_pb2', _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, "reduce_pb2", _globals) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None - _globals['_REDUCEREQUEST_PAYLOAD_HEADERSENTRY']._options = None - _globals['_REDUCEREQUEST_PAYLOAD_HEADERSENTRY']._serialized_options = b'8\001' - _globals['_REDUCEREQUEST']._serialized_start=90 - _globals['_REDUCEREQUEST']._serialized_end=626 - _globals['_REDUCEREQUEST_WINDOWOPERATION']._serialized_start=220 - _globals['_REDUCEREQUEST_WINDOWOPERATION']._serialized_end=378 - _globals['_REDUCEREQUEST_WINDOWOPERATION_EVENT']._serialized_start=338 - _globals['_REDUCEREQUEST_WINDOWOPERATION_EVENT']._serialized_end=378 - _globals['_REDUCEREQUEST_PAYLOAD']._serialized_start=381 - _globals['_REDUCEREQUEST_PAYLOAD']._serialized_end=626 - _globals['_REDUCEREQUEST_PAYLOAD_HEADERSENTRY']._serialized_start=580 - _globals['_REDUCEREQUEST_PAYLOAD_HEADERSENTRY']._serialized_end=626 - _globals['_WINDOW']._serialized_start=628 - _globals['_WINDOW']._serialized_end=734 - _globals['_REDUCERESPONSE']._serialized_start=737 - _globals['_REDUCERESPONSE']._serialized_end=904 - _globals['_REDUCERESPONSE_RESULT']._serialized_start=853 - _globals['_REDUCERESPONSE_RESULT']._serialized_end=904 - _globals['_READYRESPONSE']._serialized_start=906 - _globals['_READYRESPONSE']._serialized_end=936 - _globals['_REDUCE']._serialized_start=939 - _globals['_REDUCE']._serialized_end=1077 + DESCRIPTOR._options = None + _globals["_REDUCEREQUEST_PAYLOAD_HEADERSENTRY"]._options = None + _globals["_REDUCEREQUEST_PAYLOAD_HEADERSENTRY"]._serialized_options = b"8\001" + _globals["_REDUCEREQUEST"]._serialized_start = 90 + _globals["_REDUCEREQUEST"]._serialized_end = 626 + _globals["_REDUCEREQUEST_WINDOWOPERATION"]._serialized_start = 220 + _globals["_REDUCEREQUEST_WINDOWOPERATION"]._serialized_end = 378 + _globals["_REDUCEREQUEST_WINDOWOPERATION_EVENT"]._serialized_start = 338 + _globals["_REDUCEREQUEST_WINDOWOPERATION_EVENT"]._serialized_end = 378 + _globals["_REDUCEREQUEST_PAYLOAD"]._serialized_start = 381 + _globals["_REDUCEREQUEST_PAYLOAD"]._serialized_end = 626 + _globals["_REDUCEREQUEST_PAYLOAD_HEADERSENTRY"]._serialized_start = 580 + _globals["_REDUCEREQUEST_PAYLOAD_HEADERSENTRY"]._serialized_end = 626 + _globals["_WINDOW"]._serialized_start = 628 + _globals["_WINDOW"]._serialized_end = 734 + _globals["_REDUCERESPONSE"]._serialized_start = 737 + _globals["_REDUCERESPONSE"]._serialized_end = 904 + _globals["_REDUCERESPONSE_RESULT"]._serialized_start = 853 + _globals["_REDUCERESPONSE_RESULT"]._serialized_end = 904 + _globals["_READYRESPONSE"]._serialized_start = 906 + _globals["_READYRESPONSE"]._serialized_end = 936 + _globals["_REDUCE"]._serialized_start = 939 + _globals["_REDUCE"]._serialized_end = 1077 # @@protoc_insertion_point(module_scope) diff --git a/pynumaflow/proto/reducer/reduce_pb2.pyi b/pynumaflow/proto/reducer/reduce_pb2.pyi index 80a974ff..88b27d53 100644 --- a/pynumaflow/proto/reducer/reduce_pb2.pyi +++ b/pynumaflow/proto/reducer/reduce_pb2.pyi @@ -4,14 +4,22 @@ from google.protobuf.internal import containers as _containers from google.protobuf.internal import enum_type_wrapper as _enum_type_wrapper from google.protobuf import descriptor as _descriptor from google.protobuf import message as _message -from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union +from typing import ( + ClassVar as _ClassVar, + Iterable as _Iterable, + Mapping as _Mapping, + Optional as _Optional, + Union as _Union, +) DESCRIPTOR: _descriptor.FileDescriptor class ReduceRequest(_message.Message): __slots__ = ("payload", "operation") + class WindowOperation(_message.Message): __slots__ = ("event", "windows") + class Event(int, metaclass=_enum_type_wrapper.EnumTypeWrapper): __slots__ = () OPEN: _ClassVar[ReduceRequest.WindowOperation.Event] @@ -24,9 +32,15 @@ class ReduceRequest(_message.Message): WINDOWS_FIELD_NUMBER: _ClassVar[int] event: ReduceRequest.WindowOperation.Event windows: _containers.RepeatedCompositeFieldContainer[Window] - def __init__(self, event: _Optional[_Union[ReduceRequest.WindowOperation.Event, str]] = ..., windows: _Optional[_Iterable[_Union[Window, _Mapping]]] = ...) -> None: ... + def __init__( + self, + event: _Optional[_Union[ReduceRequest.WindowOperation.Event, str]] = ..., + windows: _Optional[_Iterable[_Union[Window, _Mapping]]] = ..., + ) -> None: ... + class Payload(_message.Message): __slots__ = ("keys", "value", "event_time", "watermark", "headers") + class HeadersEntry(_message.Message): __slots__ = ("key", "value") KEY_FIELD_NUMBER: _ClassVar[int] @@ -44,12 +58,23 @@ class ReduceRequest(_message.Message): event_time: _timestamp_pb2.Timestamp watermark: _timestamp_pb2.Timestamp headers: _containers.ScalarMap[str, str] - def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., headers: _Optional[_Mapping[str, str]] = ...) -> None: ... + def __init__( + self, + keys: _Optional[_Iterable[str]] = ..., + value: _Optional[bytes] = ..., + event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + headers: _Optional[_Mapping[str, str]] = ..., + ) -> None: ... PAYLOAD_FIELD_NUMBER: _ClassVar[int] OPERATION_FIELD_NUMBER: _ClassVar[int] payload: ReduceRequest.Payload operation: ReduceRequest.WindowOperation - def __init__(self, payload: _Optional[_Union[ReduceRequest.Payload, _Mapping]] = ..., operation: _Optional[_Union[ReduceRequest.WindowOperation, _Mapping]] = ...) -> None: ... + def __init__( + self, + payload: _Optional[_Union[ReduceRequest.Payload, _Mapping]] = ..., + operation: _Optional[_Union[ReduceRequest.WindowOperation, _Mapping]] = ..., + ) -> None: ... class Window(_message.Message): __slots__ = ("start", "end", "slot") @@ -59,10 +84,16 @@ class Window(_message.Message): start: _timestamp_pb2.Timestamp end: _timestamp_pb2.Timestamp slot: str - def __init__(self, start: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., end: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., slot: _Optional[str] = ...) -> None: ... + def __init__( + self, + start: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + end: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + slot: _Optional[str] = ..., + ) -> None: ... class ReduceResponse(_message.Message): __slots__ = ("result", "window", "EOF") + class Result(_message.Message): __slots__ = ("keys", "value", "tags") KEYS_FIELD_NUMBER: _ClassVar[int] @@ -71,14 +102,24 @@ class ReduceResponse(_message.Message): keys: _containers.RepeatedScalarFieldContainer[str] value: bytes tags: _containers.RepeatedScalarFieldContainer[str] - def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., tags: _Optional[_Iterable[str]] = ...) -> None: ... + def __init__( + self, + keys: _Optional[_Iterable[str]] = ..., + value: _Optional[bytes] = ..., + tags: _Optional[_Iterable[str]] = ..., + ) -> None: ... RESULT_FIELD_NUMBER: _ClassVar[int] WINDOW_FIELD_NUMBER: _ClassVar[int] EOF_FIELD_NUMBER: _ClassVar[int] result: ReduceResponse.Result window: Window EOF: bool - def __init__(self, result: _Optional[_Union[ReduceResponse.Result, _Mapping]] = ..., window: _Optional[_Union[Window, _Mapping]] = ..., EOF: bool = ...) -> None: ... + def __init__( + self, + result: _Optional[_Union[ReduceResponse.Result, _Mapping]] = ..., + window: _Optional[_Union[Window, _Mapping]] = ..., + EOF: bool = ..., + ) -> None: ... class ReadyResponse(_message.Message): __slots__ = ("ready",) diff --git a/pynumaflow/proto/reducer/reduce_pb2_grpc.py b/pynumaflow/proto/reducer/reduce_pb2_grpc.py index 1fd860b4..5a0a15f6 100644 --- a/pynumaflow/proto/reducer/reduce_pb2_grpc.py +++ b/pynumaflow/proto/reducer/reduce_pb2_grpc.py @@ -16,87 +16,108 @@ def __init__(self, channel): channel: A grpc.Channel. """ self.ReduceFn = channel.stream_stream( - '/reduce.v1.Reduce/ReduceFn', - request_serializer=reduce__pb2.ReduceRequest.SerializeToString, - response_deserializer=reduce__pb2.ReduceResponse.FromString, - ) + "/reduce.v1.Reduce/ReduceFn", + request_serializer=reduce__pb2.ReduceRequest.SerializeToString, + response_deserializer=reduce__pb2.ReduceResponse.FromString, + ) self.IsReady = channel.unary_unary( - '/reduce.v1.Reduce/IsReady', - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=reduce__pb2.ReadyResponse.FromString, - ) + "/reduce.v1.Reduce/IsReady", + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=reduce__pb2.ReadyResponse.FromString, + ) class ReduceServicer(object): """Missing associated documentation comment in .proto file.""" def ReduceFn(self, request_iterator, context): - """ReduceFn applies a reduce function to a request stream. - """ + """ReduceFn applies a reduce function to a request stream.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def IsReady(self, request, context): - """IsReady is the heartbeat endpoint for gRPC. - """ + """IsReady is the heartbeat endpoint for gRPC.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def add_ReduceServicer_to_server(servicer, server): rpc_method_handlers = { - 'ReduceFn': grpc.stream_stream_rpc_method_handler( - servicer.ReduceFn, - request_deserializer=reduce__pb2.ReduceRequest.FromString, - response_serializer=reduce__pb2.ReduceResponse.SerializeToString, - ), - 'IsReady': grpc.unary_unary_rpc_method_handler( - servicer.IsReady, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=reduce__pb2.ReadyResponse.SerializeToString, - ), + "ReduceFn": grpc.stream_stream_rpc_method_handler( + servicer.ReduceFn, + request_deserializer=reduce__pb2.ReduceRequest.FromString, + response_serializer=reduce__pb2.ReduceResponse.SerializeToString, + ), + "IsReady": grpc.unary_unary_rpc_method_handler( + servicer.IsReady, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=reduce__pb2.ReadyResponse.SerializeToString, + ), } - generic_handler = grpc.method_handlers_generic_handler( - 'reduce.v1.Reduce', rpc_method_handlers) + generic_handler = grpc.method_handlers_generic_handler("reduce.v1.Reduce", rpc_method_handlers) server.add_generic_rpc_handlers((generic_handler,)) - # This class is part of an EXPERIMENTAL API. +# This class is part of an EXPERIMENTAL API. class Reduce(object): """Missing associated documentation comment in .proto file.""" @staticmethod - def ReduceFn(request_iterator, + def ReduceFn( + request_iterator, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.stream_stream( + request_iterator, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.stream_stream(request_iterator, target, '/reduce.v1.Reduce/ReduceFn', + "/reduce.v1.Reduce/ReduceFn", reduce__pb2.ReduceRequest.SerializeToString, reduce__pb2.ReduceResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) @staticmethod - def IsReady(request, + def IsReady( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.unary_unary(request, target, '/reduce.v1.Reduce/IsReady', + "/reduce.v1.Reduce/IsReady", google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, reduce__pb2.ReadyResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) diff --git a/pynumaflow/proto/sideinput/sideinput_pb2.py b/pynumaflow/proto/sideinput/sideinput_pb2.py index 0160315f..c0dab051 100644 --- a/pynumaflow/proto/sideinput/sideinput_pb2.py +++ b/pynumaflow/proto/sideinput/sideinput_pb2.py @@ -7,6 +7,7 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import symbol_database as _symbol_database from google.protobuf.internal import builder as _builder + # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() @@ -15,17 +16,19 @@ from google.protobuf import empty_pb2 as google_dot_protobuf_dot_empty__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x0fsideinput.proto\x12\x0csideinput.v1\x1a\x1bgoogle/protobuf/empty.proto\"8\n\x11SideInputResponse\x12\r\n\x05value\x18\x01 \x01(\x0c\x12\x14\n\x0cno_broadcast\x18\x02 \x01(\x08\"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\x99\x01\n\tSideInput\x12L\n\x11RetrieveSideInput\x12\x16.google.protobuf.Empty\x1a\x1f.sideinput.v1.SideInputResponse\x12>\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x1b.sideinput.v1.ReadyResponseb\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( + b'\n\x0fsideinput.proto\x12\x0csideinput.v1\x1a\x1bgoogle/protobuf/empty.proto"8\n\x11SideInputResponse\x12\r\n\x05value\x18\x01 \x01(\x0c\x12\x14\n\x0cno_broadcast\x18\x02 \x01(\x08"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\x99\x01\n\tSideInput\x12L\n\x11RetrieveSideInput\x12\x16.google.protobuf.Empty\x1a\x1f.sideinput.v1.SideInputResponse\x12>\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x1b.sideinput.v1.ReadyResponseb\x06proto3' +) _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) -_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'sideinput_pb2', _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, "sideinput_pb2", _globals) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None - _globals['_SIDEINPUTRESPONSE']._serialized_start=62 - _globals['_SIDEINPUTRESPONSE']._serialized_end=118 - _globals['_READYRESPONSE']._serialized_start=120 - _globals['_READYRESPONSE']._serialized_end=150 - _globals['_SIDEINPUT']._serialized_start=153 - _globals['_SIDEINPUT']._serialized_end=306 + DESCRIPTOR._options = None + _globals["_SIDEINPUTRESPONSE"]._serialized_start = 62 + _globals["_SIDEINPUTRESPONSE"]._serialized_end = 118 + _globals["_READYRESPONSE"]._serialized_start = 120 + _globals["_READYRESPONSE"]._serialized_end = 150 + _globals["_SIDEINPUT"]._serialized_start = 153 + _globals["_SIDEINPUT"]._serialized_end = 306 # @@protoc_insertion_point(module_scope) diff --git a/pynumaflow/proto/sideinput/sideinput_pb2_grpc.py b/pynumaflow/proto/sideinput/sideinput_pb2_grpc.py index 8abe64d2..72ea87ed 100644 --- a/pynumaflow/proto/sideinput/sideinput_pb2_grpc.py +++ b/pynumaflow/proto/sideinput/sideinput_pb2_grpc.py @@ -24,15 +24,15 @@ def __init__(self, channel): channel: A grpc.Channel. """ self.RetrieveSideInput = channel.unary_unary( - '/sideinput.v1.SideInput/RetrieveSideInput', - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=sideinput__pb2.SideInputResponse.FromString, - ) + "/sideinput.v1.SideInput/RetrieveSideInput", + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=sideinput__pb2.SideInputResponse.FromString, + ) self.IsReady = channel.unary_unary( - '/sideinput.v1.SideInput/IsReady', - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=sideinput__pb2.ReadyResponse.FromString, - ) + "/sideinput.v1.SideInput/IsReady", + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=sideinput__pb2.ReadyResponse.FromString, + ) class SideInputServicer(object): @@ -47,39 +47,38 @@ class SideInputServicer(object): """ def RetrieveSideInput(self, request, context): - """RetrieveSideInput is the endpoint to retrieve the latest value of a given Side Input. - """ + """RetrieveSideInput is the endpoint to retrieve the latest value of a given Side Input.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def IsReady(self, request, context): - """IsReady is the health check endpoint to indicate whether the service is ready to be used. - """ + """IsReady is the health check endpoint to indicate whether the service is ready to be used.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def add_SideInputServicer_to_server(servicer, server): rpc_method_handlers = { - 'RetrieveSideInput': grpc.unary_unary_rpc_method_handler( - servicer.RetrieveSideInput, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=sideinput__pb2.SideInputResponse.SerializeToString, - ), - 'IsReady': grpc.unary_unary_rpc_method_handler( - servicer.IsReady, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=sideinput__pb2.ReadyResponse.SerializeToString, - ), + "RetrieveSideInput": grpc.unary_unary_rpc_method_handler( + servicer.RetrieveSideInput, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=sideinput__pb2.SideInputResponse.SerializeToString, + ), + "IsReady": grpc.unary_unary_rpc_method_handler( + servicer.IsReady, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=sideinput__pb2.ReadyResponse.SerializeToString, + ), } generic_handler = grpc.method_handlers_generic_handler( - 'sideinput.v1.SideInput', rpc_method_handlers) + "sideinput.v1.SideInput", rpc_method_handlers + ) server.add_generic_rpc_handlers((generic_handler,)) - # This class is part of an EXPERIMENTAL API. +# This class is part of an EXPERIMENTAL API. class SideInput(object): """SideInput is the gRPC service for user-defined Side Inputs. It is used to propagate changes in the values of the provided Side Inputs @@ -92,35 +91,59 @@ class SideInput(object): """ @staticmethod - def RetrieveSideInput(request, + def RetrieveSideInput( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.unary_unary(request, target, '/sideinput.v1.SideInput/RetrieveSideInput', + "/sideinput.v1.SideInput/RetrieveSideInput", google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, sideinput__pb2.SideInputResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) @staticmethod - def IsReady(request, + def IsReady( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.unary_unary(request, target, '/sideinput.v1.SideInput/IsReady', + "/sideinput.v1.SideInput/IsReady", google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, sideinput__pb2.ReadyResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) diff --git a/pynumaflow/proto/sinker/sink_pb2.py b/pynumaflow/proto/sinker/sink_pb2.py index 67d905c5..27082a0e 100644 --- a/pynumaflow/proto/sinker/sink_pb2.py +++ b/pynumaflow/proto/sinker/sink_pb2.py @@ -7,6 +7,7 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import symbol_database as _symbol_database from google.protobuf.internal import builder as _builder + # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() @@ -16,33 +17,35 @@ from google.protobuf import timestamp_pb2 as google_dot_protobuf_dot_timestamp__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\nsink.proto\x12\x07sink.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto\"\xa3\x03\n\x0bSinkRequest\x12-\n\x07request\x18\x01 \x01(\x0b\x32\x1c.sink.v1.SinkRequest.Request\x12+\n\x06status\x18\x02 \x01(\x0b\x32\x1b.sink.v1.TransmissionStatus\x12*\n\thandshake\x18\x03 \x01(\x0b\x32\x12.sink.v1.HandshakeH\x00\x88\x01\x01\x1a\xfd\x01\n\x07Request\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\n\n\x02id\x18\x05 \x01(\t\x12:\n\x07headers\x18\x06 \x03(\x0b\x32).sink.v1.SinkRequest.Request.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x42\x0c\n\n_handshake\"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08\"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\"!\n\x12TransmissionStatus\x12\x0b\n\x03\x65ot\x18\x01 \x01(\x08\"\xfc\x01\n\x0cSinkResponse\x12-\n\x07results\x18\x01 \x03(\x0b\x32\x1c.sink.v1.SinkResponse.Result\x12*\n\thandshake\x18\x02 \x01(\x0b\x32\x12.sink.v1.HandshakeH\x00\x88\x01\x01\x12\x30\n\x06status\x18\x03 \x01(\x0b\x32\x1b.sink.v1.TransmissionStatusH\x01\x88\x01\x01\x1a\x46\n\x06Result\x12\n\n\x02id\x18\x01 \x01(\t\x12\x1f\n\x06status\x18\x02 \x01(\x0e\x32\x0f.sink.v1.Status\x12\x0f\n\x07\x65rr_msg\x18\x03 \x01(\tB\x0c\n\n_handshakeB\t\n\x07_status*0\n\x06Status\x12\x0b\n\x07SUCCESS\x10\x00\x12\x0b\n\x07\x46\x41ILURE\x10\x01\x12\x0c\n\x08\x46\x41LLBACK\x10\x02\x32|\n\x04Sink\x12\x39\n\x06SinkFn\x12\x14.sink.v1.SinkRequest\x1a\x15.sink.v1.SinkResponse(\x01\x30\x01\x12\x39\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x16.sink.v1.ReadyResponseb\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( + b'\n\nsink.proto\x12\x07sink.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto"\xa3\x03\n\x0bSinkRequest\x12-\n\x07request\x18\x01 \x01(\x0b\x32\x1c.sink.v1.SinkRequest.Request\x12+\n\x06status\x18\x02 \x01(\x0b\x32\x1b.sink.v1.TransmissionStatus\x12*\n\thandshake\x18\x03 \x01(\x0b\x32\x12.sink.v1.HandshakeH\x00\x88\x01\x01\x1a\xfd\x01\n\x07Request\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\n\n\x02id\x18\x05 \x01(\t\x12:\n\x07headers\x18\x06 \x03(\x0b\x32).sink.v1.SinkRequest.Request.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x42\x0c\n\n_handshake"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08"!\n\x12TransmissionStatus\x12\x0b\n\x03\x65ot\x18\x01 \x01(\x08"\xfc\x01\n\x0cSinkResponse\x12-\n\x07results\x18\x01 \x03(\x0b\x32\x1c.sink.v1.SinkResponse.Result\x12*\n\thandshake\x18\x02 \x01(\x0b\x32\x12.sink.v1.HandshakeH\x00\x88\x01\x01\x12\x30\n\x06status\x18\x03 \x01(\x0b\x32\x1b.sink.v1.TransmissionStatusH\x01\x88\x01\x01\x1a\x46\n\x06Result\x12\n\n\x02id\x18\x01 \x01(\t\x12\x1f\n\x06status\x18\x02 \x01(\x0e\x32\x0f.sink.v1.Status\x12\x0f\n\x07\x65rr_msg\x18\x03 \x01(\tB\x0c\n\n_handshakeB\t\n\x07_status*0\n\x06Status\x12\x0b\n\x07SUCCESS\x10\x00\x12\x0b\n\x07\x46\x41ILURE\x10\x01\x12\x0c\n\x08\x46\x41LLBACK\x10\x02\x32|\n\x04Sink\x12\x39\n\x06SinkFn\x12\x14.sink.v1.SinkRequest\x1a\x15.sink.v1.SinkResponse(\x01\x30\x01\x12\x39\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x16.sink.v1.ReadyResponseb\x06proto3' +) _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) -_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'sink_pb2', _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, "sink_pb2", _globals) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None - _globals['_SINKREQUEST_REQUEST_HEADERSENTRY']._options = None - _globals['_SINKREQUEST_REQUEST_HEADERSENTRY']._serialized_options = b'8\001' - _globals['_STATUS']._serialized_start=855 - _globals['_STATUS']._serialized_end=903 - _globals['_SINKREQUEST']._serialized_start=86 - _globals['_SINKREQUEST']._serialized_end=505 - _globals['_SINKREQUEST_REQUEST']._serialized_start=238 - _globals['_SINKREQUEST_REQUEST']._serialized_end=491 - _globals['_SINKREQUEST_REQUEST_HEADERSENTRY']._serialized_start=445 - _globals['_SINKREQUEST_REQUEST_HEADERSENTRY']._serialized_end=491 - _globals['_HANDSHAKE']._serialized_start=507 - _globals['_HANDSHAKE']._serialized_end=531 - _globals['_READYRESPONSE']._serialized_start=533 - _globals['_READYRESPONSE']._serialized_end=563 - _globals['_TRANSMISSIONSTATUS']._serialized_start=565 - _globals['_TRANSMISSIONSTATUS']._serialized_end=598 - _globals['_SINKRESPONSE']._serialized_start=601 - _globals['_SINKRESPONSE']._serialized_end=853 - _globals['_SINKRESPONSE_RESULT']._serialized_start=758 - _globals['_SINKRESPONSE_RESULT']._serialized_end=828 - _globals['_SINK']._serialized_start=905 - _globals['_SINK']._serialized_end=1029 + DESCRIPTOR._options = None + _globals["_SINKREQUEST_REQUEST_HEADERSENTRY"]._options = None + _globals["_SINKREQUEST_REQUEST_HEADERSENTRY"]._serialized_options = b"8\001" + _globals["_STATUS"]._serialized_start = 855 + _globals["_STATUS"]._serialized_end = 903 + _globals["_SINKREQUEST"]._serialized_start = 86 + _globals["_SINKREQUEST"]._serialized_end = 505 + _globals["_SINKREQUEST_REQUEST"]._serialized_start = 238 + _globals["_SINKREQUEST_REQUEST"]._serialized_end = 491 + _globals["_SINKREQUEST_REQUEST_HEADERSENTRY"]._serialized_start = 445 + _globals["_SINKREQUEST_REQUEST_HEADERSENTRY"]._serialized_end = 491 + _globals["_HANDSHAKE"]._serialized_start = 507 + _globals["_HANDSHAKE"]._serialized_end = 531 + _globals["_READYRESPONSE"]._serialized_start = 533 + _globals["_READYRESPONSE"]._serialized_end = 563 + _globals["_TRANSMISSIONSTATUS"]._serialized_start = 565 + _globals["_TRANSMISSIONSTATUS"]._serialized_end = 598 + _globals["_SINKRESPONSE"]._serialized_start = 601 + _globals["_SINKRESPONSE"]._serialized_end = 853 + _globals["_SINKRESPONSE_RESULT"]._serialized_start = 758 + _globals["_SINKRESPONSE_RESULT"]._serialized_end = 828 + _globals["_SINK"]._serialized_start = 905 + _globals["_SINK"]._serialized_end = 1029 # @@protoc_insertion_point(module_scope) diff --git a/pynumaflow/proto/sinker/sink_pb2.pyi b/pynumaflow/proto/sinker/sink_pb2.pyi index 6c0292ca..78926321 100644 --- a/pynumaflow/proto/sinker/sink_pb2.pyi +++ b/pynumaflow/proto/sinker/sink_pb2.pyi @@ -4,7 +4,13 @@ from google.protobuf.internal import containers as _containers from google.protobuf.internal import enum_type_wrapper as _enum_type_wrapper from google.protobuf import descriptor as _descriptor from google.protobuf import message as _message -from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union +from typing import ( + ClassVar as _ClassVar, + Iterable as _Iterable, + Mapping as _Mapping, + Optional as _Optional, + Union as _Union, +) DESCRIPTOR: _descriptor.FileDescriptor @@ -13,14 +19,17 @@ class Status(int, metaclass=_enum_type_wrapper.EnumTypeWrapper): SUCCESS: _ClassVar[Status] FAILURE: _ClassVar[Status] FALLBACK: _ClassVar[Status] + SUCCESS: Status FAILURE: Status FALLBACK: Status class SinkRequest(_message.Message): __slots__ = ("request", "status", "handshake") + class Request(_message.Message): __slots__ = ("keys", "value", "event_time", "watermark", "id", "headers") + class HeadersEntry(_message.Message): __slots__ = ("key", "value") KEY_FIELD_NUMBER: _ClassVar[int] @@ -40,14 +49,27 @@ class SinkRequest(_message.Message): watermark: _timestamp_pb2.Timestamp id: str headers: _containers.ScalarMap[str, str] - def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., id: _Optional[str] = ..., headers: _Optional[_Mapping[str, str]] = ...) -> None: ... + def __init__( + self, + keys: _Optional[_Iterable[str]] = ..., + value: _Optional[bytes] = ..., + event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + id: _Optional[str] = ..., + headers: _Optional[_Mapping[str, str]] = ..., + ) -> None: ... REQUEST_FIELD_NUMBER: _ClassVar[int] STATUS_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] request: SinkRequest.Request status: TransmissionStatus handshake: Handshake - def __init__(self, request: _Optional[_Union[SinkRequest.Request, _Mapping]] = ..., status: _Optional[_Union[TransmissionStatus, _Mapping]] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ...) -> None: ... + def __init__( + self, + request: _Optional[_Union[SinkRequest.Request, _Mapping]] = ..., + status: _Optional[_Union[TransmissionStatus, _Mapping]] = ..., + handshake: _Optional[_Union[Handshake, _Mapping]] = ..., + ) -> None: ... class Handshake(_message.Message): __slots__ = ("sot",) @@ -69,6 +91,7 @@ class TransmissionStatus(_message.Message): class SinkResponse(_message.Message): __slots__ = ("results", "handshake", "status") + class Result(_message.Message): __slots__ = ("id", "status", "err_msg") ID_FIELD_NUMBER: _ClassVar[int] @@ -77,11 +100,21 @@ class SinkResponse(_message.Message): id: str status: Status err_msg: str - def __init__(self, id: _Optional[str] = ..., status: _Optional[_Union[Status, str]] = ..., err_msg: _Optional[str] = ...) -> None: ... + def __init__( + self, + id: _Optional[str] = ..., + status: _Optional[_Union[Status, str]] = ..., + err_msg: _Optional[str] = ..., + ) -> None: ... RESULTS_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] STATUS_FIELD_NUMBER: _ClassVar[int] results: _containers.RepeatedCompositeFieldContainer[SinkResponse.Result] handshake: Handshake status: TransmissionStatus - def __init__(self, results: _Optional[_Iterable[_Union[SinkResponse.Result, _Mapping]]] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ..., status: _Optional[_Union[TransmissionStatus, _Mapping]] = ...) -> None: ... + def __init__( + self, + results: _Optional[_Iterable[_Union[SinkResponse.Result, _Mapping]]] = ..., + handshake: _Optional[_Union[Handshake, _Mapping]] = ..., + status: _Optional[_Union[TransmissionStatus, _Mapping]] = ..., + ) -> None: ... diff --git a/pynumaflow/proto/sinker/sink_pb2_grpc.py b/pynumaflow/proto/sinker/sink_pb2_grpc.py index 4f53a34e..9609c76e 100644 --- a/pynumaflow/proto/sinker/sink_pb2_grpc.py +++ b/pynumaflow/proto/sinker/sink_pb2_grpc.py @@ -16,87 +16,108 @@ def __init__(self, channel): channel: A grpc.Channel. """ self.SinkFn = channel.stream_stream( - '/sink.v1.Sink/SinkFn', - request_serializer=sink__pb2.SinkRequest.SerializeToString, - response_deserializer=sink__pb2.SinkResponse.FromString, - ) + "/sink.v1.Sink/SinkFn", + request_serializer=sink__pb2.SinkRequest.SerializeToString, + response_deserializer=sink__pb2.SinkResponse.FromString, + ) self.IsReady = channel.unary_unary( - '/sink.v1.Sink/IsReady', - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=sink__pb2.ReadyResponse.FromString, - ) + "/sink.v1.Sink/IsReady", + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=sink__pb2.ReadyResponse.FromString, + ) class SinkServicer(object): """Missing associated documentation comment in .proto file.""" def SinkFn(self, request_iterator, context): - """SinkFn writes the request to a user defined sink. - """ + """SinkFn writes the request to a user defined sink.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def IsReady(self, request, context): - """IsReady is the heartbeat endpoint for gRPC. - """ + """IsReady is the heartbeat endpoint for gRPC.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def add_SinkServicer_to_server(servicer, server): rpc_method_handlers = { - 'SinkFn': grpc.stream_stream_rpc_method_handler( - servicer.SinkFn, - request_deserializer=sink__pb2.SinkRequest.FromString, - response_serializer=sink__pb2.SinkResponse.SerializeToString, - ), - 'IsReady': grpc.unary_unary_rpc_method_handler( - servicer.IsReady, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=sink__pb2.ReadyResponse.SerializeToString, - ), + "SinkFn": grpc.stream_stream_rpc_method_handler( + servicer.SinkFn, + request_deserializer=sink__pb2.SinkRequest.FromString, + response_serializer=sink__pb2.SinkResponse.SerializeToString, + ), + "IsReady": grpc.unary_unary_rpc_method_handler( + servicer.IsReady, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=sink__pb2.ReadyResponse.SerializeToString, + ), } - generic_handler = grpc.method_handlers_generic_handler( - 'sink.v1.Sink', rpc_method_handlers) + generic_handler = grpc.method_handlers_generic_handler("sink.v1.Sink", rpc_method_handlers) server.add_generic_rpc_handlers((generic_handler,)) - # This class is part of an EXPERIMENTAL API. +# This class is part of an EXPERIMENTAL API. class Sink(object): """Missing associated documentation comment in .proto file.""" @staticmethod - def SinkFn(request_iterator, + def SinkFn( + request_iterator, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.stream_stream( + request_iterator, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.stream_stream(request_iterator, target, '/sink.v1.Sink/SinkFn', + "/sink.v1.Sink/SinkFn", sink__pb2.SinkRequest.SerializeToString, sink__pb2.SinkResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) @staticmethod - def IsReady(request, + def IsReady( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.unary_unary(request, target, '/sink.v1.Sink/IsReady', + "/sink.v1.Sink/IsReady", google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, sink__pb2.ReadyResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) diff --git a/pynumaflow/proto/sourcer/source_pb2.py b/pynumaflow/proto/sourcer/source_pb2.py index e8d62515..6cd6f0b2 100644 --- a/pynumaflow/proto/sourcer/source_pb2.py +++ b/pynumaflow/proto/sourcer/source_pb2.py @@ -7,6 +7,7 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import symbol_database as _symbol_database from google.protobuf.internal import builder as _builder + # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() @@ -16,53 +17,55 @@ from google.protobuf import empty_pb2 as google_dot_protobuf_dot_empty__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x0csource.proto\x12\tsource.v1\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1bgoogle/protobuf/empty.proto\"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08\"\xb1\x01\n\x0bReadRequest\x12/\n\x07request\x18\x01 \x01(\x0b\x32\x1e.source.v1.ReadRequest.Request\x12,\n\thandshake\x18\x02 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a\x35\n\x07Request\x12\x13\n\x0bnum_records\x18\x01 \x01(\x04\x12\x15\n\rtimeout_in_ms\x18\x02 \x01(\rB\x0c\n\n_handshake\"\x81\x05\n\x0cReadResponse\x12.\n\x06result\x18\x01 \x01(\x0b\x32\x1e.source.v1.ReadResponse.Result\x12.\n\x06status\x18\x02 \x01(\x0b\x32\x1e.source.v1.ReadResponse.Status\x12,\n\thandshake\x18\x03 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a\xe8\x01\n\x06Result\x12\x0f\n\x07payload\x18\x01 \x01(\x0c\x12!\n\x06offset\x18\x02 \x01(\x0b\x32\x11.source.v1.Offset\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04keys\x18\x04 \x03(\t\x12<\n\x07headers\x18\x05 \x03(\x0b\x32+.source.v1.ReadResponse.Result.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x1a\xe9\x01\n\x06Status\x12\x0b\n\x03\x65ot\x18\x01 \x01(\x08\x12\x31\n\x04\x63ode\x18\x02 \x01(\x0e\x32#.source.v1.ReadResponse.Status.Code\x12\x38\n\x05\x65rror\x18\x03 \x01(\x0e\x32$.source.v1.ReadResponse.Status.ErrorH\x00\x88\x01\x01\x12\x10\n\x03msg\x18\x04 \x01(\tH\x01\x88\x01\x01\" \n\x04\x43ode\x12\x0b\n\x07SUCCESS\x10\x00\x12\x0b\n\x07\x46\x41ILURE\x10\x01\"\x1f\n\x05\x45rror\x12\x0b\n\x07UNACKED\x10\x00\x12\t\n\x05OTHER\x10\x01\x42\x08\n\x06_errorB\x06\n\x04_msgB\x0c\n\n_handshake\"\xa7\x01\n\nAckRequest\x12.\n\x07request\x18\x01 \x01(\x0b\x32\x1d.source.v1.AckRequest.Request\x12,\n\thandshake\x18\x02 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a-\n\x07Request\x12\"\n\x07offsets\x18\x01 \x03(\x0b\x32\x11.source.v1.OffsetB\x0c\n\n_handshake\"\xab\x01\n\x0b\x41\x63kResponse\x12-\n\x06result\x18\x01 \x01(\x0b\x32\x1d.source.v1.AckResponse.Result\x12,\n\thandshake\x18\x02 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a\x31\n\x06Result\x12\'\n\x07success\x18\x01 \x01(\x0b\x32\x16.google.protobuf.EmptyB\x0c\n\n_handshake\"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\"]\n\x0fPendingResponse\x12\x31\n\x06result\x18\x01 \x01(\x0b\x32!.source.v1.PendingResponse.Result\x1a\x17\n\x06Result\x12\r\n\x05\x63ount\x18\x01 \x01(\x03\"h\n\x12PartitionsResponse\x12\x34\n\x06result\x18\x01 \x01(\x0b\x32$.source.v1.PartitionsResponse.Result\x1a\x1c\n\x06Result\x12\x12\n\npartitions\x18\x01 \x03(\x05\".\n\x06Offset\x12\x0e\n\x06offset\x18\x01 \x01(\x0c\x12\x14\n\x0cpartition_id\x18\x02 \x01(\x05\x32\xc8\x02\n\x06Source\x12=\n\x06ReadFn\x12\x16.source.v1.ReadRequest\x1a\x17.source.v1.ReadResponse(\x01\x30\x01\x12:\n\x05\x41\x63kFn\x12\x15.source.v1.AckRequest\x1a\x16.source.v1.AckResponse(\x01\x30\x01\x12?\n\tPendingFn\x12\x16.google.protobuf.Empty\x1a\x1a.source.v1.PendingResponse\x12\x45\n\x0cPartitionsFn\x12\x16.google.protobuf.Empty\x1a\x1d.source.v1.PartitionsResponse\x12;\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x18.source.v1.ReadyResponseb\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( + b'\n\x0csource.proto\x12\tsource.v1\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1bgoogle/protobuf/empty.proto"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08"\xb1\x01\n\x0bReadRequest\x12/\n\x07request\x18\x01 \x01(\x0b\x32\x1e.source.v1.ReadRequest.Request\x12,\n\thandshake\x18\x02 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a\x35\n\x07Request\x12\x13\n\x0bnum_records\x18\x01 \x01(\x04\x12\x15\n\rtimeout_in_ms\x18\x02 \x01(\rB\x0c\n\n_handshake"\x81\x05\n\x0cReadResponse\x12.\n\x06result\x18\x01 \x01(\x0b\x32\x1e.source.v1.ReadResponse.Result\x12.\n\x06status\x18\x02 \x01(\x0b\x32\x1e.source.v1.ReadResponse.Status\x12,\n\thandshake\x18\x03 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a\xe8\x01\n\x06Result\x12\x0f\n\x07payload\x18\x01 \x01(\x0c\x12!\n\x06offset\x18\x02 \x01(\x0b\x32\x11.source.v1.Offset\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04keys\x18\x04 \x03(\t\x12<\n\x07headers\x18\x05 \x03(\x0b\x32+.source.v1.ReadResponse.Result.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x1a\xe9\x01\n\x06Status\x12\x0b\n\x03\x65ot\x18\x01 \x01(\x08\x12\x31\n\x04\x63ode\x18\x02 \x01(\x0e\x32#.source.v1.ReadResponse.Status.Code\x12\x38\n\x05\x65rror\x18\x03 \x01(\x0e\x32$.source.v1.ReadResponse.Status.ErrorH\x00\x88\x01\x01\x12\x10\n\x03msg\x18\x04 \x01(\tH\x01\x88\x01\x01" \n\x04\x43ode\x12\x0b\n\x07SUCCESS\x10\x00\x12\x0b\n\x07\x46\x41ILURE\x10\x01"\x1f\n\x05\x45rror\x12\x0b\n\x07UNACKED\x10\x00\x12\t\n\x05OTHER\x10\x01\x42\x08\n\x06_errorB\x06\n\x04_msgB\x0c\n\n_handshake"\xa7\x01\n\nAckRequest\x12.\n\x07request\x18\x01 \x01(\x0b\x32\x1d.source.v1.AckRequest.Request\x12,\n\thandshake\x18\x02 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a-\n\x07Request\x12"\n\x07offsets\x18\x01 \x03(\x0b\x32\x11.source.v1.OffsetB\x0c\n\n_handshake"\xab\x01\n\x0b\x41\x63kResponse\x12-\n\x06result\x18\x01 \x01(\x0b\x32\x1d.source.v1.AckResponse.Result\x12,\n\thandshake\x18\x02 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a\x31\n\x06Result\x12\'\n\x07success\x18\x01 \x01(\x0b\x32\x16.google.protobuf.EmptyB\x0c\n\n_handshake"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08"]\n\x0fPendingResponse\x12\x31\n\x06result\x18\x01 \x01(\x0b\x32!.source.v1.PendingResponse.Result\x1a\x17\n\x06Result\x12\r\n\x05\x63ount\x18\x01 \x01(\x03"h\n\x12PartitionsResponse\x12\x34\n\x06result\x18\x01 \x01(\x0b\x32$.source.v1.PartitionsResponse.Result\x1a\x1c\n\x06Result\x12\x12\n\npartitions\x18\x01 \x03(\x05".\n\x06Offset\x12\x0e\n\x06offset\x18\x01 \x01(\x0c\x12\x14\n\x0cpartition_id\x18\x02 \x01(\x05\x32\xc8\x02\n\x06Source\x12=\n\x06ReadFn\x12\x16.source.v1.ReadRequest\x1a\x17.source.v1.ReadResponse(\x01\x30\x01\x12:\n\x05\x41\x63kFn\x12\x15.source.v1.AckRequest\x1a\x16.source.v1.AckResponse(\x01\x30\x01\x12?\n\tPendingFn\x12\x16.google.protobuf.Empty\x1a\x1a.source.v1.PendingResponse\x12\x45\n\x0cPartitionsFn\x12\x16.google.protobuf.Empty\x1a\x1d.source.v1.PartitionsResponse\x12;\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x18.source.v1.ReadyResponseb\x06proto3' +) _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) -_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'source_pb2', _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, "source_pb2", _globals) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None - _globals['_READRESPONSE_RESULT_HEADERSENTRY']._options = None - _globals['_READRESPONSE_RESULT_HEADERSENTRY']._serialized_options = b'8\001' - _globals['_HANDSHAKE']._serialized_start=89 - _globals['_HANDSHAKE']._serialized_end=113 - _globals['_READREQUEST']._serialized_start=116 - _globals['_READREQUEST']._serialized_end=293 - _globals['_READREQUEST_REQUEST']._serialized_start=226 - _globals['_READREQUEST_REQUEST']._serialized_end=279 - _globals['_READRESPONSE']._serialized_start=296 - _globals['_READRESPONSE']._serialized_end=937 - _globals['_READRESPONSE_RESULT']._serialized_start=455 - _globals['_READRESPONSE_RESULT']._serialized_end=687 - _globals['_READRESPONSE_RESULT_HEADERSENTRY']._serialized_start=641 - _globals['_READRESPONSE_RESULT_HEADERSENTRY']._serialized_end=687 - _globals['_READRESPONSE_STATUS']._serialized_start=690 - _globals['_READRESPONSE_STATUS']._serialized_end=923 - _globals['_READRESPONSE_STATUS_CODE']._serialized_start=840 - _globals['_READRESPONSE_STATUS_CODE']._serialized_end=872 - _globals['_READRESPONSE_STATUS_ERROR']._serialized_start=874 - _globals['_READRESPONSE_STATUS_ERROR']._serialized_end=905 - _globals['_ACKREQUEST']._serialized_start=940 - _globals['_ACKREQUEST']._serialized_end=1107 - _globals['_ACKREQUEST_REQUEST']._serialized_start=1048 - _globals['_ACKREQUEST_REQUEST']._serialized_end=1093 - _globals['_ACKRESPONSE']._serialized_start=1110 - _globals['_ACKRESPONSE']._serialized_end=1281 - _globals['_ACKRESPONSE_RESULT']._serialized_start=1218 - _globals['_ACKRESPONSE_RESULT']._serialized_end=1267 - _globals['_READYRESPONSE']._serialized_start=1283 - _globals['_READYRESPONSE']._serialized_end=1313 - _globals['_PENDINGRESPONSE']._serialized_start=1315 - _globals['_PENDINGRESPONSE']._serialized_end=1408 - _globals['_PENDINGRESPONSE_RESULT']._serialized_start=1385 - _globals['_PENDINGRESPONSE_RESULT']._serialized_end=1408 - _globals['_PARTITIONSRESPONSE']._serialized_start=1410 - _globals['_PARTITIONSRESPONSE']._serialized_end=1514 - _globals['_PARTITIONSRESPONSE_RESULT']._serialized_start=1486 - _globals['_PARTITIONSRESPONSE_RESULT']._serialized_end=1514 - _globals['_OFFSET']._serialized_start=1516 - _globals['_OFFSET']._serialized_end=1562 - _globals['_SOURCE']._serialized_start=1565 - _globals['_SOURCE']._serialized_end=1893 + DESCRIPTOR._options = None + _globals["_READRESPONSE_RESULT_HEADERSENTRY"]._options = None + _globals["_READRESPONSE_RESULT_HEADERSENTRY"]._serialized_options = b"8\001" + _globals["_HANDSHAKE"]._serialized_start = 89 + _globals["_HANDSHAKE"]._serialized_end = 113 + _globals["_READREQUEST"]._serialized_start = 116 + _globals["_READREQUEST"]._serialized_end = 293 + _globals["_READREQUEST_REQUEST"]._serialized_start = 226 + _globals["_READREQUEST_REQUEST"]._serialized_end = 279 + _globals["_READRESPONSE"]._serialized_start = 296 + _globals["_READRESPONSE"]._serialized_end = 937 + _globals["_READRESPONSE_RESULT"]._serialized_start = 455 + _globals["_READRESPONSE_RESULT"]._serialized_end = 687 + _globals["_READRESPONSE_RESULT_HEADERSENTRY"]._serialized_start = 641 + _globals["_READRESPONSE_RESULT_HEADERSENTRY"]._serialized_end = 687 + _globals["_READRESPONSE_STATUS"]._serialized_start = 690 + _globals["_READRESPONSE_STATUS"]._serialized_end = 923 + _globals["_READRESPONSE_STATUS_CODE"]._serialized_start = 840 + _globals["_READRESPONSE_STATUS_CODE"]._serialized_end = 872 + _globals["_READRESPONSE_STATUS_ERROR"]._serialized_start = 874 + _globals["_READRESPONSE_STATUS_ERROR"]._serialized_end = 905 + _globals["_ACKREQUEST"]._serialized_start = 940 + _globals["_ACKREQUEST"]._serialized_end = 1107 + _globals["_ACKREQUEST_REQUEST"]._serialized_start = 1048 + _globals["_ACKREQUEST_REQUEST"]._serialized_end = 1093 + _globals["_ACKRESPONSE"]._serialized_start = 1110 + _globals["_ACKRESPONSE"]._serialized_end = 1281 + _globals["_ACKRESPONSE_RESULT"]._serialized_start = 1218 + _globals["_ACKRESPONSE_RESULT"]._serialized_end = 1267 + _globals["_READYRESPONSE"]._serialized_start = 1283 + _globals["_READYRESPONSE"]._serialized_end = 1313 + _globals["_PENDINGRESPONSE"]._serialized_start = 1315 + _globals["_PENDINGRESPONSE"]._serialized_end = 1408 + _globals["_PENDINGRESPONSE_RESULT"]._serialized_start = 1385 + _globals["_PENDINGRESPONSE_RESULT"]._serialized_end = 1408 + _globals["_PARTITIONSRESPONSE"]._serialized_start = 1410 + _globals["_PARTITIONSRESPONSE"]._serialized_end = 1514 + _globals["_PARTITIONSRESPONSE_RESULT"]._serialized_start = 1486 + _globals["_PARTITIONSRESPONSE_RESULT"]._serialized_end = 1514 + _globals["_OFFSET"]._serialized_start = 1516 + _globals["_OFFSET"]._serialized_end = 1562 + _globals["_SOURCE"]._serialized_start = 1565 + _globals["_SOURCE"]._serialized_end = 1893 # @@protoc_insertion_point(module_scope) diff --git a/pynumaflow/proto/sourcer/source_pb2.pyi b/pynumaflow/proto/sourcer/source_pb2.pyi index 52c20928..f2cdc70e 100644 --- a/pynumaflow/proto/sourcer/source_pb2.pyi +++ b/pynumaflow/proto/sourcer/source_pb2.pyi @@ -4,7 +4,13 @@ from google.protobuf.internal import containers as _containers from google.protobuf.internal import enum_type_wrapper as _enum_type_wrapper from google.protobuf import descriptor as _descriptor from google.protobuf import message as _message -from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union +from typing import ( + ClassVar as _ClassVar, + Iterable as _Iterable, + Mapping as _Mapping, + Optional as _Optional, + Union as _Union, +) DESCRIPTOR: _descriptor.FileDescriptor @@ -16,23 +22,32 @@ class Handshake(_message.Message): class ReadRequest(_message.Message): __slots__ = ("request", "handshake") + class Request(_message.Message): __slots__ = ("num_records", "timeout_in_ms") NUM_RECORDS_FIELD_NUMBER: _ClassVar[int] TIMEOUT_IN_MS_FIELD_NUMBER: _ClassVar[int] num_records: int timeout_in_ms: int - def __init__(self, num_records: _Optional[int] = ..., timeout_in_ms: _Optional[int] = ...) -> None: ... + def __init__( + self, num_records: _Optional[int] = ..., timeout_in_ms: _Optional[int] = ... + ) -> None: ... REQUEST_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] request: ReadRequest.Request handshake: Handshake - def __init__(self, request: _Optional[_Union[ReadRequest.Request, _Mapping]] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ...) -> None: ... + def __init__( + self, + request: _Optional[_Union[ReadRequest.Request, _Mapping]] = ..., + handshake: _Optional[_Union[Handshake, _Mapping]] = ..., + ) -> None: ... class ReadResponse(_message.Message): __slots__ = ("result", "status", "handshake") + class Result(_message.Message): __slots__ = ("payload", "offset", "event_time", "keys", "headers") + class HeadersEntry(_message.Message): __slots__ = ("key", "value") KEY_FIELD_NUMBER: _ClassVar[int] @@ -50,15 +65,25 @@ class ReadResponse(_message.Message): event_time: _timestamp_pb2.Timestamp keys: _containers.RepeatedScalarFieldContainer[str] headers: _containers.ScalarMap[str, str] - def __init__(self, payload: _Optional[bytes] = ..., offset: _Optional[_Union[Offset, _Mapping]] = ..., event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., keys: _Optional[_Iterable[str]] = ..., headers: _Optional[_Mapping[str, str]] = ...) -> None: ... + def __init__( + self, + payload: _Optional[bytes] = ..., + offset: _Optional[_Union[Offset, _Mapping]] = ..., + event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + keys: _Optional[_Iterable[str]] = ..., + headers: _Optional[_Mapping[str, str]] = ..., + ) -> None: ... + class Status(_message.Message): __slots__ = ("eot", "code", "error", "msg") + class Code(int, metaclass=_enum_type_wrapper.EnumTypeWrapper): __slots__ = () SUCCESS: _ClassVar[ReadResponse.Status.Code] FAILURE: _ClassVar[ReadResponse.Status.Code] SUCCESS: ReadResponse.Status.Code FAILURE: ReadResponse.Status.Code + class Error(int, metaclass=_enum_type_wrapper.EnumTypeWrapper): __slots__ = () UNACKED: _ClassVar[ReadResponse.Status.Error] @@ -73,40 +98,65 @@ class ReadResponse(_message.Message): code: ReadResponse.Status.Code error: ReadResponse.Status.Error msg: str - def __init__(self, eot: bool = ..., code: _Optional[_Union[ReadResponse.Status.Code, str]] = ..., error: _Optional[_Union[ReadResponse.Status.Error, str]] = ..., msg: _Optional[str] = ...) -> None: ... + def __init__( + self, + eot: bool = ..., + code: _Optional[_Union[ReadResponse.Status.Code, str]] = ..., + error: _Optional[_Union[ReadResponse.Status.Error, str]] = ..., + msg: _Optional[str] = ..., + ) -> None: ... RESULT_FIELD_NUMBER: _ClassVar[int] STATUS_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] result: ReadResponse.Result status: ReadResponse.Status handshake: Handshake - def __init__(self, result: _Optional[_Union[ReadResponse.Result, _Mapping]] = ..., status: _Optional[_Union[ReadResponse.Status, _Mapping]] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ...) -> None: ... + def __init__( + self, + result: _Optional[_Union[ReadResponse.Result, _Mapping]] = ..., + status: _Optional[_Union[ReadResponse.Status, _Mapping]] = ..., + handshake: _Optional[_Union[Handshake, _Mapping]] = ..., + ) -> None: ... class AckRequest(_message.Message): __slots__ = ("request", "handshake") + class Request(_message.Message): __slots__ = ("offsets",) OFFSETS_FIELD_NUMBER: _ClassVar[int] offsets: _containers.RepeatedCompositeFieldContainer[Offset] - def __init__(self, offsets: _Optional[_Iterable[_Union[Offset, _Mapping]]] = ...) -> None: ... + def __init__( + self, offsets: _Optional[_Iterable[_Union[Offset, _Mapping]]] = ... + ) -> None: ... REQUEST_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] request: AckRequest.Request handshake: Handshake - def __init__(self, request: _Optional[_Union[AckRequest.Request, _Mapping]] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ...) -> None: ... + def __init__( + self, + request: _Optional[_Union[AckRequest.Request, _Mapping]] = ..., + handshake: _Optional[_Union[Handshake, _Mapping]] = ..., + ) -> None: ... class AckResponse(_message.Message): __slots__ = ("result", "handshake") + class Result(_message.Message): __slots__ = ("success",) SUCCESS_FIELD_NUMBER: _ClassVar[int] success: _empty_pb2.Empty - def __init__(self, success: _Optional[_Union[_empty_pb2.Empty, _Mapping]] = ...) -> None: ... + def __init__( + self, success: _Optional[_Union[_empty_pb2.Empty, _Mapping]] = ... + ) -> None: ... RESULT_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] result: AckResponse.Result handshake: Handshake - def __init__(self, result: _Optional[_Union[AckResponse.Result, _Mapping]] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ...) -> None: ... + def __init__( + self, + result: _Optional[_Union[AckResponse.Result, _Mapping]] = ..., + handshake: _Optional[_Union[Handshake, _Mapping]] = ..., + ) -> None: ... class ReadyResponse(_message.Message): __slots__ = ("ready",) @@ -116,6 +166,7 @@ class ReadyResponse(_message.Message): class PendingResponse(_message.Message): __slots__ = ("result",) + class Result(_message.Message): __slots__ = ("count",) COUNT_FIELD_NUMBER: _ClassVar[int] @@ -123,10 +174,13 @@ class PendingResponse(_message.Message): def __init__(self, count: _Optional[int] = ...) -> None: ... RESULT_FIELD_NUMBER: _ClassVar[int] result: PendingResponse.Result - def __init__(self, result: _Optional[_Union[PendingResponse.Result, _Mapping]] = ...) -> None: ... + def __init__( + self, result: _Optional[_Union[PendingResponse.Result, _Mapping]] = ... + ) -> None: ... class PartitionsResponse(_message.Message): __slots__ = ("result",) + class Result(_message.Message): __slots__ = ("partitions",) PARTITIONS_FIELD_NUMBER: _ClassVar[int] @@ -134,7 +188,9 @@ class PartitionsResponse(_message.Message): def __init__(self, partitions: _Optional[_Iterable[int]] = ...) -> None: ... RESULT_FIELD_NUMBER: _ClassVar[int] result: PartitionsResponse.Result - def __init__(self, result: _Optional[_Union[PartitionsResponse.Result, _Mapping]] = ...) -> None: ... + def __init__( + self, result: _Optional[_Union[PartitionsResponse.Result, _Mapping]] = ... + ) -> None: ... class Offset(_message.Message): __slots__ = ("offset", "partition_id") @@ -142,4 +198,6 @@ class Offset(_message.Message): PARTITION_ID_FIELD_NUMBER: _ClassVar[int] offset: bytes partition_id: int - def __init__(self, offset: _Optional[bytes] = ..., partition_id: _Optional[int] = ...) -> None: ... + def __init__( + self, offset: _Optional[bytes] = ..., partition_id: _Optional[int] = ... + ) -> None: ... diff --git a/pynumaflow/proto/sourcer/source_pb2_grpc.py b/pynumaflow/proto/sourcer/source_pb2_grpc.py index 14ad1d5a..d5e09f1b 100644 --- a/pynumaflow/proto/sourcer/source_pb2_grpc.py +++ b/pynumaflow/proto/sourcer/source_pb2_grpc.py @@ -16,30 +16,30 @@ def __init__(self, channel): channel: A grpc.Channel. """ self.ReadFn = channel.stream_stream( - '/source.v1.Source/ReadFn', - request_serializer=source__pb2.ReadRequest.SerializeToString, - response_deserializer=source__pb2.ReadResponse.FromString, - ) + "/source.v1.Source/ReadFn", + request_serializer=source__pb2.ReadRequest.SerializeToString, + response_deserializer=source__pb2.ReadResponse.FromString, + ) self.AckFn = channel.stream_stream( - '/source.v1.Source/AckFn', - request_serializer=source__pb2.AckRequest.SerializeToString, - response_deserializer=source__pb2.AckResponse.FromString, - ) + "/source.v1.Source/AckFn", + request_serializer=source__pb2.AckRequest.SerializeToString, + response_deserializer=source__pb2.AckResponse.FromString, + ) self.PendingFn = channel.unary_unary( - '/source.v1.Source/PendingFn', - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=source__pb2.PendingResponse.FromString, - ) + "/source.v1.Source/PendingFn", + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=source__pb2.PendingResponse.FromString, + ) self.PartitionsFn = channel.unary_unary( - '/source.v1.Source/PartitionsFn', - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=source__pb2.PartitionsResponse.FromString, - ) + "/source.v1.Source/PartitionsFn", + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=source__pb2.PartitionsResponse.FromString, + ) self.IsReady = channel.unary_unary( - '/source.v1.Source/IsReady', - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=source__pb2.ReadyResponse.FromString, - ) + "/source.v1.Source/IsReady", + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=source__pb2.ReadyResponse.FromString, + ) class SourceServicer(object): @@ -53,8 +53,8 @@ def ReadFn(self, request_iterator, context): Once it has sent all the datum, the server will send a ReadResponse with the end of transmission flag set to true. """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def AckFn(self, request_iterator, context): """AckFn acknowledges a stream of datum offsets. @@ -65,149 +65,205 @@ def AckFn(self, request_iterator, context): Clients sends n requests and expects n responses. """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def PendingFn(self, request, context): - """PendingFn returns the number of pending records at the user defined source. - """ + """PendingFn returns the number of pending records at the user defined source.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def PartitionsFn(self, request, context): - """PartitionsFn returns the list of partitions for the user defined source. - """ + """PartitionsFn returns the list of partitions for the user defined source.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def IsReady(self, request, context): - """IsReady is the heartbeat endpoint for user defined source gRPC. - """ + """IsReady is the heartbeat endpoint for user defined source gRPC.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def add_SourceServicer_to_server(servicer, server): rpc_method_handlers = { - 'ReadFn': grpc.stream_stream_rpc_method_handler( - servicer.ReadFn, - request_deserializer=source__pb2.ReadRequest.FromString, - response_serializer=source__pb2.ReadResponse.SerializeToString, - ), - 'AckFn': grpc.stream_stream_rpc_method_handler( - servicer.AckFn, - request_deserializer=source__pb2.AckRequest.FromString, - response_serializer=source__pb2.AckResponse.SerializeToString, - ), - 'PendingFn': grpc.unary_unary_rpc_method_handler( - servicer.PendingFn, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=source__pb2.PendingResponse.SerializeToString, - ), - 'PartitionsFn': grpc.unary_unary_rpc_method_handler( - servicer.PartitionsFn, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=source__pb2.PartitionsResponse.SerializeToString, - ), - 'IsReady': grpc.unary_unary_rpc_method_handler( - servicer.IsReady, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=source__pb2.ReadyResponse.SerializeToString, - ), + "ReadFn": grpc.stream_stream_rpc_method_handler( + servicer.ReadFn, + request_deserializer=source__pb2.ReadRequest.FromString, + response_serializer=source__pb2.ReadResponse.SerializeToString, + ), + "AckFn": grpc.stream_stream_rpc_method_handler( + servicer.AckFn, + request_deserializer=source__pb2.AckRequest.FromString, + response_serializer=source__pb2.AckResponse.SerializeToString, + ), + "PendingFn": grpc.unary_unary_rpc_method_handler( + servicer.PendingFn, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=source__pb2.PendingResponse.SerializeToString, + ), + "PartitionsFn": grpc.unary_unary_rpc_method_handler( + servicer.PartitionsFn, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=source__pb2.PartitionsResponse.SerializeToString, + ), + "IsReady": grpc.unary_unary_rpc_method_handler( + servicer.IsReady, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=source__pb2.ReadyResponse.SerializeToString, + ), } - generic_handler = grpc.method_handlers_generic_handler( - 'source.v1.Source', rpc_method_handlers) + generic_handler = grpc.method_handlers_generic_handler("source.v1.Source", rpc_method_handlers) server.add_generic_rpc_handlers((generic_handler,)) - # This class is part of an EXPERIMENTAL API. +# This class is part of an EXPERIMENTAL API. class Source(object): """Missing associated documentation comment in .proto file.""" @staticmethod - def ReadFn(request_iterator, + def ReadFn( + request_iterator, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.stream_stream( + request_iterator, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.stream_stream(request_iterator, target, '/source.v1.Source/ReadFn', + "/source.v1.Source/ReadFn", source__pb2.ReadRequest.SerializeToString, source__pb2.ReadResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) @staticmethod - def AckFn(request_iterator, + def AckFn( + request_iterator, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.stream_stream( + request_iterator, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.stream_stream(request_iterator, target, '/source.v1.Source/AckFn', + "/source.v1.Source/AckFn", source__pb2.AckRequest.SerializeToString, source__pb2.AckResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) @staticmethod - def PendingFn(request, + def PendingFn( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.unary_unary(request, target, '/source.v1.Source/PendingFn', + "/source.v1.Source/PendingFn", google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, source__pb2.PendingResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) @staticmethod - def PartitionsFn(request, + def PartitionsFn( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.unary_unary(request, target, '/source.v1.Source/PartitionsFn', + "/source.v1.Source/PartitionsFn", google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, source__pb2.PartitionsResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) @staticmethod - def IsReady(request, + def IsReady( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.unary_unary(request, target, '/source.v1.Source/IsReady', + "/source.v1.Source/IsReady", google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, source__pb2.ReadyResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) diff --git a/pynumaflow/proto/sourcetransformer/transform_pb2.py b/pynumaflow/proto/sourcetransformer/transform_pb2.py index 8309b7ca..a7b08491 100644 --- a/pynumaflow/proto/sourcetransformer/transform_pb2.py +++ b/pynumaflow/proto/sourcetransformer/transform_pb2.py @@ -7,6 +7,7 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import symbol_database as _symbol_database from google.protobuf.internal import builder as _builder + # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() @@ -16,29 +17,31 @@ from google.protobuf import empty_pb2 as google_dot_protobuf_dot_empty__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x0ftransform.proto\x12\x14sourcetransformer.v1\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1bgoogle/protobuf/empty.proto\"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08\"\xbe\x03\n\x16SourceTransformRequest\x12\x45\n\x07request\x18\x01 \x01(\x0b\x32\x34.sourcetransformer.v1.SourceTransformRequest.Request\x12\x37\n\thandshake\x18\x02 \x01(\x0b\x32\x1f.sourcetransformer.v1.HandshakeH\x00\x88\x01\x01\x1a\x95\x02\n\x07Request\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12R\n\x07headers\x18\x05 \x03(\x0b\x32\x41.sourcetransformer.v1.SourceTransformRequest.Request.HeadersEntry\x12\n\n\x02id\x18\x06 \x01(\t\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x42\x0c\n\n_handshake\"\x98\x02\n\x17SourceTransformResponse\x12\x45\n\x07results\x18\x01 \x03(\x0b\x32\x34.sourcetransformer.v1.SourceTransformResponse.Result\x12\n\n\x02id\x18\x02 \x01(\t\x12\x37\n\thandshake\x18\x03 \x01(\x0b\x32\x1f.sourcetransformer.v1.HandshakeH\x00\x88\x01\x01\x1a\x63\n\x06Result\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04tags\x18\x04 \x03(\tB\x0c\n\n_handshake\"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\xcf\x01\n\x0fSourceTransform\x12t\n\x11SourceTransformFn\x12,.sourcetransformer.v1.SourceTransformRequest\x1a-.sourcetransformer.v1.SourceTransformResponse(\x01\x30\x01\x12\x46\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a#.sourcetransformer.v1.ReadyResponseb\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( + b'\n\x0ftransform.proto\x12\x14sourcetransformer.v1\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1bgoogle/protobuf/empty.proto"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08"\xbe\x03\n\x16SourceTransformRequest\x12\x45\n\x07request\x18\x01 \x01(\x0b\x32\x34.sourcetransformer.v1.SourceTransformRequest.Request\x12\x37\n\thandshake\x18\x02 \x01(\x0b\x32\x1f.sourcetransformer.v1.HandshakeH\x00\x88\x01\x01\x1a\x95\x02\n\x07Request\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12R\n\x07headers\x18\x05 \x03(\x0b\x32\x41.sourcetransformer.v1.SourceTransformRequest.Request.HeadersEntry\x12\n\n\x02id\x18\x06 \x01(\t\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x42\x0c\n\n_handshake"\x98\x02\n\x17SourceTransformResponse\x12\x45\n\x07results\x18\x01 \x03(\x0b\x32\x34.sourcetransformer.v1.SourceTransformResponse.Result\x12\n\n\x02id\x18\x02 \x01(\t\x12\x37\n\thandshake\x18\x03 \x01(\x0b\x32\x1f.sourcetransformer.v1.HandshakeH\x00\x88\x01\x01\x1a\x63\n\x06Result\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04tags\x18\x04 \x03(\tB\x0c\n\n_handshake"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\xcf\x01\n\x0fSourceTransform\x12t\n\x11SourceTransformFn\x12,.sourcetransformer.v1.SourceTransformRequest\x1a-.sourcetransformer.v1.SourceTransformResponse(\x01\x30\x01\x12\x46\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a#.sourcetransformer.v1.ReadyResponseb\x06proto3' +) _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) -_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'transform_pb2', _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, "transform_pb2", _globals) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None - _globals['_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY']._options = None - _globals['_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY']._serialized_options = b'8\001' - _globals['_HANDSHAKE']._serialized_start=103 - _globals['_HANDSHAKE']._serialized_end=127 - _globals['_SOURCETRANSFORMREQUEST']._serialized_start=130 - _globals['_SOURCETRANSFORMREQUEST']._serialized_end=576 - _globals['_SOURCETRANSFORMREQUEST_REQUEST']._serialized_start=285 - _globals['_SOURCETRANSFORMREQUEST_REQUEST']._serialized_end=562 - _globals['_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY']._serialized_start=516 - _globals['_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY']._serialized_end=562 - _globals['_SOURCETRANSFORMRESPONSE']._serialized_start=579 - _globals['_SOURCETRANSFORMRESPONSE']._serialized_end=859 - _globals['_SOURCETRANSFORMRESPONSE_RESULT']._serialized_start=746 - _globals['_SOURCETRANSFORMRESPONSE_RESULT']._serialized_end=845 - _globals['_READYRESPONSE']._serialized_start=861 - _globals['_READYRESPONSE']._serialized_end=891 - _globals['_SOURCETRANSFORM']._serialized_start=894 - _globals['_SOURCETRANSFORM']._serialized_end=1101 + DESCRIPTOR._options = None + _globals["_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY"]._options = None + _globals["_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY"]._serialized_options = b"8\001" + _globals["_HANDSHAKE"]._serialized_start = 103 + _globals["_HANDSHAKE"]._serialized_end = 127 + _globals["_SOURCETRANSFORMREQUEST"]._serialized_start = 130 + _globals["_SOURCETRANSFORMREQUEST"]._serialized_end = 576 + _globals["_SOURCETRANSFORMREQUEST_REQUEST"]._serialized_start = 285 + _globals["_SOURCETRANSFORMREQUEST_REQUEST"]._serialized_end = 562 + _globals["_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY"]._serialized_start = 516 + _globals["_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY"]._serialized_end = 562 + _globals["_SOURCETRANSFORMRESPONSE"]._serialized_start = 579 + _globals["_SOURCETRANSFORMRESPONSE"]._serialized_end = 859 + _globals["_SOURCETRANSFORMRESPONSE_RESULT"]._serialized_start = 746 + _globals["_SOURCETRANSFORMRESPONSE_RESULT"]._serialized_end = 845 + _globals["_READYRESPONSE"]._serialized_start = 861 + _globals["_READYRESPONSE"]._serialized_end = 891 + _globals["_SOURCETRANSFORM"]._serialized_start = 894 + _globals["_SOURCETRANSFORM"]._serialized_end = 1101 # @@protoc_insertion_point(module_scope) diff --git a/pynumaflow/proto/sourcetransformer/transform_pb2.pyi b/pynumaflow/proto/sourcetransformer/transform_pb2.pyi index c7c15079..cc8fe420 100644 --- a/pynumaflow/proto/sourcetransformer/transform_pb2.pyi +++ b/pynumaflow/proto/sourcetransformer/transform_pb2.pyi @@ -3,7 +3,13 @@ from google.protobuf import empty_pb2 as _empty_pb2 from google.protobuf.internal import containers as _containers from google.protobuf import descriptor as _descriptor from google.protobuf import message as _message -from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union +from typing import ( + ClassVar as _ClassVar, + Iterable as _Iterable, + Mapping as _Mapping, + Optional as _Optional, + Union as _Union, +) DESCRIPTOR: _descriptor.FileDescriptor @@ -15,8 +21,10 @@ class Handshake(_message.Message): class SourceTransformRequest(_message.Message): __slots__ = ("request", "handshake") + class Request(_message.Message): __slots__ = ("keys", "value", "event_time", "watermark", "headers", "id") + class HeadersEntry(_message.Message): __slots__ = ("key", "value") KEY_FIELD_NUMBER: _ClassVar[int] @@ -36,15 +44,28 @@ class SourceTransformRequest(_message.Message): watermark: _timestamp_pb2.Timestamp headers: _containers.ScalarMap[str, str] id: str - def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., headers: _Optional[_Mapping[str, str]] = ..., id: _Optional[str] = ...) -> None: ... + def __init__( + self, + keys: _Optional[_Iterable[str]] = ..., + value: _Optional[bytes] = ..., + event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + headers: _Optional[_Mapping[str, str]] = ..., + id: _Optional[str] = ..., + ) -> None: ... REQUEST_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] request: SourceTransformRequest.Request handshake: Handshake - def __init__(self, request: _Optional[_Union[SourceTransformRequest.Request, _Mapping]] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ...) -> None: ... + def __init__( + self, + request: _Optional[_Union[SourceTransformRequest.Request, _Mapping]] = ..., + handshake: _Optional[_Union[Handshake, _Mapping]] = ..., + ) -> None: ... class SourceTransformResponse(_message.Message): __slots__ = ("results", "id", "handshake") + class Result(_message.Message): __slots__ = ("keys", "value", "event_time", "tags") KEYS_FIELD_NUMBER: _ClassVar[int] @@ -55,14 +76,25 @@ class SourceTransformResponse(_message.Message): value: bytes event_time: _timestamp_pb2.Timestamp tags: _containers.RepeatedScalarFieldContainer[str] - def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., tags: _Optional[_Iterable[str]] = ...) -> None: ... + def __init__( + self, + keys: _Optional[_Iterable[str]] = ..., + value: _Optional[bytes] = ..., + event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + tags: _Optional[_Iterable[str]] = ..., + ) -> None: ... RESULTS_FIELD_NUMBER: _ClassVar[int] ID_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] results: _containers.RepeatedCompositeFieldContainer[SourceTransformResponse.Result] id: str handshake: Handshake - def __init__(self, results: _Optional[_Iterable[_Union[SourceTransformResponse.Result, _Mapping]]] = ..., id: _Optional[str] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ...) -> None: ... + def __init__( + self, + results: _Optional[_Iterable[_Union[SourceTransformResponse.Result, _Mapping]]] = ..., + id: _Optional[str] = ..., + handshake: _Optional[_Union[Handshake, _Mapping]] = ..., + ) -> None: ... class ReadyResponse(_message.Message): __slots__ = ("ready",) diff --git a/pynumaflow/proto/sourcetransformer/transform_pb2_grpc.py b/pynumaflow/proto/sourcetransformer/transform_pb2_grpc.py index 1a1ea499..c6820d39 100644 --- a/pynumaflow/proto/sourcetransformer/transform_pb2_grpc.py +++ b/pynumaflow/proto/sourcetransformer/transform_pb2_grpc.py @@ -16,15 +16,15 @@ def __init__(self, channel): channel: A grpc.Channel. """ self.SourceTransformFn = channel.stream_stream( - '/sourcetransformer.v1.SourceTransform/SourceTransformFn', - request_serializer=transform__pb2.SourceTransformRequest.SerializeToString, - response_deserializer=transform__pb2.SourceTransformResponse.FromString, - ) + "/sourcetransformer.v1.SourceTransform/SourceTransformFn", + request_serializer=transform__pb2.SourceTransformRequest.SerializeToString, + response_deserializer=transform__pb2.SourceTransformResponse.FromString, + ) self.IsReady = channel.unary_unary( - '/sourcetransformer.v1.SourceTransform/IsReady', - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=transform__pb2.ReadyResponse.FromString, - ) + "/sourcetransformer.v1.SourceTransform/IsReady", + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=transform__pb2.ReadyResponse.FromString, + ) class SourceTransformServicer(object): @@ -36,69 +36,93 @@ def SourceTransformFn(self, request_iterator, context): SourceTransformFn can be used only at source vertex by source data transformer. """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def IsReady(self, request, context): - """IsReady is the heartbeat endpoint for gRPC. - """ + """IsReady is the heartbeat endpoint for gRPC.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def add_SourceTransformServicer_to_server(servicer, server): rpc_method_handlers = { - 'SourceTransformFn': grpc.stream_stream_rpc_method_handler( - servicer.SourceTransformFn, - request_deserializer=transform__pb2.SourceTransformRequest.FromString, - response_serializer=transform__pb2.SourceTransformResponse.SerializeToString, - ), - 'IsReady': grpc.unary_unary_rpc_method_handler( - servicer.IsReady, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=transform__pb2.ReadyResponse.SerializeToString, - ), + "SourceTransformFn": grpc.stream_stream_rpc_method_handler( + servicer.SourceTransformFn, + request_deserializer=transform__pb2.SourceTransformRequest.FromString, + response_serializer=transform__pb2.SourceTransformResponse.SerializeToString, + ), + "IsReady": grpc.unary_unary_rpc_method_handler( + servicer.IsReady, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=transform__pb2.ReadyResponse.SerializeToString, + ), } generic_handler = grpc.method_handlers_generic_handler( - 'sourcetransformer.v1.SourceTransform', rpc_method_handlers) + "sourcetransformer.v1.SourceTransform", rpc_method_handlers + ) server.add_generic_rpc_handlers((generic_handler,)) - # This class is part of an EXPERIMENTAL API. +# This class is part of an EXPERIMENTAL API. class SourceTransform(object): """Missing associated documentation comment in .proto file.""" @staticmethod - def SourceTransformFn(request_iterator, + def SourceTransformFn( + request_iterator, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.stream_stream( + request_iterator, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.stream_stream(request_iterator, target, '/sourcetransformer.v1.SourceTransform/SourceTransformFn', + "/sourcetransformer.v1.SourceTransform/SourceTransformFn", transform__pb2.SourceTransformRequest.SerializeToString, transform__pb2.SourceTransformResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) @staticmethod - def IsReady(request, + def IsReady( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.unary_unary(request, target, '/sourcetransformer.v1.SourceTransform/IsReady', + "/sourcetransformer.v1.SourceTransform/IsReady", google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, transform__pb2.ReadyResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) From 7cb73d7f22a77975f46e0a329418f22487c1c0dd Mon Sep 17 00:00:00 2001 From: srao12 Date: Mon, 21 Jul 2025 11:59:11 -0700 Subject: [PATCH 20/30] fix: update docker image name Signed-off-by: srao12 --- examples/accumulator/streamsorter/Makefile | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/examples/accumulator/streamsorter/Makefile b/examples/accumulator/streamsorter/Makefile index 77cde191..5eb6a3e8 100644 --- a/examples/accumulator/streamsorter/Makefile +++ b/examples/accumulator/streamsorter/Makefile @@ -1,6 +1,6 @@ -TAG ?= test4 +TAG ?= stable PUSH ?= false -IMAGE_REGISTRY = docker.intuit.com/personal/srao12/py-accumulator-sorter:${TAG} +IMAGE_REGISTRY = quay.io/numaio/numaflow-python/streamsorter:${TAG} DOCKER_FILE_PATH = examples/accumulator/streamsorter/Dockerfile .PHONY: update From d6968b608d9aebfe599260a973b0f4ea3e7a9265 Mon Sep 17 00:00:00 2001 From: srao12 Date: Mon, 21 Jul 2025 18:19:09 -0700 Subject: [PATCH 21/30] fix: update close task Signed-off-by: srao12 --- pynumaflow/accumulator/async_server.py | 76 ++++++++++--------- .../accumulator/servicer/task_manager.py | 2 + 2 files changed, 41 insertions(+), 37 deletions(-) diff --git a/pynumaflow/accumulator/async_server.py b/pynumaflow/accumulator/async_server.py index fc4da947..042359ca 100644 --- a/pynumaflow/accumulator/async_server.py +++ b/pynumaflow/accumulator/async_server.py @@ -75,55 +75,57 @@ class AccumulatorAsyncServer(NumaflowServer): Example invocation: import os from collections.abc import AsyncIterable - from pynumaflow.accumulator import Messages, Message, Datum, Metadata, - AccumulatorAsyncServer, Accumulator + from datetime import datetime + + from pynumaflow.accumulator import Accumulator, AccumulatorAsyncServer + from pynumaflow.accumulator import ( + Message, + Datum, + ) + from pynumaflow.shared.asynciter import NonBlockingIterator class StreamSorter(Accumulator): def __init__(self, counter): - self.counter = counter + self.latest_wm = datetime.fromtimestamp(-1) + self.sorted_buffer: list[Datum] = [] async def handler( self, - keys: list[str], datums: AsyncIterable[Datum], output: NonBlockingIterator, - md: Metadata, ): async for _ in datums: - self.counter += 1 - if self.counter > 20: - msg = f"counter:{self.counter}" - await output.put(Message(str.encode(msg), keys=keys)) - self.counter = 0 - msg = f"counter:{self.counter}" - await output.put(Message(str.encode(msg), keys=keys)) - - async def reduce_handler( - keys: list[str], - datums: AsyncIterable[Datum], - output: NonBlockingIterator, - md: Metadata, - ): - counter = 0 - async for _ in datums: - counter += 1 - if counter > 20: - msg = f"counter:{counter}" - await output.put(Message(str.encode(msg), keys=keys)) - counter = 0 - msg = f"counter:{counter}" - await output.put(Message(str.encode(msg), keys=keys)) + # Process the datums and send output + if datum.watermark and datum.watermark > self.latest_wm: + self.latest_wm = datum.watermark + await self.flush_buffer(output) + + self.insert_sorted(datum) + + def insert_sorted(self, datum: Datum): + # Binary insert to keep sorted buffer in order + left, right = 0, len(self.sorted_buffer) + while left < right: + mid = (left + right) // 2 + if self.sorted_buffer[mid].event_time > datum.event_time: + right = mid + else: + left = mid + 1 + self.sorted_buffer.insert(left, datum) + + async def flush_buffer(self, output: NonBlockingIterator): + i = 0 + for datum in self.sorted_buffer: + if datum.event_time > self.latest_wm: + break + await output.put(Message.from_datum(datum)) + i += 1 + # Remove flushed items + self.sorted_buffer = self.sorted_buffer[i:] + if __name__ == "__main__": - invoke = os.getenv("INVOKE", "func_handler") - if invoke == "class": - # Here we are using the class instance as the accumulator_instance - # which will be used to invoke the handler function. - # We are passing the init_args for the class instance. - grpc_server = AccumulatorAsyncServer(StreamSorter, init_args=(0,)) - else: - # Here we are using the handler function directly as the accumulator_instance. - grpc_server = AccumulatorAsyncServer(accumulator_handler) + grpc_server = AccumulatorAsyncServer(StreamSorter) grpc_server.start() """ diff --git a/pynumaflow/accumulator/servicer/task_manager.py b/pynumaflow/accumulator/servicer/task_manager.py index f8255e6e..18f61358 100644 --- a/pynumaflow/accumulator/servicer/task_manager.py +++ b/pynumaflow/accumulator/servicer/task_manager.py @@ -112,6 +112,8 @@ async def close_task(self, req): if curr_task: await self.tasks[unified_key].iterator.put(STREAM_EOF) + await curr_task.future + await curr_task.consumer_future self.tasks.pop(unified_key) else: _LOGGER.critical("accumulator task not found", exc_info=True) From 7faf39bf5339aace67caa6d93f4d7d26dc89f600 Mon Sep 17 00:00:00 2001 From: Takashi Menjo Date: Wed, 23 Jul 2025 02:27:02 +0900 Subject: [PATCH 22/30] chore: fix broken make proto (#235) Signed-off-by: Takashi Menjo --- Makefile | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/Makefile b/Makefile index b403ac66..1ababadc 100644 --- a/Makefile +++ b/Makefile @@ -26,14 +26,13 @@ setup: poetry install --with dev --no-root proto: - python3 -m grpc_tools.protoc --pyi_out=pynumaflow/proto/sinker -I=pynumaflow/proto/sinker --python_out=pynumaflow/proto/sinker --grpc_python_out=pynumaflow/proto/sinker pynumaflow/proto/sinker/*.proto - python3 -m grpc_tools.protoc --pyi_out=pynumaflow/proto/mapper -I=pynumaflow/proto/mapper --python_out=pynumaflow/proto/mapper --grpc_python_out=pynumaflow/proto/mapper pynumaflow/proto/mapper/*.proto - python3 -m grpc_tools.protoc --pyi_out=pynumaflow/proto/mapstreamer -I=pynumaflow/proto/mapstreamer --python_out=pynumaflow/proto/mapstreamer --grpc_python_out=pynumaflow/proto/mapstreamer pynumaflow/proto/mapstreamer/*.proto - python3 -m grpc_tools.protoc --pyi_out=pynumaflow/proto/reducer -I=pynumaflow/proto/reducer --python_out=pynumaflow/proto/reducer --grpc_python_out=pynumaflow/proto/reducer pynumaflow/proto/reducer/*.proto - python3 -m grpc_tools.protoc --pyi_out=pynumaflow/proto/sourcetransformer -I=pynumaflow/proto/sourcetransformer --python_out=pynumaflow/proto/sourcetransformer --grpc_python_out=pynumaflow/proto/sourcetransformer pynumaflow/proto/sourcetransformer/*.proto - python3 -m grpc_tools.protoc --pyi_out=pynumaflow/proto/sideinput -I=pynumaflow/proto/sideinput --python_out=pynumaflow/proto/sideinput --grpc_python_out=pynumaflow/proto/sideinput pynumaflow/proto/sideinput/*.proto - python3 -m grpc_tools.protoc --pyi_out=pynumaflow/proto/sourcer -I=pynumaflow/proto/sourcer --python_out=pynumaflow/proto/sourcer --grpc_python_out=pynumaflow/proto/sourcer pynumaflow/proto/sourcer/*.proto - python3 -m grpc_tools.protoc --pyi_out=pynumaflow/proto/batchmapper -I=pynumaflow/proto/batchmapper --python_out=pynumaflow/proto/batchmapper --grpc_python_out=pynumaflow/proto/batchmapper pynumaflow/proto/batchmapper/*.proto + poetry run python3 -m grpc_tools.protoc --pyi_out=pynumaflow/proto/sinker -I=pynumaflow/proto/sinker --python_out=pynumaflow/proto/sinker --grpc_python_out=pynumaflow/proto/sinker pynumaflow/proto/sinker/*.proto + poetry run python3 -m grpc_tools.protoc --pyi_out=pynumaflow/proto/mapper -I=pynumaflow/proto/mapper --python_out=pynumaflow/proto/mapper --grpc_python_out=pynumaflow/proto/mapper pynumaflow/proto/mapper/*.proto + poetry run python3 -m grpc_tools.protoc --pyi_out=pynumaflow/proto/reducer -I=pynumaflow/proto/reducer --python_out=pynumaflow/proto/reducer --grpc_python_out=pynumaflow/proto/reducer pynumaflow/proto/reducer/*.proto + poetry run python3 -m grpc_tools.protoc --pyi_out=pynumaflow/proto/sourcetransformer -I=pynumaflow/proto/sourcetransformer --python_out=pynumaflow/proto/sourcetransformer --grpc_python_out=pynumaflow/proto/sourcetransformer pynumaflow/proto/sourcetransformer/*.proto + poetry run python3 -m grpc_tools.protoc --pyi_out=pynumaflow/proto/sideinput -I=pynumaflow/proto/sideinput --python_out=pynumaflow/proto/sideinput --grpc_python_out=pynumaflow/proto/sideinput pynumaflow/proto/sideinput/*.proto + poetry run python3 -m grpc_tools.protoc --pyi_out=pynumaflow/proto/sourcer -I=pynumaflow/proto/sourcer --python_out=pynumaflow/proto/sourcer --grpc_python_out=pynumaflow/proto/sourcer pynumaflow/proto/sourcer/*.proto - sed -i '' 's/^\(import.*_pb2\)/from . \1/' pynumaflow/proto/*/*.py + sed -i.bak -e 's/^\(import.*_pb2\)/from . \1/' pynumaflow/proto/*/*.py + rm pynumaflow/proto/*/*.py.bak From 8aebf045109b2cb01a0fd0982efb29a40a59cd0f Mon Sep 17 00:00:00 2001 From: srao12 Date: Thu, 24 Jul 2025 19:00:23 -0700 Subject: [PATCH 23/30] fix: tests Signed-off-by: srao12 --- .../accumulator/servicer/task_manager.py | 26 +-- tests/accumulator/test_async_accumulator.py | 198 +++++++++++++++++- tests/accumulator/test_datatypes.py | 2 - tests/accumulator/test_use_cases.py | 60 +++++- 4 files changed, 250 insertions(+), 36 deletions(-) diff --git a/pynumaflow/accumulator/servicer/task_manager.py b/pynumaflow/accumulator/servicer/task_manager.py index 18f61358..fd2cb3c7 100644 --- a/pynumaflow/accumulator/servicer/task_manager.py +++ b/pynumaflow/accumulator/servicer/task_manager.py @@ -64,11 +64,6 @@ def __init__(self, handler: Union[AccumulatorAsyncCallable, _AccumulatorBuilderC # This queue is also used to send the error/exceptions to the client # if the accumulator operation fails. self.global_result_queue = NonBlockingIterator() - # EOF response counting to ensure proper termination - self._expected_eof_count = 0 - self._received_eof_count = 0 - self._eof_count_lock = asyncio.Lock() - self._stream_termination_event = asyncio.Event() def get_unique_windows(self): """ @@ -102,7 +97,6 @@ async def stream_send_eof(self): task_keys = list(self.tasks.keys()) for unified_key in task_keys: await self.tasks[unified_key].iterator.put(STREAM_EOF) - self.tasks.pop(unified_key) async def close_task(self, req): d = req.payload @@ -170,10 +164,6 @@ async def create_task(self, req): # Save the result of the accumulator operation to the task list self.tasks[unified_key] = curr_task - # Increment expected EOF count since we created a new task - async with self._eof_count_lock: - self._expected_eof_count += 1 - # Put the request in the iterator await curr_task.iterator.put(d) @@ -275,10 +265,7 @@ async def process_input_stream( # all the results of this task to the global result queue con_future = task.consumer_future await con_future - - # Wait for all tasks to send their EOF responses before terminating the stream - # This ensures proper ordering: all messages -> all EOF responses -> STREAM_EOF - await self._stream_termination_event.wait() + self.tasks.clear() # Now send STREAM_EOF to terminate the global result queue iterator await self.global_result_queue.put(STREAM_EOF) @@ -353,17 +340,6 @@ async def write_to_global_queue( ) await output_queue.put(res) - # Increment received EOF count and check if all tasks are done - async with self._eof_count_lock: - self._received_eof_count += 1 - - # Check if all tasks have sent their EOF responses - if self._received_eof_count == self._expected_eof_count: - self._stream_termination_event.set() - elif self._received_eof_count > self._expected_eof_count: - # Still set the event to prevent hanging, but log the error - self._stream_termination_event.set() - def clean_background(self, task): """ Remove the task from the background tasks collection diff --git a/tests/accumulator/test_async_accumulator.py b/tests/accumulator/test_async_accumulator.py index 2786acd7..c9144852 100644 --- a/tests/accumulator/test_async_accumulator.py +++ b/tests/accumulator/test_async_accumulator.py @@ -27,7 +27,7 @@ LOGGER = setup_logging(__name__) -def request_generator(count, request, resetkey: bool = False): +def request_generator(count, request, resetkey: bool = False, send_close: bool = False): for i in range(count): if resetkey: # Clear previous keys and add new ones @@ -43,6 +43,36 @@ def request_generator(count, request, resetkey: bool = False): ) yield request + if send_close: + # Send a close operation after all requests + request.operation.event = accumulator_pb2.AccumulatorRequest.WindowOperation.Event.CLOSE + yield request + +def request_generator_append_only(count, request, resetkey: bool = False): + for i in range(count): + if resetkey: + # Clear previous keys and add new ones + del request.payload.keys[:] + request.payload.keys.extend([f"key-{i}"]) + + # Set operation to APPEND for all requests + request.operation.event = accumulator_pb2.AccumulatorRequest.WindowOperation.Event.APPEND + yield request + +def request_generator_mixed(count, request, resetkey: bool = False): + for i in range(count): + if resetkey: + # Clear previous keys and add new ones + del request.payload.keys[:] + request.payload.keys.extend([f"key-{i}"]) + + if i% 2 == 0: + # Set operation to OPEN for the first request + request.operation.event = accumulator_pb2.AccumulatorRequest.WindowOperation.Event.APPEND + else: + # Set operation to APPEND for all requests + request.operation.event = accumulator_pb2.AccumulatorRequest.WindowOperation.Event.CLOSE + yield request def start_request() -> accumulator_pb2.AccumulatorRequest: event_time_timestamp, watermark_timestamp = get_time_args() @@ -70,6 +100,23 @@ def start_request() -> accumulator_pb2.AccumulatorRequest: return request +def start_request_without_open() -> accumulator_pb2.AccumulatorRequest: + event_time_timestamp, watermark_timestamp = get_time_args() + + payload = accumulator_pb2.Payload( + keys=["test_key"], + value=mock_message(), + event_time=event_time_timestamp, + watermark=watermark_timestamp, + id="test_id", + ) + + request = accumulator_pb2.AccumulatorRequest( + payload=payload, + ) + return request + + _s: Server = None _channel = grpc.insecure_channel("unix:///tmp/accumulator.sock") _loop = None @@ -161,6 +208,7 @@ def test_accumulate(self) -> None: count = 0 eof_count = 0 for r in generator_response: + print(r) if hasattr(r, "payload") and r.payload.value: count += 1 # Each datum should increment the counter @@ -221,6 +269,154 @@ def test_accumulate_with_multiple_keys(self) -> None: # Each key should appear once self.assertEqual(len(key_counts), 10) + def test_accumulate_with_close(self) -> None: + stub = self.__stub() + request = start_request() + generator_response = None + try: + generator_response = stub.AccumulateFn( + request_iterator=request_generator(count=5, request=request, send_close=True) + ) + except grpc.RpcError as e: + logging.error(e) + + # capture the output from the AccumulateFn generator and assert. + count = 0 + eof_count = 0 + for r in generator_response: + print(r) + if hasattr(r, "payload") and r.payload.value: + count += 1 + # Each datum should increment the counter + expected_msg = f"counter:{count}" + self.assertEqual( + bytes(expected_msg, encoding="utf-8"), + r.payload.value, + ) + self.assertEqual(r.EOF, False) + # Check that keys are preserved + self.assertEqual(list(r.payload.keys), ["test_key"]) + else: + self.assertEqual(r.EOF, True) + eof_count += 1 + + # We should have received 5 messages (one for each datum) + self.assertEqual(5, count) + self.assertEqual(1, eof_count) + + def test_accumulate_append_without_open(self) -> None: + stub = self.__stub() + request = start_request_without_open() + generator_response = None + try: + generator_response = stub.AccumulateFn( + request_iterator=request_generator_append_only(count=5, request=request) + ) + except grpc.RpcError as e: + logging.error(e) + + # capture the output from the AccumulateFn generator and assert. + count = 0 + eof_count = 0 + for r in generator_response: + print(r) + if hasattr(r, "payload") and r.payload.value: + count += 1 + # Each datum should increment the counter + expected_msg = f"counter:{count}" + self.assertEqual( + bytes(expected_msg, encoding="utf-8"), + r.payload.value, + ) + self.assertEqual(r.EOF, False) + # Check that keys are preserved + self.assertEqual(list(r.payload.keys), ["test_key"]) + else: + self.assertEqual(r.EOF, True) + eof_count += 1 + + # We should have received 5 messages (one for each datum) + self.assertEqual(5, count) + self.assertEqual(1, eof_count) + + def test_accumulate_with_multiple_keys_append_only(self) -> None: + stub = self.__stub() + request = start_request_without_open() + generator_response = None + try: + generator_response = stub.AccumulateFn( + request_iterator=request_generator_append_only( + count=10, request=request, resetkey=True + ) + ) + except grpc.RpcError as e: + print(e) + + count = 0 + eof_count = 0 + key_counts = {} + + # capture the output from the AccumulateFn generator and assert. + for r in generator_response: + # Check for responses with values + if r.payload.value: + count += 1 + # Track count per key + key = r.payload.keys[0] if r.payload.keys else "no_key" + key_counts[key] = key_counts.get(key, 0) + 1 + + # Each key should have its own counter starting from 1 + expected_msg = f"counter:{key_counts[key]}" + self.assertEqual( + bytes(expected_msg, encoding="utf-8"), + r.payload.value, + ) + self.assertEqual(r.EOF, False) + else: + eof_count += 1 + self.assertEqual(r.EOF, True) + + # We should have 10 messages (one for each key) + self.assertEqual(10, count) + self.assertEqual(10, eof_count) + # Each key should appear once + self.assertEqual(len(key_counts), 10) + + def test_accumulate_append_mixed(self) -> None: + stub = self.__stub() + request = start_request() + generator_response = None + try: + generator_response = stub.AccumulateFn( + request_iterator=request_generator_mixed(count=5, request=request) + ) + except grpc.RpcError as e: + logging.error(e) + + # capture the output from the AccumulateFn generator and assert. + count = 0 + eof_count = 0 + for r in generator_response: + print(r) + if hasattr(r, "payload") and r.payload.value: + count += 1 + # Each datum should increment the counter + expected_msg = f"counter:1" + self.assertEqual( + bytes(expected_msg, encoding="utf-8"), + r.payload.value, + ) + self.assertEqual(r.EOF, False) + # Check that keys are preserved + self.assertEqual(list(r.payload.keys), ["test_key"]) + else: + self.assertEqual(r.EOF, True) + eof_count += 1 + + # We should have received 5 messages (one for each datum) + self.assertEqual(3, count) + self.assertEqual(3, eof_count) + def test_is_ready(self) -> None: with grpc.insecure_channel("unix:///tmp/accumulator.sock") as channel: stub = accumulator_pb2_grpc.AccumulatorStub(channel) diff --git a/tests/accumulator/test_datatypes.py b/tests/accumulator/test_datatypes.py index 75ca5cc0..e0d94c49 100644 --- a/tests/accumulator/test_datatypes.py +++ b/tests/accumulator/test_datatypes.py @@ -564,8 +564,6 @@ async def run_test(): self.assertEqual(task_manager._TaskManager__accumulator_handler, handler) self.assertEqual(len(task_manager.tasks), 0) self.assertEqual(len(task_manager.background_tasks), 0) - self.assertEqual(task_manager._expected_eof_count, 0) - self.assertEqual(task_manager._received_eof_count, 0) asyncio.run(run_test()) diff --git a/tests/accumulator/test_use_cases.py b/tests/accumulator/test_use_cases.py index 5f377da6..7a07881e 100644 --- a/tests/accumulator/test_use_cases.py +++ b/tests/accumulator/test_use_cases.py @@ -23,15 +23,23 @@ def __init__(self): self.buffer: list[Datum] = [] async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterator): + print("[StreamSorterAccumulator] Starting handler") # Collect all datums + datum_count = 0 async for datum in datums: + datum_count += 1 + print(f"[StreamSorterAccumulator] Received datum {datum_count}: {datum.value}") self.buffer.append(datum) + print(f"[StreamSorterAccumulator] Total datums received: {datum_count}") # Sort by event_time self.buffer.sort(key=lambda d: d.event_time) # Emit sorted datums + message_count = 0 for datum in self.buffer: + message_count += 1 + print(f"[StreamSorterAccumulator] Emitting message {message_count}: {datum.value}") await output.put( Message( value=datum.value, @@ -40,6 +48,8 @@ async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterato ) ) + print(f"[StreamSorterAccumulator] Total messages emitted: {message_count}") + print(f"[StreamSorterAccumulator] Handler completed, buffer cleared") # Clear buffer for next window self.buffer.clear() @@ -310,6 +320,7 @@ def test_stream_sorter_functionality(self): """Test actual sorting functionality""" async def _test_stream_sorter_functionality(): + print("[Test] Starting stream sorter functionality test") sorter = StreamSorterAccumulator() output = NonBlockingIterator() @@ -340,13 +351,42 @@ async def _test_stream_sorter_functionality(): async def datum_generator(): for datum in datums: + print(f"[Test] Yielding datum: {datum.value}") yield datum + print("[Test] Datum generator completed") # Process the datums + print("[Test] Calling sorter.handler") await sorter.handler(datum_generator(), output) + print("[Test] sorter.handler completed") + + # Send STREAM_EOF to signal completion (this is normally done by TaskManager) + print("[Test] Sending STREAM_EOF to output queue") + await output.put(STREAM_EOF) + print("[Test] STREAM_EOF sent") + + # Now let's read from the output to see what was written + print("[Test] Reading from output queue") + results = [] + reader = output.read_iterator() + async for item in reader: + print(f"[Test] Read from output: {item}") + results.append(item) + if item == STREAM_EOF: + print("[Test] Received STREAM_EOF, breaking") + break + # Safety break to avoid infinite loop + if len(results) >= 10: + print("[Test] Safety break - too many results") + break + + print(f"[Test] Total results read: {len(results)}") + for i, result in enumerate(results): + print(f"[Test] Result {i}: {result}") # Verify the buffer is cleared self.assertEqual(len(sorter.buffer), 0) + print("[Test] Test completed") asyncio.run(_test_stream_sorter_functionality()) @@ -578,19 +618,21 @@ async def failing_request_iterator(): mock_task_manager = Mock() mock_tm.return_value = mock_task_manager - # Mock read_iterator to raise exception + # Mock read_iterator to return an async generator that raises exception async def failing_reader(): raise RuntimeError("Consumer error") mock_result_queue = Mock() - mock_result_queue.read_iterator.return_value = failing_reader() + # Mock the method to return a coroutine when called + mock_result_queue.read_iterator = lambda: failing_reader() mock_task_manager.global_result_queue = mock_result_queue - # Mock process_input_stream + # Mock process_input_stream to return a coroutine async def mock_process(): pass - mock_task_manager.process_input_stream.return_value = mock_process() + # Mock the method to return a coroutine when called + mock_task_manager.process_input_stream = lambda x: mock_process() # This should handle the consumer exception with patch( @@ -610,20 +652,22 @@ async def mock_process(): mock_task_manager2 = Mock() mock_tm2.return_value = mock_task_manager2 - # Mock read_iterator to work normally + # Mock read_iterator to return an async generator async def normal_reader(): return yield # Empty generator mock_result_queue2 = Mock() - mock_result_queue2.read_iterator.return_value = normal_reader() + # Mock the method to return a coroutine when called + mock_result_queue2.read_iterator = lambda: normal_reader() mock_task_manager2.global_result_queue = mock_result_queue2 - # Mock process_input_stream to raise exception when awaited + # Mock process_input_stream to return a coroutine that raises exception async def failing_process(): raise RuntimeError("Producer error") - mock_task_manager2.process_input_stream.return_value = failing_process() + # Mock the method to return a coroutine when called + mock_task_manager2.process_input_stream = lambda x: failing_process() # This should handle the producer exception with patch( From 20704011984433ac2e4c3b2d43fa3e33e0f5fa99 Mon Sep 17 00:00:00 2001 From: srao12 Date: Thu, 24 Jul 2025 19:21:06 -0700 Subject: [PATCH 24/30] fix: add comprehensive accumulator window operation tests MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - Add tests for OPEN, APPEND, and CLOSE window operations - Test standard flow (OPEN→APPEND→CLOSE), append-only, and mixed operation patterns - Add multi-key processing and error handling test scenarios Signed-off-by: srao12 --- tests/accumulator/test_async_accumulator.py | 23 +++++++++--------- tests/accumulator/test_use_cases.py | 26 ++------------------- 2 files changed, 14 insertions(+), 35 deletions(-) diff --git a/tests/accumulator/test_async_accumulator.py b/tests/accumulator/test_async_accumulator.py index c9144852..9d529889 100644 --- a/tests/accumulator/test_async_accumulator.py +++ b/tests/accumulator/test_async_accumulator.py @@ -48,6 +48,7 @@ def request_generator(count, request, resetkey: bool = False, send_close: bool = request.operation.event = accumulator_pb2.AccumulatorRequest.WindowOperation.Event.CLOSE yield request + def request_generator_append_only(count, request, resetkey: bool = False): for i in range(count): if resetkey: @@ -59,6 +60,7 @@ def request_generator_append_only(count, request, resetkey: bool = False): request.operation.event = accumulator_pb2.AccumulatorRequest.WindowOperation.Event.APPEND yield request + def request_generator_mixed(count, request, resetkey: bool = False): for i in range(count): if resetkey: @@ -66,14 +68,17 @@ def request_generator_mixed(count, request, resetkey: bool = False): del request.payload.keys[:] request.payload.keys.extend([f"key-{i}"]) - if i% 2 == 0: - # Set operation to OPEN for the first request - request.operation.event = accumulator_pb2.AccumulatorRequest.WindowOperation.Event.APPEND + if i % 2 == 0: + # Set operation to APPEND for even requests + request.operation.event = ( + accumulator_pb2.AccumulatorRequest.WindowOperation.Event.APPEND + ) else: - # Set operation to APPEND for all requests + # Set operation to CLOSE for odd requests request.operation.event = accumulator_pb2.AccumulatorRequest.WindowOperation.Event.CLOSE yield request + def start_request() -> accumulator_pb2.AccumulatorRequest: event_time_timestamp, watermark_timestamp = get_time_args() window = accumulator_pb2.KeyedWindow( @@ -208,7 +213,6 @@ def test_accumulate(self) -> None: count = 0 eof_count = 0 for r in generator_response: - print(r) if hasattr(r, "payload") and r.payload.value: count += 1 # Each datum should increment the counter @@ -237,7 +241,7 @@ def test_accumulate_with_multiple_keys(self) -> None: request_iterator=request_generator(count=10, request=request, resetkey=True), ) except grpc.RpcError as e: - print(e) + LOGGER.error(e) count = 0 eof_count = 0 @@ -284,7 +288,6 @@ def test_accumulate_with_close(self) -> None: count = 0 eof_count = 0 for r in generator_response: - print(r) if hasattr(r, "payload") and r.payload.value: count += 1 # Each datum should increment the counter @@ -319,7 +322,6 @@ def test_accumulate_append_without_open(self) -> None: count = 0 eof_count = 0 for r in generator_response: - print(r) if hasattr(r, "payload") and r.payload.value: count += 1 # Each datum should increment the counter @@ -350,7 +352,7 @@ def test_accumulate_with_multiple_keys_append_only(self) -> None: ) ) except grpc.RpcError as e: - print(e) + LOGGER.error(e) count = 0 eof_count = 0 @@ -397,11 +399,10 @@ def test_accumulate_append_mixed(self) -> None: count = 0 eof_count = 0 for r in generator_response: - print(r) if hasattr(r, "payload") and r.payload.value: count += 1 # Each datum should increment the counter - expected_msg = f"counter:1" + expected_msg = "counter:1" self.assertEqual( bytes(expected_msg, encoding="utf-8"), r.payload.value, diff --git a/tests/accumulator/test_use_cases.py b/tests/accumulator/test_use_cases.py index 7a07881e..643c3c8b 100644 --- a/tests/accumulator/test_use_cases.py +++ b/tests/accumulator/test_use_cases.py @@ -23,15 +23,12 @@ def __init__(self): self.buffer: list[Datum] = [] async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterator): - print("[StreamSorterAccumulator] Starting handler") # Collect all datums datum_count = 0 async for datum in datums: datum_count += 1 - print(f"[StreamSorterAccumulator] Received datum {datum_count}: {datum.value}") self.buffer.append(datum) - print(f"[StreamSorterAccumulator] Total datums received: {datum_count}") # Sort by event_time self.buffer.sort(key=lambda d: d.event_time) @@ -39,7 +36,6 @@ async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterato message_count = 0 for datum in self.buffer: message_count += 1 - print(f"[StreamSorterAccumulator] Emitting message {message_count}: {datum.value}") await output.put( Message( value=datum.value, @@ -48,8 +44,6 @@ async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterato ) ) - print(f"[StreamSorterAccumulator] Total messages emitted: {message_count}") - print(f"[StreamSorterAccumulator] Handler completed, buffer cleared") # Clear buffer for next window self.buffer.clear() @@ -320,7 +314,6 @@ def test_stream_sorter_functionality(self): """Test actual sorting functionality""" async def _test_stream_sorter_functionality(): - print("[Test] Starting stream sorter functionality test") sorter = StreamSorterAccumulator() output = NonBlockingIterator() @@ -351,42 +344,27 @@ async def _test_stream_sorter_functionality(): async def datum_generator(): for datum in datums: - print(f"[Test] Yielding datum: {datum.value}") yield datum - print("[Test] Datum generator completed") # Process the datums - print("[Test] Calling sorter.handler") await sorter.handler(datum_generator(), output) - print("[Test] sorter.handler completed") - + # Send STREAM_EOF to signal completion (this is normally done by TaskManager) - print("[Test] Sending STREAM_EOF to output queue") await output.put(STREAM_EOF) - print("[Test] STREAM_EOF sent") # Now let's read from the output to see what was written - print("[Test] Reading from output queue") results = [] reader = output.read_iterator() async for item in reader: - print(f"[Test] Read from output: {item}") results.append(item) if item == STREAM_EOF: - print("[Test] Received STREAM_EOF, breaking") break # Safety break to avoid infinite loop if len(results) >= 10: - print("[Test] Safety break - too many results") break - - print(f"[Test] Total results read: {len(results)}") - for i, result in enumerate(results): - print(f"[Test] Result {i}: {result}") # Verify the buffer is cleared self.assertEqual(len(sorter.buffer), 0) - print("[Test] Test completed") asyncio.run(_test_stream_sorter_functionality()) @@ -652,7 +630,7 @@ async def mock_process(): mock_task_manager2 = Mock() mock_tm2.return_value = mock_task_manager2 - # Mock read_iterator to return an async generator + # Mock read_iterator to return an async generator async def normal_reader(): return yield # Empty generator From 6d8399af5d961c46f91428f599833022ffdd9fd7 Mon Sep 17 00:00:00 2001 From: srao12 Date: Sat, 26 Jul 2025 19:52:14 -0700 Subject: [PATCH 25/30] fix: comments and tests Signed-off-by: srao12 --- pynumaflow/accumulator/__init__.py | 2 - pynumaflow/accumulator/_dtypes.py | 29 +- .../accumulator/servicer/async_servicer.py | 2 +- .../accumulator/servicer/task_manager.py | 27 +- .../proto/accumulator/accumulator_pb2.py | 53 +- .../proto/accumulator/accumulator_pb2.pyi | 49 +- .../proto/accumulator/accumulator_pb2_grpc.py | 127 ++- pynumaflow/proto/mapper/map_pb2.py | 53 +- pynumaflow/proto/mapper/map_pb2.pyi | 43 +- pynumaflow/proto/mapper/map_pb2_grpc.py | 127 ++- pynumaflow/proto/reducer/reduce_pb2.py | 53 +- pynumaflow/proto/reducer/reduce_pb2.pyi | 55 +- pynumaflow/proto/reducer/reduce_pb2_grpc.py | 127 ++- pynumaflow/proto/sideinput/sideinput_pb2.py | 21 +- .../proto/sideinput/sideinput_pb2_grpc.py | 127 ++- pynumaflow/proto/sinker/sink_pb2.py | 53 +- pynumaflow/proto/sinker/sink_pb2.pyi | 43 +- pynumaflow/proto/sinker/sink_pb2_grpc.py | 127 ++- pynumaflow/proto/sourcer/source_pb2.py | 93 ++- pynumaflow/proto/sourcer/source_pb2.pyi | 84 +- pynumaflow/proto/sourcer/source_pb2_grpc.py | 304 +++----- .../proto/sourcetransformer/transform_pb2.py | 45 +- .../proto/sourcetransformer/transform_pb2.pyi | 42 +- .../sourcetransformer/transform_pb2_grpc.py | 124 ++- tests/accumulator/test_async_accumulator.py | 67 +- .../accumulator/test_async_accumulator_err.py | 30 +- tests/accumulator/test_datatypes.py | 353 +-------- tests/accumulator/test_use_cases.py | 723 ++++-------------- tests/accumulator/utils.py | 165 ---- 29 files changed, 824 insertions(+), 2324 deletions(-) diff --git a/pynumaflow/accumulator/__init__.py b/pynumaflow/accumulator/__init__.py index c6ea8d0b..0d1368d8 100644 --- a/pynumaflow/accumulator/__init__.py +++ b/pynumaflow/accumulator/__init__.py @@ -2,7 +2,6 @@ Message, Datum, IntervalWindow, - Metadata, DROP, KeyedWindow, Accumulator, @@ -13,7 +12,6 @@ "Message", "Datum", "IntervalWindow", - "Metadata", "DROP", "AccumulatorAsyncServer", "KeyedWindow", diff --git a/pynumaflow/accumulator/_dtypes.py b/pynumaflow/accumulator/_dtypes.py index 8a8922d6..31a0d5fe 100644 --- a/pynumaflow/accumulator/_dtypes.py +++ b/pynumaflow/accumulator/_dtypes.py @@ -118,7 +118,7 @@ def headers(self) -> dict[str, str]: Returns: dict[str, str]: A dictionary containing header key-value pairs for this event. """ - return self._headers + return self._headers.copy() @property def id(self) -> str: @@ -226,27 +226,6 @@ def keys(self) -> list[str]: return self._keys -@dataclass(init=False) -class Metadata: - """Defines the metadata for the event.""" - - __slots__ = ("_interval_window",) - - _interval_window: IntervalWindow - - def __init__(self, interval_window: IntervalWindow): - self._interval_window = interval_window - - @property - def interval_window(self) -> IntervalWindow: - """Returns the interval window for the event. - - Returns: - IntervalWindow: The interval window associated with this event. - """ - return self._interval_window - - @dataclass class AccumulatorResult: """Defines the object to hold the result of accumulator computation.""" @@ -486,7 +465,7 @@ def headers(self) -> dict[str, str]: Returns: dict[str, str]: A dictionary containing header key-value pairs for this message. """ - return self._headers + return self._headers.copy() @property def id(self) -> str: @@ -517,9 +496,7 @@ def from_datum(cls, datum: Datum): ) -AccumulatorAsyncCallable = Callable[ - [list[str], AsyncIterable[Datum], NonBlockingIterator, Metadata], None -] +AccumulatorAsyncCallable = Callable[[list[str], AsyncIterable[Datum], NonBlockingIterator], None] class Accumulator(metaclass=ABCMeta): diff --git a/pynumaflow/accumulator/servicer/async_servicer.py b/pynumaflow/accumulator/servicer/async_servicer.py index 1891e5a9..6eebdbeb 100644 --- a/pynumaflow/accumulator/servicer/async_servicer.py +++ b/pynumaflow/accumulator/servicer/async_servicer.py @@ -39,7 +39,7 @@ async def datum_generator( value=d.payload.value, event_time=d.payload.event_time.ToDatetime(), watermark=d.payload.watermark.ToDatetime(), - id_=d.payload.id, # Added missing id field + id_=d.payload.id, headers=dict(d.payload.headers), ), ) diff --git a/pynumaflow/accumulator/servicer/task_manager.py b/pynumaflow/accumulator/servicer/task_manager.py index fd2cb3c7..523a1aee 100644 --- a/pynumaflow/accumulator/servicer/task_manager.py +++ b/pynumaflow/accumulator/servicer/task_manager.py @@ -38,11 +38,6 @@ def build_window_hash(window): return f"{window.start.ToMilliseconds()}:{window.end.ToMilliseconds()}" -def create_window_eof_response(window): - """Create a Accumulator response with EOF=True for a given window""" - return accumulator_pb2.AccumulatorResponse(window=window, EOF=True) - - class TaskManager: """ TaskManager is responsible for managing the Accumulator tasks. @@ -89,8 +84,9 @@ def get_tasks(self): async def stream_send_eof(self): """ - Sends EOF to input streams of all the Accumulator - tasks that are currently being processed. + Function used to indicate to all processing tasks that no + more requests are expected by sending EOF message to + local input streams of individual tasks. This is called when the input grpc stream is closed. """ # Create a copy of the keys to avoid dictionary size change during iteration @@ -99,6 +95,16 @@ async def stream_send_eof(self): await self.tasks[unified_key].iterator.put(STREAM_EOF) async def close_task(self, req): + """ + Closes a running accumulator task for a given key. + Based on the request we compute the unique key, and then + signal the corresponding task for it to closure. + The steps involve + 1. Send a signal to the local request queue of the task to stop reading + 2. Wait for the user function to complete + 3. Wait for all the results from the task to be written to the global result queue + 4. Remove the task from the tracker + """ d = req.payload keys = d.keys() unified_key = build_unique_key_name(keys) @@ -259,6 +265,7 @@ async def process_input_stream( # # Send an EOF message to the local result queue # # This will signal that the task has completed processing + # TODO: remove this and test end to end as we are sending EOF in _invoke_accumulator await task.result_queue.put(STREAM_EOF) # Wait for the local queue to write @@ -278,9 +285,9 @@ async def write_to_global_queue( self, input_queue: NonBlockingIterator, output_queue: NonBlockingIterator, unified_key: str ): """ - This task is for given Accumulator task. - This would from the local result queue for the task and then write - to the global result queue + This function is used to route the messages from the + local result queue for a given task to the global result queue. + Once all messages are routed, it sends the window EOF messages for the same. """ reader = input_queue.read_iterator() task = self.tasks[unified_key] diff --git a/pynumaflow/proto/accumulator/accumulator_pb2.py b/pynumaflow/proto/accumulator/accumulator_pb2.py index f1e8ec8d..bc29e898 100644 --- a/pynumaflow/proto/accumulator/accumulator_pb2.py +++ b/pynumaflow/proto/accumulator/accumulator_pb2.py @@ -7,7 +7,6 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import symbol_database as _symbol_database from google.protobuf.internal import builder as _builder - # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() @@ -17,36 +16,32 @@ from google.protobuf import timestamp_pb2 as google_dot_protobuf_dot_timestamp__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x11\x61\x63\x63umulator.proto\x12\x0e\x61\x63\x63umulator.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto"\xf8\x01\n\x07Payload\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\n\n\x02id\x18\x05 \x01(\t\x12\x35\n\x07headers\x18\x06 \x03(\x0b\x32$.accumulator.v1.Payload.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01"\xbe\x02\n\x12\x41\x63\x63umulatorRequest\x12(\n\x07payload\x18\x01 \x01(\x0b\x32\x17.accumulator.v1.Payload\x12\x45\n\toperation\x18\x02 \x01(\x0b\x32\x32.accumulator.v1.AccumulatorRequest.WindowOperation\x1a\xb6\x01\n\x0fWindowOperation\x12G\n\x05\x65vent\x18\x01 \x01(\x0e\x32\x38.accumulator.v1.AccumulatorRequest.WindowOperation.Event\x12\x30\n\x0bkeyedWindow\x18\x02 \x01(\x0b\x32\x1b.accumulator.v1.KeyedWindow"(\n\x05\x45vent\x12\x08\n\x04OPEN\x10\x00\x12\t\n\x05\x43LOSE\x10\x01\x12\n\n\x06\x41PPEND\x10\x02"}\n\x0bKeyedWindow\x12)\n\x05start\x18\x01 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\'\n\x03\x65nd\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04slot\x18\x03 \x01(\t\x12\x0c\n\x04keys\x18\x04 \x03(\t"\x87\x01\n\x13\x41\x63\x63umulatorResponse\x12(\n\x07payload\x18\x01 \x01(\x0b\x32\x17.accumulator.v1.Payload\x12+\n\x06window\x18\x02 \x01(\x0b\x32\x1b.accumulator.v1.KeyedWindow\x12\x0c\n\x04tags\x18\x03 \x03(\t\x12\x0b\n\x03\x45OF\x18\x04 \x01(\x08"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\xac\x01\n\x0b\x41\x63\x63umulator\x12[\n\x0c\x41\x63\x63umulateFn\x12".accumulator.v1.AccumulatorRequest\x1a#.accumulator.v1.AccumulatorResponse(\x01\x30\x01\x12@\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x1d.accumulator.v1.ReadyResponseBd\n#io.numaproj.numaflow.accumulator.v1Z=github.com/numaproj/numaflow-go/pkg/apis/proto/accumulator/v1b\x06proto3' -) +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x11\x61\x63\x63umulator.proto\x12\x0e\x61\x63\x63umulator.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto\"\xf8\x01\n\x07Payload\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\n\n\x02id\x18\x05 \x01(\t\x12\x35\n\x07headers\x18\x06 \x03(\x0b\x32$.accumulator.v1.Payload.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\"\xbe\x02\n\x12\x41\x63\x63umulatorRequest\x12(\n\x07payload\x18\x01 \x01(\x0b\x32\x17.accumulator.v1.Payload\x12\x45\n\toperation\x18\x02 \x01(\x0b\x32\x32.accumulator.v1.AccumulatorRequest.WindowOperation\x1a\xb6\x01\n\x0fWindowOperation\x12G\n\x05\x65vent\x18\x01 \x01(\x0e\x32\x38.accumulator.v1.AccumulatorRequest.WindowOperation.Event\x12\x30\n\x0bkeyedWindow\x18\x02 \x01(\x0b\x32\x1b.accumulator.v1.KeyedWindow\"(\n\x05\x45vent\x12\x08\n\x04OPEN\x10\x00\x12\t\n\x05\x43LOSE\x10\x01\x12\n\n\x06\x41PPEND\x10\x02\"}\n\x0bKeyedWindow\x12)\n\x05start\x18\x01 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\'\n\x03\x65nd\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04slot\x18\x03 \x01(\t\x12\x0c\n\x04keys\x18\x04 \x03(\t\"\x87\x01\n\x13\x41\x63\x63umulatorResponse\x12(\n\x07payload\x18\x01 \x01(\x0b\x32\x17.accumulator.v1.Payload\x12+\n\x06window\x18\x02 \x01(\x0b\x32\x1b.accumulator.v1.KeyedWindow\x12\x0c\n\x04tags\x18\x03 \x03(\t\x12\x0b\n\x03\x45OF\x18\x04 \x01(\x08\"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\xac\x01\n\x0b\x41\x63\x63umulator\x12[\n\x0c\x41\x63\x63umulateFn\x12\".accumulator.v1.AccumulatorRequest\x1a#.accumulator.v1.AccumulatorResponse(\x01\x30\x01\x12@\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x1d.accumulator.v1.ReadyResponseBd\n#io.numaproj.numaflow.accumulator.v1Z=github.com/numaproj/numaflow-go/pkg/apis/proto/accumulator/v1b\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) -_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, "accumulator_pb2", _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'accumulator_pb2', _globals) if _descriptor._USE_C_DESCRIPTORS == False: - _globals["DESCRIPTOR"]._options = None - _globals[ - "DESCRIPTOR" - ]._serialized_options = b"\n#io.numaproj.numaflow.accumulator.v1Z=github.com/numaproj/numaflow-go/pkg/apis/proto/accumulator/v1" - _globals["_PAYLOAD_HEADERSENTRY"]._options = None - _globals["_PAYLOAD_HEADERSENTRY"]._serialized_options = b"8\001" - _globals["_PAYLOAD"]._serialized_start = 100 - _globals["_PAYLOAD"]._serialized_end = 348 - _globals["_PAYLOAD_HEADERSENTRY"]._serialized_start = 302 - _globals["_PAYLOAD_HEADERSENTRY"]._serialized_end = 348 - _globals["_ACCUMULATORREQUEST"]._serialized_start = 351 - _globals["_ACCUMULATORREQUEST"]._serialized_end = 669 - _globals["_ACCUMULATORREQUEST_WINDOWOPERATION"]._serialized_start = 487 - _globals["_ACCUMULATORREQUEST_WINDOWOPERATION"]._serialized_end = 669 - _globals["_ACCUMULATORREQUEST_WINDOWOPERATION_EVENT"]._serialized_start = 629 - _globals["_ACCUMULATORREQUEST_WINDOWOPERATION_EVENT"]._serialized_end = 669 - _globals["_KEYEDWINDOW"]._serialized_start = 671 - _globals["_KEYEDWINDOW"]._serialized_end = 796 - _globals["_ACCUMULATORRESPONSE"]._serialized_start = 799 - _globals["_ACCUMULATORRESPONSE"]._serialized_end = 934 - _globals["_READYRESPONSE"]._serialized_start = 936 - _globals["_READYRESPONSE"]._serialized_end = 966 - _globals["_ACCUMULATOR"]._serialized_start = 969 - _globals["_ACCUMULATOR"]._serialized_end = 1141 + _globals['DESCRIPTOR']._options = None + _globals['DESCRIPTOR']._serialized_options = b'\n#io.numaproj.numaflow.accumulator.v1Z=github.com/numaproj/numaflow-go/pkg/apis/proto/accumulator/v1' + _globals['_PAYLOAD_HEADERSENTRY']._options = None + _globals['_PAYLOAD_HEADERSENTRY']._serialized_options = b'8\001' + _globals['_PAYLOAD']._serialized_start=100 + _globals['_PAYLOAD']._serialized_end=348 + _globals['_PAYLOAD_HEADERSENTRY']._serialized_start=302 + _globals['_PAYLOAD_HEADERSENTRY']._serialized_end=348 + _globals['_ACCUMULATORREQUEST']._serialized_start=351 + _globals['_ACCUMULATORREQUEST']._serialized_end=669 + _globals['_ACCUMULATORREQUEST_WINDOWOPERATION']._serialized_start=487 + _globals['_ACCUMULATORREQUEST_WINDOWOPERATION']._serialized_end=669 + _globals['_ACCUMULATORREQUEST_WINDOWOPERATION_EVENT']._serialized_start=629 + _globals['_ACCUMULATORREQUEST_WINDOWOPERATION_EVENT']._serialized_end=669 + _globals['_KEYEDWINDOW']._serialized_start=671 + _globals['_KEYEDWINDOW']._serialized_end=796 + _globals['_ACCUMULATORRESPONSE']._serialized_start=799 + _globals['_ACCUMULATORRESPONSE']._serialized_end=934 + _globals['_READYRESPONSE']._serialized_start=936 + _globals['_READYRESPONSE']._serialized_end=966 + _globals['_ACCUMULATOR']._serialized_start=969 + _globals['_ACCUMULATOR']._serialized_end=1141 # @@protoc_insertion_point(module_scope) diff --git a/pynumaflow/proto/accumulator/accumulator_pb2.pyi b/pynumaflow/proto/accumulator/accumulator_pb2.pyi index d9f0f7a5..bfa0bbac 100644 --- a/pynumaflow/proto/accumulator/accumulator_pb2.pyi +++ b/pynumaflow/proto/accumulator/accumulator_pb2.pyi @@ -4,19 +4,12 @@ from google.protobuf.internal import containers as _containers from google.protobuf.internal import enum_type_wrapper as _enum_type_wrapper from google.protobuf import descriptor as _descriptor from google.protobuf import message as _message -from typing import ( - ClassVar as _ClassVar, - Iterable as _Iterable, - Mapping as _Mapping, - Optional as _Optional, - Union as _Union, -) +from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union DESCRIPTOR: _descriptor.FileDescriptor class Payload(_message.Message): __slots__ = ("keys", "value", "event_time", "watermark", "id", "headers") - class HeadersEntry(_message.Message): __slots__ = ("key", "value") KEY_FIELD_NUMBER: _ClassVar[int] @@ -36,22 +29,12 @@ class Payload(_message.Message): watermark: _timestamp_pb2.Timestamp id: str headers: _containers.ScalarMap[str, str] - def __init__( - self, - keys: _Optional[_Iterable[str]] = ..., - value: _Optional[bytes] = ..., - event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - id: _Optional[str] = ..., - headers: _Optional[_Mapping[str, str]] = ..., - ) -> None: ... + def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., id: _Optional[str] = ..., headers: _Optional[_Mapping[str, str]] = ...) -> None: ... class AccumulatorRequest(_message.Message): __slots__ = ("payload", "operation") - class WindowOperation(_message.Message): __slots__ = ("event", "keyedWindow") - class Event(int, metaclass=_enum_type_wrapper.EnumTypeWrapper): __slots__ = () OPEN: _ClassVar[AccumulatorRequest.WindowOperation.Event] @@ -64,20 +47,12 @@ class AccumulatorRequest(_message.Message): KEYEDWINDOW_FIELD_NUMBER: _ClassVar[int] event: AccumulatorRequest.WindowOperation.Event keyedWindow: KeyedWindow - def __init__( - self, - event: _Optional[_Union[AccumulatorRequest.WindowOperation.Event, str]] = ..., - keyedWindow: _Optional[_Union[KeyedWindow, _Mapping]] = ..., - ) -> None: ... + def __init__(self, event: _Optional[_Union[AccumulatorRequest.WindowOperation.Event, str]] = ..., keyedWindow: _Optional[_Union[KeyedWindow, _Mapping]] = ...) -> None: ... PAYLOAD_FIELD_NUMBER: _ClassVar[int] OPERATION_FIELD_NUMBER: _ClassVar[int] payload: Payload operation: AccumulatorRequest.WindowOperation - def __init__( - self, - payload: _Optional[_Union[Payload, _Mapping]] = ..., - operation: _Optional[_Union[AccumulatorRequest.WindowOperation, _Mapping]] = ..., - ) -> None: ... + def __init__(self, payload: _Optional[_Union[Payload, _Mapping]] = ..., operation: _Optional[_Union[AccumulatorRequest.WindowOperation, _Mapping]] = ...) -> None: ... class KeyedWindow(_message.Message): __slots__ = ("start", "end", "slot", "keys") @@ -89,13 +64,7 @@ class KeyedWindow(_message.Message): end: _timestamp_pb2.Timestamp slot: str keys: _containers.RepeatedScalarFieldContainer[str] - def __init__( - self, - start: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - end: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - slot: _Optional[str] = ..., - keys: _Optional[_Iterable[str]] = ..., - ) -> None: ... + def __init__(self, start: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., end: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., slot: _Optional[str] = ..., keys: _Optional[_Iterable[str]] = ...) -> None: ... class AccumulatorResponse(_message.Message): __slots__ = ("payload", "window", "tags", "EOF") @@ -107,13 +76,7 @@ class AccumulatorResponse(_message.Message): window: KeyedWindow tags: _containers.RepeatedScalarFieldContainer[str] EOF: bool - def __init__( - self, - payload: _Optional[_Union[Payload, _Mapping]] = ..., - window: _Optional[_Union[KeyedWindow, _Mapping]] = ..., - tags: _Optional[_Iterable[str]] = ..., - EOF: bool = ..., - ) -> None: ... + def __init__(self, payload: _Optional[_Union[Payload, _Mapping]] = ..., window: _Optional[_Union[KeyedWindow, _Mapping]] = ..., tags: _Optional[_Iterable[str]] = ..., EOF: bool = ...) -> None: ... class ReadyResponse(_message.Message): __slots__ = ("ready",) diff --git a/pynumaflow/proto/accumulator/accumulator_pb2_grpc.py b/pynumaflow/proto/accumulator/accumulator_pb2_grpc.py index f41606dd..4f0f208f 100644 --- a/pynumaflow/proto/accumulator/accumulator_pb2_grpc.py +++ b/pynumaflow/proto/accumulator/accumulator_pb2_grpc.py @@ -19,15 +19,15 @@ def __init__(self, channel): channel: A grpc.Channel. """ self.AccumulateFn = channel.stream_stream( - "/accumulator.v1.Accumulator/AccumulateFn", - request_serializer=accumulator__pb2.AccumulatorRequest.SerializeToString, - response_deserializer=accumulator__pb2.AccumulatorResponse.FromString, - ) + '/accumulator.v1.Accumulator/AccumulateFn', + request_serializer=accumulator__pb2.AccumulatorRequest.SerializeToString, + response_deserializer=accumulator__pb2.AccumulatorResponse.FromString, + ) self.IsReady = channel.unary_unary( - "/accumulator.v1.Accumulator/IsReady", - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=accumulator__pb2.ReadyResponse.FromString, - ) + '/accumulator.v1.Accumulator/IsReady', + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=accumulator__pb2.ReadyResponse.FromString, + ) class AccumulatorServicer(object): @@ -37,38 +37,39 @@ class AccumulatorServicer(object): """ def AccumulateFn(self, request_iterator, context): - """AccumulateFn applies a accumulate function to a request stream.""" + """AccumulateFn applies a accumulate function to a request stream. + """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def IsReady(self, request, context): - """IsReady is the heartbeat endpoint for gRPC.""" + """IsReady is the heartbeat endpoint for gRPC. + """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def add_AccumulatorServicer_to_server(servicer, server): rpc_method_handlers = { - "AccumulateFn": grpc.stream_stream_rpc_method_handler( - servicer.AccumulateFn, - request_deserializer=accumulator__pb2.AccumulatorRequest.FromString, - response_serializer=accumulator__pb2.AccumulatorResponse.SerializeToString, - ), - "IsReady": grpc.unary_unary_rpc_method_handler( - servicer.IsReady, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=accumulator__pb2.ReadyResponse.SerializeToString, - ), + 'AccumulateFn': grpc.stream_stream_rpc_method_handler( + servicer.AccumulateFn, + request_deserializer=accumulator__pb2.AccumulatorRequest.FromString, + response_serializer=accumulator__pb2.AccumulatorResponse.SerializeToString, + ), + 'IsReady': grpc.unary_unary_rpc_method_handler( + servicer.IsReady, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=accumulator__pb2.ReadyResponse.SerializeToString, + ), } generic_handler = grpc.method_handlers_generic_handler( - "accumulator.v1.Accumulator", rpc_method_handlers - ) + 'accumulator.v1.Accumulator', rpc_method_handlers) server.add_generic_rpc_handlers((generic_handler,)) -# This class is part of an EXPERIMENTAL API. + # This class is part of an EXPERIMENTAL API. class Accumulator(object): """AccumulatorWindow describes a special kind of SessionWindow (similar to Global Window) where output should always have monotonically increasing WM but it can be manipulated through event-time by reordering the messages. @@ -76,59 +77,35 @@ class Accumulator(object): """ @staticmethod - def AccumulateFn( - request_iterator, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.stream_stream( - request_iterator, + def AccumulateFn(request_iterator, target, - "/accumulator.v1.Accumulator/AccumulateFn", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.stream_stream(request_iterator, target, '/accumulator.v1.Accumulator/AccumulateFn', accumulator__pb2.AccumulatorRequest.SerializeToString, accumulator__pb2.AccumulatorResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) @staticmethod - def IsReady( - request, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.unary_unary( - request, + def IsReady(request, target, - "/accumulator.v1.Accumulator/IsReady", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.unary_unary(request, target, '/accumulator.v1.Accumulator/IsReady', google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, accumulator__pb2.ReadyResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) diff --git a/pynumaflow/proto/mapper/map_pb2.py b/pynumaflow/proto/mapper/map_pb2.py index 80472786..78abbd69 100644 --- a/pynumaflow/proto/mapper/map_pb2.py +++ b/pynumaflow/proto/mapper/map_pb2.py @@ -7,7 +7,6 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import symbol_database as _symbol_database from google.protobuf.internal import builder as _builder - # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() @@ -17,36 +16,32 @@ from google.protobuf import timestamp_pb2 as google_dot_protobuf_dot_timestamp__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\tmap.proto\x12\x06map.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto"\xac\x03\n\nMapRequest\x12+\n\x07request\x18\x01 \x01(\x0b\x32\x1a.map.v1.MapRequest.Request\x12\n\n\x02id\x18\x02 \x01(\t\x12)\n\thandshake\x18\x03 \x01(\x0b\x32\x11.map.v1.HandshakeH\x00\x88\x01\x01\x12/\n\x06status\x18\x04 \x01(\x0b\x32\x1a.map.v1.TransmissionStatusH\x01\x88\x01\x01\x1a\xef\x01\n\x07Request\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x38\n\x07headers\x18\x05 \x03(\x0b\x32\'.map.v1.MapRequest.Request.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x42\x0c\n\n_handshakeB\t\n\x07_status"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08"!\n\x12TransmissionStatus\x12\x0b\n\x03\x65ot\x18\x01 \x01(\x08"\xf0\x01\n\x0bMapResponse\x12+\n\x07results\x18\x01 \x03(\x0b\x32\x1a.map.v1.MapResponse.Result\x12\n\n\x02id\x18\x02 \x01(\t\x12)\n\thandshake\x18\x03 \x01(\x0b\x32\x11.map.v1.HandshakeH\x00\x88\x01\x01\x12/\n\x06status\x18\x04 \x01(\x0b\x32\x1a.map.v1.TransmissionStatusH\x01\x88\x01\x01\x1a\x33\n\x06Result\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12\x0c\n\x04tags\x18\x03 \x03(\tB\x0c\n\n_handshakeB\t\n\x07_status"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32u\n\x03Map\x12\x34\n\x05MapFn\x12\x12.map.v1.MapRequest\x1a\x13.map.v1.MapResponse(\x01\x30\x01\x12\x38\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x15.map.v1.ReadyResponseB7Z5github.com/numaproj/numaflow-go/pkg/apis/proto/map/v1b\x06proto3' -) +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\tmap.proto\x12\x06map.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto\"\xac\x03\n\nMapRequest\x12+\n\x07request\x18\x01 \x01(\x0b\x32\x1a.map.v1.MapRequest.Request\x12\n\n\x02id\x18\x02 \x01(\t\x12)\n\thandshake\x18\x03 \x01(\x0b\x32\x11.map.v1.HandshakeH\x00\x88\x01\x01\x12/\n\x06status\x18\x04 \x01(\x0b\x32\x1a.map.v1.TransmissionStatusH\x01\x88\x01\x01\x1a\xef\x01\n\x07Request\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x38\n\x07headers\x18\x05 \x03(\x0b\x32\'.map.v1.MapRequest.Request.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x42\x0c\n\n_handshakeB\t\n\x07_status\"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08\"!\n\x12TransmissionStatus\x12\x0b\n\x03\x65ot\x18\x01 \x01(\x08\"\xf0\x01\n\x0bMapResponse\x12+\n\x07results\x18\x01 \x03(\x0b\x32\x1a.map.v1.MapResponse.Result\x12\n\n\x02id\x18\x02 \x01(\t\x12)\n\thandshake\x18\x03 \x01(\x0b\x32\x11.map.v1.HandshakeH\x00\x88\x01\x01\x12/\n\x06status\x18\x04 \x01(\x0b\x32\x1a.map.v1.TransmissionStatusH\x01\x88\x01\x01\x1a\x33\n\x06Result\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12\x0c\n\x04tags\x18\x03 \x03(\tB\x0c\n\n_handshakeB\t\n\x07_status\"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32u\n\x03Map\x12\x34\n\x05MapFn\x12\x12.map.v1.MapRequest\x1a\x13.map.v1.MapResponse(\x01\x30\x01\x12\x38\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x15.map.v1.ReadyResponseB7Z5github.com/numaproj/numaflow-go/pkg/apis/proto/map/v1b\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) -_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, "map_pb2", _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'map_pb2', _globals) if _descriptor._USE_C_DESCRIPTORS == False: - _globals["DESCRIPTOR"]._options = None - _globals[ - "DESCRIPTOR" - ]._serialized_options = b"Z5github.com/numaproj/numaflow-go/pkg/apis/proto/map/v1" - _globals["_MAPREQUEST_REQUEST_HEADERSENTRY"]._options = None - _globals["_MAPREQUEST_REQUEST_HEADERSENTRY"]._serialized_options = b"8\001" - _globals["_MAPREQUEST"]._serialized_start = 84 - _globals["_MAPREQUEST"]._serialized_end = 512 - _globals["_MAPREQUEST_REQUEST"]._serialized_start = 248 - _globals["_MAPREQUEST_REQUEST"]._serialized_end = 487 - _globals["_MAPREQUEST_REQUEST_HEADERSENTRY"]._serialized_start = 441 - _globals["_MAPREQUEST_REQUEST_HEADERSENTRY"]._serialized_end = 487 - _globals["_HANDSHAKE"]._serialized_start = 514 - _globals["_HANDSHAKE"]._serialized_end = 538 - _globals["_TRANSMISSIONSTATUS"]._serialized_start = 540 - _globals["_TRANSMISSIONSTATUS"]._serialized_end = 573 - _globals["_MAPRESPONSE"]._serialized_start = 576 - _globals["_MAPRESPONSE"]._serialized_end = 816 - _globals["_MAPRESPONSE_RESULT"]._serialized_start = 740 - _globals["_MAPRESPONSE_RESULT"]._serialized_end = 791 - _globals["_READYRESPONSE"]._serialized_start = 818 - _globals["_READYRESPONSE"]._serialized_end = 848 - _globals["_MAP"]._serialized_start = 850 - _globals["_MAP"]._serialized_end = 967 + _globals['DESCRIPTOR']._options = None + _globals['DESCRIPTOR']._serialized_options = b'Z5github.com/numaproj/numaflow-go/pkg/apis/proto/map/v1' + _globals['_MAPREQUEST_REQUEST_HEADERSENTRY']._options = None + _globals['_MAPREQUEST_REQUEST_HEADERSENTRY']._serialized_options = b'8\001' + _globals['_MAPREQUEST']._serialized_start=84 + _globals['_MAPREQUEST']._serialized_end=512 + _globals['_MAPREQUEST_REQUEST']._serialized_start=248 + _globals['_MAPREQUEST_REQUEST']._serialized_end=487 + _globals['_MAPREQUEST_REQUEST_HEADERSENTRY']._serialized_start=441 + _globals['_MAPREQUEST_REQUEST_HEADERSENTRY']._serialized_end=487 + _globals['_HANDSHAKE']._serialized_start=514 + _globals['_HANDSHAKE']._serialized_end=538 + _globals['_TRANSMISSIONSTATUS']._serialized_start=540 + _globals['_TRANSMISSIONSTATUS']._serialized_end=573 + _globals['_MAPRESPONSE']._serialized_start=576 + _globals['_MAPRESPONSE']._serialized_end=816 + _globals['_MAPRESPONSE_RESULT']._serialized_start=740 + _globals['_MAPRESPONSE_RESULT']._serialized_end=791 + _globals['_READYRESPONSE']._serialized_start=818 + _globals['_READYRESPONSE']._serialized_end=848 + _globals['_MAP']._serialized_start=850 + _globals['_MAP']._serialized_end=967 # @@protoc_insertion_point(module_scope) diff --git a/pynumaflow/proto/mapper/map_pb2.pyi b/pynumaflow/proto/mapper/map_pb2.pyi index 9832bc3e..22eed7b9 100644 --- a/pynumaflow/proto/mapper/map_pb2.pyi +++ b/pynumaflow/proto/mapper/map_pb2.pyi @@ -3,22 +3,14 @@ from google.protobuf import timestamp_pb2 as _timestamp_pb2 from google.protobuf.internal import containers as _containers from google.protobuf import descriptor as _descriptor from google.protobuf import message as _message -from typing import ( - ClassVar as _ClassVar, - Iterable as _Iterable, - Mapping as _Mapping, - Optional as _Optional, - Union as _Union, -) +from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union DESCRIPTOR: _descriptor.FileDescriptor class MapRequest(_message.Message): __slots__ = ("request", "id", "handshake", "status") - class Request(_message.Message): __slots__ = ("keys", "value", "event_time", "watermark", "headers") - class HeadersEntry(_message.Message): __slots__ = ("key", "value") KEY_FIELD_NUMBER: _ClassVar[int] @@ -36,14 +28,7 @@ class MapRequest(_message.Message): event_time: _timestamp_pb2.Timestamp watermark: _timestamp_pb2.Timestamp headers: _containers.ScalarMap[str, str] - def __init__( - self, - keys: _Optional[_Iterable[str]] = ..., - value: _Optional[bytes] = ..., - event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - headers: _Optional[_Mapping[str, str]] = ..., - ) -> None: ... + def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., headers: _Optional[_Mapping[str, str]] = ...) -> None: ... REQUEST_FIELD_NUMBER: _ClassVar[int] ID_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] @@ -52,13 +37,7 @@ class MapRequest(_message.Message): id: str handshake: Handshake status: TransmissionStatus - def __init__( - self, - request: _Optional[_Union[MapRequest.Request, _Mapping]] = ..., - id: _Optional[str] = ..., - handshake: _Optional[_Union[Handshake, _Mapping]] = ..., - status: _Optional[_Union[TransmissionStatus, _Mapping]] = ..., - ) -> None: ... + def __init__(self, request: _Optional[_Union[MapRequest.Request, _Mapping]] = ..., id: _Optional[str] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ..., status: _Optional[_Union[TransmissionStatus, _Mapping]] = ...) -> None: ... class Handshake(_message.Message): __slots__ = ("sot",) @@ -74,7 +53,6 @@ class TransmissionStatus(_message.Message): class MapResponse(_message.Message): __slots__ = ("results", "id", "handshake", "status") - class Result(_message.Message): __slots__ = ("keys", "value", "tags") KEYS_FIELD_NUMBER: _ClassVar[int] @@ -83,12 +61,7 @@ class MapResponse(_message.Message): keys: _containers.RepeatedScalarFieldContainer[str] value: bytes tags: _containers.RepeatedScalarFieldContainer[str] - def __init__( - self, - keys: _Optional[_Iterable[str]] = ..., - value: _Optional[bytes] = ..., - tags: _Optional[_Iterable[str]] = ..., - ) -> None: ... + def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., tags: _Optional[_Iterable[str]] = ...) -> None: ... RESULTS_FIELD_NUMBER: _ClassVar[int] ID_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] @@ -97,13 +70,7 @@ class MapResponse(_message.Message): id: str handshake: Handshake status: TransmissionStatus - def __init__( - self, - results: _Optional[_Iterable[_Union[MapResponse.Result, _Mapping]]] = ..., - id: _Optional[str] = ..., - handshake: _Optional[_Union[Handshake, _Mapping]] = ..., - status: _Optional[_Union[TransmissionStatus, _Mapping]] = ..., - ) -> None: ... + def __init__(self, results: _Optional[_Iterable[_Union[MapResponse.Result, _Mapping]]] = ..., id: _Optional[str] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ..., status: _Optional[_Union[TransmissionStatus, _Mapping]] = ...) -> None: ... class ReadyResponse(_message.Message): __slots__ = ("ready",) diff --git a/pynumaflow/proto/mapper/map_pb2_grpc.py b/pynumaflow/proto/mapper/map_pb2_grpc.py index 97834865..87fa0b27 100644 --- a/pynumaflow/proto/mapper/map_pb2_grpc.py +++ b/pynumaflow/proto/mapper/map_pb2_grpc.py @@ -16,108 +16,87 @@ def __init__(self, channel): channel: A grpc.Channel. """ self.MapFn = channel.stream_stream( - "/map.v1.Map/MapFn", - request_serializer=map__pb2.MapRequest.SerializeToString, - response_deserializer=map__pb2.MapResponse.FromString, - ) + '/map.v1.Map/MapFn', + request_serializer=map__pb2.MapRequest.SerializeToString, + response_deserializer=map__pb2.MapResponse.FromString, + ) self.IsReady = channel.unary_unary( - "/map.v1.Map/IsReady", - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=map__pb2.ReadyResponse.FromString, - ) + '/map.v1.Map/IsReady', + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=map__pb2.ReadyResponse.FromString, + ) class MapServicer(object): """Missing associated documentation comment in .proto file.""" def MapFn(self, request_iterator, context): - """MapFn applies a function to each map request element.""" + """MapFn applies a function to each map request element. + """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def IsReady(self, request, context): - """IsReady is the heartbeat endpoint for gRPC.""" + """IsReady is the heartbeat endpoint for gRPC. + """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def add_MapServicer_to_server(servicer, server): rpc_method_handlers = { - "MapFn": grpc.stream_stream_rpc_method_handler( - servicer.MapFn, - request_deserializer=map__pb2.MapRequest.FromString, - response_serializer=map__pb2.MapResponse.SerializeToString, - ), - "IsReady": grpc.unary_unary_rpc_method_handler( - servicer.IsReady, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=map__pb2.ReadyResponse.SerializeToString, - ), + 'MapFn': grpc.stream_stream_rpc_method_handler( + servicer.MapFn, + request_deserializer=map__pb2.MapRequest.FromString, + response_serializer=map__pb2.MapResponse.SerializeToString, + ), + 'IsReady': grpc.unary_unary_rpc_method_handler( + servicer.IsReady, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=map__pb2.ReadyResponse.SerializeToString, + ), } - generic_handler = grpc.method_handlers_generic_handler("map.v1.Map", rpc_method_handlers) + generic_handler = grpc.method_handlers_generic_handler( + 'map.v1.Map', rpc_method_handlers) server.add_generic_rpc_handlers((generic_handler,)) -# This class is part of an EXPERIMENTAL API. + # This class is part of an EXPERIMENTAL API. class Map(object): """Missing associated documentation comment in .proto file.""" @staticmethod - def MapFn( - request_iterator, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.stream_stream( - request_iterator, + def MapFn(request_iterator, target, - "/map.v1.Map/MapFn", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.stream_stream(request_iterator, target, '/map.v1.Map/MapFn', map__pb2.MapRequest.SerializeToString, map__pb2.MapResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) @staticmethod - def IsReady( - request, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.unary_unary( - request, + def IsReady(request, target, - "/map.v1.Map/IsReady", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.unary_unary(request, target, '/map.v1.Map/IsReady', google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, map__pb2.ReadyResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) diff --git a/pynumaflow/proto/reducer/reduce_pb2.py b/pynumaflow/proto/reducer/reduce_pb2.py index f5c1ef83..ecb076e3 100644 --- a/pynumaflow/proto/reducer/reduce_pb2.py +++ b/pynumaflow/proto/reducer/reduce_pb2.py @@ -7,7 +7,6 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import symbol_database as _symbol_database from google.protobuf.internal import builder as _builder - # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() @@ -17,35 +16,33 @@ from google.protobuf import timestamp_pb2 as google_dot_protobuf_dot_timestamp__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x0creduce.proto\x12\treduce.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto"\x98\x04\n\rReduceRequest\x12\x31\n\x07payload\x18\x01 \x01(\x0b\x32 .reduce.v1.ReduceRequest.Payload\x12;\n\toperation\x18\x02 \x01(\x0b\x32(.reduce.v1.ReduceRequest.WindowOperation\x1a\x9e\x01\n\x0fWindowOperation\x12=\n\x05\x65vent\x18\x01 \x01(\x0e\x32..reduce.v1.ReduceRequest.WindowOperation.Event\x12"\n\x07windows\x18\x02 \x03(\x0b\x32\x11.reduce.v1.Window"(\n\x05\x45vent\x12\x08\n\x04OPEN\x10\x00\x12\t\n\x05\x43LOSE\x10\x01\x12\n\n\x06\x41PPEND\x10\x04\x1a\xf5\x01\n\x07Payload\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12>\n\x07headers\x18\x05 \x03(\x0b\x32-.reduce.v1.ReduceRequest.Payload.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01"j\n\x06Window\x12)\n\x05start\x18\x01 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\'\n\x03\x65nd\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04slot\x18\x03 \x01(\t"\xa7\x01\n\x0eReduceResponse\x12\x30\n\x06result\x18\x01 \x01(\x0b\x32 .reduce.v1.ReduceResponse.Result\x12!\n\x06window\x18\x02 \x01(\x0b\x32\x11.reduce.v1.Window\x12\x0b\n\x03\x45OF\x18\x03 \x01(\x08\x1a\x33\n\x06Result\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12\x0c\n\x04tags\x18\x03 \x03(\t"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\x8a\x01\n\x06Reduce\x12\x43\n\x08ReduceFn\x12\x18.reduce.v1.ReduceRequest\x1a\x19.reduce.v1.ReduceResponse(\x01\x30\x01\x12;\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x18.reduce.v1.ReadyResponseb\x06proto3' -) +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x0creduce.proto\x12\treduce.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto\"\x98\x04\n\rReduceRequest\x12\x31\n\x07payload\x18\x01 \x01(\x0b\x32 .reduce.v1.ReduceRequest.Payload\x12;\n\toperation\x18\x02 \x01(\x0b\x32(.reduce.v1.ReduceRequest.WindowOperation\x1a\x9e\x01\n\x0fWindowOperation\x12=\n\x05\x65vent\x18\x01 \x01(\x0e\x32..reduce.v1.ReduceRequest.WindowOperation.Event\x12\"\n\x07windows\x18\x02 \x03(\x0b\x32\x11.reduce.v1.Window\"(\n\x05\x45vent\x12\x08\n\x04OPEN\x10\x00\x12\t\n\x05\x43LOSE\x10\x01\x12\n\n\x06\x41PPEND\x10\x04\x1a\xf5\x01\n\x07Payload\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12>\n\x07headers\x18\x05 \x03(\x0b\x32-.reduce.v1.ReduceRequest.Payload.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\"j\n\x06Window\x12)\n\x05start\x18\x01 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\'\n\x03\x65nd\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04slot\x18\x03 \x01(\t\"\xa7\x01\n\x0eReduceResponse\x12\x30\n\x06result\x18\x01 \x01(\x0b\x32 .reduce.v1.ReduceResponse.Result\x12!\n\x06window\x18\x02 \x01(\x0b\x32\x11.reduce.v1.Window\x12\x0b\n\x03\x45OF\x18\x03 \x01(\x08\x1a\x33\n\x06Result\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12\x0c\n\x04tags\x18\x03 \x03(\t\"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\x8a\x01\n\x06Reduce\x12\x43\n\x08ReduceFn\x12\x18.reduce.v1.ReduceRequest\x1a\x19.reduce.v1.ReduceResponse(\x01\x30\x01\x12;\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x18.reduce.v1.ReadyResponseb\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) -_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, "reduce_pb2", _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'reduce_pb2', _globals) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None - _globals["_REDUCEREQUEST_PAYLOAD_HEADERSENTRY"]._options = None - _globals["_REDUCEREQUEST_PAYLOAD_HEADERSENTRY"]._serialized_options = b"8\001" - _globals["_REDUCEREQUEST"]._serialized_start = 90 - _globals["_REDUCEREQUEST"]._serialized_end = 626 - _globals["_REDUCEREQUEST_WINDOWOPERATION"]._serialized_start = 220 - _globals["_REDUCEREQUEST_WINDOWOPERATION"]._serialized_end = 378 - _globals["_REDUCEREQUEST_WINDOWOPERATION_EVENT"]._serialized_start = 338 - _globals["_REDUCEREQUEST_WINDOWOPERATION_EVENT"]._serialized_end = 378 - _globals["_REDUCEREQUEST_PAYLOAD"]._serialized_start = 381 - _globals["_REDUCEREQUEST_PAYLOAD"]._serialized_end = 626 - _globals["_REDUCEREQUEST_PAYLOAD_HEADERSENTRY"]._serialized_start = 580 - _globals["_REDUCEREQUEST_PAYLOAD_HEADERSENTRY"]._serialized_end = 626 - _globals["_WINDOW"]._serialized_start = 628 - _globals["_WINDOW"]._serialized_end = 734 - _globals["_REDUCERESPONSE"]._serialized_start = 737 - _globals["_REDUCERESPONSE"]._serialized_end = 904 - _globals["_REDUCERESPONSE_RESULT"]._serialized_start = 853 - _globals["_REDUCERESPONSE_RESULT"]._serialized_end = 904 - _globals["_READYRESPONSE"]._serialized_start = 906 - _globals["_READYRESPONSE"]._serialized_end = 936 - _globals["_REDUCE"]._serialized_start = 939 - _globals["_REDUCE"]._serialized_end = 1077 + DESCRIPTOR._options = None + _globals['_REDUCEREQUEST_PAYLOAD_HEADERSENTRY']._options = None + _globals['_REDUCEREQUEST_PAYLOAD_HEADERSENTRY']._serialized_options = b'8\001' + _globals['_REDUCEREQUEST']._serialized_start=90 + _globals['_REDUCEREQUEST']._serialized_end=626 + _globals['_REDUCEREQUEST_WINDOWOPERATION']._serialized_start=220 + _globals['_REDUCEREQUEST_WINDOWOPERATION']._serialized_end=378 + _globals['_REDUCEREQUEST_WINDOWOPERATION_EVENT']._serialized_start=338 + _globals['_REDUCEREQUEST_WINDOWOPERATION_EVENT']._serialized_end=378 + _globals['_REDUCEREQUEST_PAYLOAD']._serialized_start=381 + _globals['_REDUCEREQUEST_PAYLOAD']._serialized_end=626 + _globals['_REDUCEREQUEST_PAYLOAD_HEADERSENTRY']._serialized_start=580 + _globals['_REDUCEREQUEST_PAYLOAD_HEADERSENTRY']._serialized_end=626 + _globals['_WINDOW']._serialized_start=628 + _globals['_WINDOW']._serialized_end=734 + _globals['_REDUCERESPONSE']._serialized_start=737 + _globals['_REDUCERESPONSE']._serialized_end=904 + _globals['_REDUCERESPONSE_RESULT']._serialized_start=853 + _globals['_REDUCERESPONSE_RESULT']._serialized_end=904 + _globals['_READYRESPONSE']._serialized_start=906 + _globals['_READYRESPONSE']._serialized_end=936 + _globals['_REDUCE']._serialized_start=939 + _globals['_REDUCE']._serialized_end=1077 # @@protoc_insertion_point(module_scope) diff --git a/pynumaflow/proto/reducer/reduce_pb2.pyi b/pynumaflow/proto/reducer/reduce_pb2.pyi index 88b27d53..80a974ff 100644 --- a/pynumaflow/proto/reducer/reduce_pb2.pyi +++ b/pynumaflow/proto/reducer/reduce_pb2.pyi @@ -4,22 +4,14 @@ from google.protobuf.internal import containers as _containers from google.protobuf.internal import enum_type_wrapper as _enum_type_wrapper from google.protobuf import descriptor as _descriptor from google.protobuf import message as _message -from typing import ( - ClassVar as _ClassVar, - Iterable as _Iterable, - Mapping as _Mapping, - Optional as _Optional, - Union as _Union, -) +from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union DESCRIPTOR: _descriptor.FileDescriptor class ReduceRequest(_message.Message): __slots__ = ("payload", "operation") - class WindowOperation(_message.Message): __slots__ = ("event", "windows") - class Event(int, metaclass=_enum_type_wrapper.EnumTypeWrapper): __slots__ = () OPEN: _ClassVar[ReduceRequest.WindowOperation.Event] @@ -32,15 +24,9 @@ class ReduceRequest(_message.Message): WINDOWS_FIELD_NUMBER: _ClassVar[int] event: ReduceRequest.WindowOperation.Event windows: _containers.RepeatedCompositeFieldContainer[Window] - def __init__( - self, - event: _Optional[_Union[ReduceRequest.WindowOperation.Event, str]] = ..., - windows: _Optional[_Iterable[_Union[Window, _Mapping]]] = ..., - ) -> None: ... - + def __init__(self, event: _Optional[_Union[ReduceRequest.WindowOperation.Event, str]] = ..., windows: _Optional[_Iterable[_Union[Window, _Mapping]]] = ...) -> None: ... class Payload(_message.Message): __slots__ = ("keys", "value", "event_time", "watermark", "headers") - class HeadersEntry(_message.Message): __slots__ = ("key", "value") KEY_FIELD_NUMBER: _ClassVar[int] @@ -58,23 +44,12 @@ class ReduceRequest(_message.Message): event_time: _timestamp_pb2.Timestamp watermark: _timestamp_pb2.Timestamp headers: _containers.ScalarMap[str, str] - def __init__( - self, - keys: _Optional[_Iterable[str]] = ..., - value: _Optional[bytes] = ..., - event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - headers: _Optional[_Mapping[str, str]] = ..., - ) -> None: ... + def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., headers: _Optional[_Mapping[str, str]] = ...) -> None: ... PAYLOAD_FIELD_NUMBER: _ClassVar[int] OPERATION_FIELD_NUMBER: _ClassVar[int] payload: ReduceRequest.Payload operation: ReduceRequest.WindowOperation - def __init__( - self, - payload: _Optional[_Union[ReduceRequest.Payload, _Mapping]] = ..., - operation: _Optional[_Union[ReduceRequest.WindowOperation, _Mapping]] = ..., - ) -> None: ... + def __init__(self, payload: _Optional[_Union[ReduceRequest.Payload, _Mapping]] = ..., operation: _Optional[_Union[ReduceRequest.WindowOperation, _Mapping]] = ...) -> None: ... class Window(_message.Message): __slots__ = ("start", "end", "slot") @@ -84,16 +59,10 @@ class Window(_message.Message): start: _timestamp_pb2.Timestamp end: _timestamp_pb2.Timestamp slot: str - def __init__( - self, - start: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - end: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - slot: _Optional[str] = ..., - ) -> None: ... + def __init__(self, start: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., end: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., slot: _Optional[str] = ...) -> None: ... class ReduceResponse(_message.Message): __slots__ = ("result", "window", "EOF") - class Result(_message.Message): __slots__ = ("keys", "value", "tags") KEYS_FIELD_NUMBER: _ClassVar[int] @@ -102,24 +71,14 @@ class ReduceResponse(_message.Message): keys: _containers.RepeatedScalarFieldContainer[str] value: bytes tags: _containers.RepeatedScalarFieldContainer[str] - def __init__( - self, - keys: _Optional[_Iterable[str]] = ..., - value: _Optional[bytes] = ..., - tags: _Optional[_Iterable[str]] = ..., - ) -> None: ... + def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., tags: _Optional[_Iterable[str]] = ...) -> None: ... RESULT_FIELD_NUMBER: _ClassVar[int] WINDOW_FIELD_NUMBER: _ClassVar[int] EOF_FIELD_NUMBER: _ClassVar[int] result: ReduceResponse.Result window: Window EOF: bool - def __init__( - self, - result: _Optional[_Union[ReduceResponse.Result, _Mapping]] = ..., - window: _Optional[_Union[Window, _Mapping]] = ..., - EOF: bool = ..., - ) -> None: ... + def __init__(self, result: _Optional[_Union[ReduceResponse.Result, _Mapping]] = ..., window: _Optional[_Union[Window, _Mapping]] = ..., EOF: bool = ...) -> None: ... class ReadyResponse(_message.Message): __slots__ = ("ready",) diff --git a/pynumaflow/proto/reducer/reduce_pb2_grpc.py b/pynumaflow/proto/reducer/reduce_pb2_grpc.py index 5a0a15f6..1fd860b4 100644 --- a/pynumaflow/proto/reducer/reduce_pb2_grpc.py +++ b/pynumaflow/proto/reducer/reduce_pb2_grpc.py @@ -16,108 +16,87 @@ def __init__(self, channel): channel: A grpc.Channel. """ self.ReduceFn = channel.stream_stream( - "/reduce.v1.Reduce/ReduceFn", - request_serializer=reduce__pb2.ReduceRequest.SerializeToString, - response_deserializer=reduce__pb2.ReduceResponse.FromString, - ) + '/reduce.v1.Reduce/ReduceFn', + request_serializer=reduce__pb2.ReduceRequest.SerializeToString, + response_deserializer=reduce__pb2.ReduceResponse.FromString, + ) self.IsReady = channel.unary_unary( - "/reduce.v1.Reduce/IsReady", - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=reduce__pb2.ReadyResponse.FromString, - ) + '/reduce.v1.Reduce/IsReady', + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=reduce__pb2.ReadyResponse.FromString, + ) class ReduceServicer(object): """Missing associated documentation comment in .proto file.""" def ReduceFn(self, request_iterator, context): - """ReduceFn applies a reduce function to a request stream.""" + """ReduceFn applies a reduce function to a request stream. + """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def IsReady(self, request, context): - """IsReady is the heartbeat endpoint for gRPC.""" + """IsReady is the heartbeat endpoint for gRPC. + """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def add_ReduceServicer_to_server(servicer, server): rpc_method_handlers = { - "ReduceFn": grpc.stream_stream_rpc_method_handler( - servicer.ReduceFn, - request_deserializer=reduce__pb2.ReduceRequest.FromString, - response_serializer=reduce__pb2.ReduceResponse.SerializeToString, - ), - "IsReady": grpc.unary_unary_rpc_method_handler( - servicer.IsReady, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=reduce__pb2.ReadyResponse.SerializeToString, - ), + 'ReduceFn': grpc.stream_stream_rpc_method_handler( + servicer.ReduceFn, + request_deserializer=reduce__pb2.ReduceRequest.FromString, + response_serializer=reduce__pb2.ReduceResponse.SerializeToString, + ), + 'IsReady': grpc.unary_unary_rpc_method_handler( + servicer.IsReady, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=reduce__pb2.ReadyResponse.SerializeToString, + ), } - generic_handler = grpc.method_handlers_generic_handler("reduce.v1.Reduce", rpc_method_handlers) + generic_handler = grpc.method_handlers_generic_handler( + 'reduce.v1.Reduce', rpc_method_handlers) server.add_generic_rpc_handlers((generic_handler,)) -# This class is part of an EXPERIMENTAL API. + # This class is part of an EXPERIMENTAL API. class Reduce(object): """Missing associated documentation comment in .proto file.""" @staticmethod - def ReduceFn( - request_iterator, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.stream_stream( - request_iterator, + def ReduceFn(request_iterator, target, - "/reduce.v1.Reduce/ReduceFn", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.stream_stream(request_iterator, target, '/reduce.v1.Reduce/ReduceFn', reduce__pb2.ReduceRequest.SerializeToString, reduce__pb2.ReduceResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) @staticmethod - def IsReady( - request, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.unary_unary( - request, + def IsReady(request, target, - "/reduce.v1.Reduce/IsReady", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.unary_unary(request, target, '/reduce.v1.Reduce/IsReady', google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, reduce__pb2.ReadyResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) diff --git a/pynumaflow/proto/sideinput/sideinput_pb2.py b/pynumaflow/proto/sideinput/sideinput_pb2.py index c0dab051..0160315f 100644 --- a/pynumaflow/proto/sideinput/sideinput_pb2.py +++ b/pynumaflow/proto/sideinput/sideinput_pb2.py @@ -7,7 +7,6 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import symbol_database as _symbol_database from google.protobuf.internal import builder as _builder - # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() @@ -16,19 +15,17 @@ from google.protobuf import empty_pb2 as google_dot_protobuf_dot_empty__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x0fsideinput.proto\x12\x0csideinput.v1\x1a\x1bgoogle/protobuf/empty.proto"8\n\x11SideInputResponse\x12\r\n\x05value\x18\x01 \x01(\x0c\x12\x14\n\x0cno_broadcast\x18\x02 \x01(\x08"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\x99\x01\n\tSideInput\x12L\n\x11RetrieveSideInput\x12\x16.google.protobuf.Empty\x1a\x1f.sideinput.v1.SideInputResponse\x12>\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x1b.sideinput.v1.ReadyResponseb\x06proto3' -) +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x0fsideinput.proto\x12\x0csideinput.v1\x1a\x1bgoogle/protobuf/empty.proto\"8\n\x11SideInputResponse\x12\r\n\x05value\x18\x01 \x01(\x0c\x12\x14\n\x0cno_broadcast\x18\x02 \x01(\x08\"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\x99\x01\n\tSideInput\x12L\n\x11RetrieveSideInput\x12\x16.google.protobuf.Empty\x1a\x1f.sideinput.v1.SideInputResponse\x12>\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x1b.sideinput.v1.ReadyResponseb\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) -_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, "sideinput_pb2", _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'sideinput_pb2', _globals) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None - _globals["_SIDEINPUTRESPONSE"]._serialized_start = 62 - _globals["_SIDEINPUTRESPONSE"]._serialized_end = 118 - _globals["_READYRESPONSE"]._serialized_start = 120 - _globals["_READYRESPONSE"]._serialized_end = 150 - _globals["_SIDEINPUT"]._serialized_start = 153 - _globals["_SIDEINPUT"]._serialized_end = 306 + DESCRIPTOR._options = None + _globals['_SIDEINPUTRESPONSE']._serialized_start=62 + _globals['_SIDEINPUTRESPONSE']._serialized_end=118 + _globals['_READYRESPONSE']._serialized_start=120 + _globals['_READYRESPONSE']._serialized_end=150 + _globals['_SIDEINPUT']._serialized_start=153 + _globals['_SIDEINPUT']._serialized_end=306 # @@protoc_insertion_point(module_scope) diff --git a/pynumaflow/proto/sideinput/sideinput_pb2_grpc.py b/pynumaflow/proto/sideinput/sideinput_pb2_grpc.py index 72ea87ed..8abe64d2 100644 --- a/pynumaflow/proto/sideinput/sideinput_pb2_grpc.py +++ b/pynumaflow/proto/sideinput/sideinput_pb2_grpc.py @@ -24,15 +24,15 @@ def __init__(self, channel): channel: A grpc.Channel. """ self.RetrieveSideInput = channel.unary_unary( - "/sideinput.v1.SideInput/RetrieveSideInput", - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=sideinput__pb2.SideInputResponse.FromString, - ) + '/sideinput.v1.SideInput/RetrieveSideInput', + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=sideinput__pb2.SideInputResponse.FromString, + ) self.IsReady = channel.unary_unary( - "/sideinput.v1.SideInput/IsReady", - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=sideinput__pb2.ReadyResponse.FromString, - ) + '/sideinput.v1.SideInput/IsReady', + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=sideinput__pb2.ReadyResponse.FromString, + ) class SideInputServicer(object): @@ -47,38 +47,39 @@ class SideInputServicer(object): """ def RetrieveSideInput(self, request, context): - """RetrieveSideInput is the endpoint to retrieve the latest value of a given Side Input.""" + """RetrieveSideInput is the endpoint to retrieve the latest value of a given Side Input. + """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def IsReady(self, request, context): - """IsReady is the health check endpoint to indicate whether the service is ready to be used.""" + """IsReady is the health check endpoint to indicate whether the service is ready to be used. + """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def add_SideInputServicer_to_server(servicer, server): rpc_method_handlers = { - "RetrieveSideInput": grpc.unary_unary_rpc_method_handler( - servicer.RetrieveSideInput, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=sideinput__pb2.SideInputResponse.SerializeToString, - ), - "IsReady": grpc.unary_unary_rpc_method_handler( - servicer.IsReady, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=sideinput__pb2.ReadyResponse.SerializeToString, - ), + 'RetrieveSideInput': grpc.unary_unary_rpc_method_handler( + servicer.RetrieveSideInput, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=sideinput__pb2.SideInputResponse.SerializeToString, + ), + 'IsReady': grpc.unary_unary_rpc_method_handler( + servicer.IsReady, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=sideinput__pb2.ReadyResponse.SerializeToString, + ), } generic_handler = grpc.method_handlers_generic_handler( - "sideinput.v1.SideInput", rpc_method_handlers - ) + 'sideinput.v1.SideInput', rpc_method_handlers) server.add_generic_rpc_handlers((generic_handler,)) -# This class is part of an EXPERIMENTAL API. + # This class is part of an EXPERIMENTAL API. class SideInput(object): """SideInput is the gRPC service for user-defined Side Inputs. It is used to propagate changes in the values of the provided Side Inputs @@ -91,59 +92,35 @@ class SideInput(object): """ @staticmethod - def RetrieveSideInput( - request, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.unary_unary( - request, + def RetrieveSideInput(request, target, - "/sideinput.v1.SideInput/RetrieveSideInput", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.unary_unary(request, target, '/sideinput.v1.SideInput/RetrieveSideInput', google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, sideinput__pb2.SideInputResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) @staticmethod - def IsReady( - request, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.unary_unary( - request, + def IsReady(request, target, - "/sideinput.v1.SideInput/IsReady", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.unary_unary(request, target, '/sideinput.v1.SideInput/IsReady', google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, sideinput__pb2.ReadyResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) diff --git a/pynumaflow/proto/sinker/sink_pb2.py b/pynumaflow/proto/sinker/sink_pb2.py index 27082a0e..67d905c5 100644 --- a/pynumaflow/proto/sinker/sink_pb2.py +++ b/pynumaflow/proto/sinker/sink_pb2.py @@ -7,7 +7,6 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import symbol_database as _symbol_database from google.protobuf.internal import builder as _builder - # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() @@ -17,35 +16,33 @@ from google.protobuf import timestamp_pb2 as google_dot_protobuf_dot_timestamp__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\nsink.proto\x12\x07sink.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto"\xa3\x03\n\x0bSinkRequest\x12-\n\x07request\x18\x01 \x01(\x0b\x32\x1c.sink.v1.SinkRequest.Request\x12+\n\x06status\x18\x02 \x01(\x0b\x32\x1b.sink.v1.TransmissionStatus\x12*\n\thandshake\x18\x03 \x01(\x0b\x32\x12.sink.v1.HandshakeH\x00\x88\x01\x01\x1a\xfd\x01\n\x07Request\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\n\n\x02id\x18\x05 \x01(\t\x12:\n\x07headers\x18\x06 \x03(\x0b\x32).sink.v1.SinkRequest.Request.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x42\x0c\n\n_handshake"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08"!\n\x12TransmissionStatus\x12\x0b\n\x03\x65ot\x18\x01 \x01(\x08"\xfc\x01\n\x0cSinkResponse\x12-\n\x07results\x18\x01 \x03(\x0b\x32\x1c.sink.v1.SinkResponse.Result\x12*\n\thandshake\x18\x02 \x01(\x0b\x32\x12.sink.v1.HandshakeH\x00\x88\x01\x01\x12\x30\n\x06status\x18\x03 \x01(\x0b\x32\x1b.sink.v1.TransmissionStatusH\x01\x88\x01\x01\x1a\x46\n\x06Result\x12\n\n\x02id\x18\x01 \x01(\t\x12\x1f\n\x06status\x18\x02 \x01(\x0e\x32\x0f.sink.v1.Status\x12\x0f\n\x07\x65rr_msg\x18\x03 \x01(\tB\x0c\n\n_handshakeB\t\n\x07_status*0\n\x06Status\x12\x0b\n\x07SUCCESS\x10\x00\x12\x0b\n\x07\x46\x41ILURE\x10\x01\x12\x0c\n\x08\x46\x41LLBACK\x10\x02\x32|\n\x04Sink\x12\x39\n\x06SinkFn\x12\x14.sink.v1.SinkRequest\x1a\x15.sink.v1.SinkResponse(\x01\x30\x01\x12\x39\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x16.sink.v1.ReadyResponseb\x06proto3' -) +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\nsink.proto\x12\x07sink.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto\"\xa3\x03\n\x0bSinkRequest\x12-\n\x07request\x18\x01 \x01(\x0b\x32\x1c.sink.v1.SinkRequest.Request\x12+\n\x06status\x18\x02 \x01(\x0b\x32\x1b.sink.v1.TransmissionStatus\x12*\n\thandshake\x18\x03 \x01(\x0b\x32\x12.sink.v1.HandshakeH\x00\x88\x01\x01\x1a\xfd\x01\n\x07Request\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\n\n\x02id\x18\x05 \x01(\t\x12:\n\x07headers\x18\x06 \x03(\x0b\x32).sink.v1.SinkRequest.Request.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x42\x0c\n\n_handshake\"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08\"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\"!\n\x12TransmissionStatus\x12\x0b\n\x03\x65ot\x18\x01 \x01(\x08\"\xfc\x01\n\x0cSinkResponse\x12-\n\x07results\x18\x01 \x03(\x0b\x32\x1c.sink.v1.SinkResponse.Result\x12*\n\thandshake\x18\x02 \x01(\x0b\x32\x12.sink.v1.HandshakeH\x00\x88\x01\x01\x12\x30\n\x06status\x18\x03 \x01(\x0b\x32\x1b.sink.v1.TransmissionStatusH\x01\x88\x01\x01\x1a\x46\n\x06Result\x12\n\n\x02id\x18\x01 \x01(\t\x12\x1f\n\x06status\x18\x02 \x01(\x0e\x32\x0f.sink.v1.Status\x12\x0f\n\x07\x65rr_msg\x18\x03 \x01(\tB\x0c\n\n_handshakeB\t\n\x07_status*0\n\x06Status\x12\x0b\n\x07SUCCESS\x10\x00\x12\x0b\n\x07\x46\x41ILURE\x10\x01\x12\x0c\n\x08\x46\x41LLBACK\x10\x02\x32|\n\x04Sink\x12\x39\n\x06SinkFn\x12\x14.sink.v1.SinkRequest\x1a\x15.sink.v1.SinkResponse(\x01\x30\x01\x12\x39\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x16.sink.v1.ReadyResponseb\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) -_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, "sink_pb2", _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'sink_pb2', _globals) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None - _globals["_SINKREQUEST_REQUEST_HEADERSENTRY"]._options = None - _globals["_SINKREQUEST_REQUEST_HEADERSENTRY"]._serialized_options = b"8\001" - _globals["_STATUS"]._serialized_start = 855 - _globals["_STATUS"]._serialized_end = 903 - _globals["_SINKREQUEST"]._serialized_start = 86 - _globals["_SINKREQUEST"]._serialized_end = 505 - _globals["_SINKREQUEST_REQUEST"]._serialized_start = 238 - _globals["_SINKREQUEST_REQUEST"]._serialized_end = 491 - _globals["_SINKREQUEST_REQUEST_HEADERSENTRY"]._serialized_start = 445 - _globals["_SINKREQUEST_REQUEST_HEADERSENTRY"]._serialized_end = 491 - _globals["_HANDSHAKE"]._serialized_start = 507 - _globals["_HANDSHAKE"]._serialized_end = 531 - _globals["_READYRESPONSE"]._serialized_start = 533 - _globals["_READYRESPONSE"]._serialized_end = 563 - _globals["_TRANSMISSIONSTATUS"]._serialized_start = 565 - _globals["_TRANSMISSIONSTATUS"]._serialized_end = 598 - _globals["_SINKRESPONSE"]._serialized_start = 601 - _globals["_SINKRESPONSE"]._serialized_end = 853 - _globals["_SINKRESPONSE_RESULT"]._serialized_start = 758 - _globals["_SINKRESPONSE_RESULT"]._serialized_end = 828 - _globals["_SINK"]._serialized_start = 905 - _globals["_SINK"]._serialized_end = 1029 + DESCRIPTOR._options = None + _globals['_SINKREQUEST_REQUEST_HEADERSENTRY']._options = None + _globals['_SINKREQUEST_REQUEST_HEADERSENTRY']._serialized_options = b'8\001' + _globals['_STATUS']._serialized_start=855 + _globals['_STATUS']._serialized_end=903 + _globals['_SINKREQUEST']._serialized_start=86 + _globals['_SINKREQUEST']._serialized_end=505 + _globals['_SINKREQUEST_REQUEST']._serialized_start=238 + _globals['_SINKREQUEST_REQUEST']._serialized_end=491 + _globals['_SINKREQUEST_REQUEST_HEADERSENTRY']._serialized_start=445 + _globals['_SINKREQUEST_REQUEST_HEADERSENTRY']._serialized_end=491 + _globals['_HANDSHAKE']._serialized_start=507 + _globals['_HANDSHAKE']._serialized_end=531 + _globals['_READYRESPONSE']._serialized_start=533 + _globals['_READYRESPONSE']._serialized_end=563 + _globals['_TRANSMISSIONSTATUS']._serialized_start=565 + _globals['_TRANSMISSIONSTATUS']._serialized_end=598 + _globals['_SINKRESPONSE']._serialized_start=601 + _globals['_SINKRESPONSE']._serialized_end=853 + _globals['_SINKRESPONSE_RESULT']._serialized_start=758 + _globals['_SINKRESPONSE_RESULT']._serialized_end=828 + _globals['_SINK']._serialized_start=905 + _globals['_SINK']._serialized_end=1029 # @@protoc_insertion_point(module_scope) diff --git a/pynumaflow/proto/sinker/sink_pb2.pyi b/pynumaflow/proto/sinker/sink_pb2.pyi index 78926321..6c0292ca 100644 --- a/pynumaflow/proto/sinker/sink_pb2.pyi +++ b/pynumaflow/proto/sinker/sink_pb2.pyi @@ -4,13 +4,7 @@ from google.protobuf.internal import containers as _containers from google.protobuf.internal import enum_type_wrapper as _enum_type_wrapper from google.protobuf import descriptor as _descriptor from google.protobuf import message as _message -from typing import ( - ClassVar as _ClassVar, - Iterable as _Iterable, - Mapping as _Mapping, - Optional as _Optional, - Union as _Union, -) +from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union DESCRIPTOR: _descriptor.FileDescriptor @@ -19,17 +13,14 @@ class Status(int, metaclass=_enum_type_wrapper.EnumTypeWrapper): SUCCESS: _ClassVar[Status] FAILURE: _ClassVar[Status] FALLBACK: _ClassVar[Status] - SUCCESS: Status FAILURE: Status FALLBACK: Status class SinkRequest(_message.Message): __slots__ = ("request", "status", "handshake") - class Request(_message.Message): __slots__ = ("keys", "value", "event_time", "watermark", "id", "headers") - class HeadersEntry(_message.Message): __slots__ = ("key", "value") KEY_FIELD_NUMBER: _ClassVar[int] @@ -49,27 +40,14 @@ class SinkRequest(_message.Message): watermark: _timestamp_pb2.Timestamp id: str headers: _containers.ScalarMap[str, str] - def __init__( - self, - keys: _Optional[_Iterable[str]] = ..., - value: _Optional[bytes] = ..., - event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - id: _Optional[str] = ..., - headers: _Optional[_Mapping[str, str]] = ..., - ) -> None: ... + def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., id: _Optional[str] = ..., headers: _Optional[_Mapping[str, str]] = ...) -> None: ... REQUEST_FIELD_NUMBER: _ClassVar[int] STATUS_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] request: SinkRequest.Request status: TransmissionStatus handshake: Handshake - def __init__( - self, - request: _Optional[_Union[SinkRequest.Request, _Mapping]] = ..., - status: _Optional[_Union[TransmissionStatus, _Mapping]] = ..., - handshake: _Optional[_Union[Handshake, _Mapping]] = ..., - ) -> None: ... + def __init__(self, request: _Optional[_Union[SinkRequest.Request, _Mapping]] = ..., status: _Optional[_Union[TransmissionStatus, _Mapping]] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ...) -> None: ... class Handshake(_message.Message): __slots__ = ("sot",) @@ -91,7 +69,6 @@ class TransmissionStatus(_message.Message): class SinkResponse(_message.Message): __slots__ = ("results", "handshake", "status") - class Result(_message.Message): __slots__ = ("id", "status", "err_msg") ID_FIELD_NUMBER: _ClassVar[int] @@ -100,21 +77,11 @@ class SinkResponse(_message.Message): id: str status: Status err_msg: str - def __init__( - self, - id: _Optional[str] = ..., - status: _Optional[_Union[Status, str]] = ..., - err_msg: _Optional[str] = ..., - ) -> None: ... + def __init__(self, id: _Optional[str] = ..., status: _Optional[_Union[Status, str]] = ..., err_msg: _Optional[str] = ...) -> None: ... RESULTS_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] STATUS_FIELD_NUMBER: _ClassVar[int] results: _containers.RepeatedCompositeFieldContainer[SinkResponse.Result] handshake: Handshake status: TransmissionStatus - def __init__( - self, - results: _Optional[_Iterable[_Union[SinkResponse.Result, _Mapping]]] = ..., - handshake: _Optional[_Union[Handshake, _Mapping]] = ..., - status: _Optional[_Union[TransmissionStatus, _Mapping]] = ..., - ) -> None: ... + def __init__(self, results: _Optional[_Iterable[_Union[SinkResponse.Result, _Mapping]]] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ..., status: _Optional[_Union[TransmissionStatus, _Mapping]] = ...) -> None: ... diff --git a/pynumaflow/proto/sinker/sink_pb2_grpc.py b/pynumaflow/proto/sinker/sink_pb2_grpc.py index 9609c76e..4f53a34e 100644 --- a/pynumaflow/proto/sinker/sink_pb2_grpc.py +++ b/pynumaflow/proto/sinker/sink_pb2_grpc.py @@ -16,108 +16,87 @@ def __init__(self, channel): channel: A grpc.Channel. """ self.SinkFn = channel.stream_stream( - "/sink.v1.Sink/SinkFn", - request_serializer=sink__pb2.SinkRequest.SerializeToString, - response_deserializer=sink__pb2.SinkResponse.FromString, - ) + '/sink.v1.Sink/SinkFn', + request_serializer=sink__pb2.SinkRequest.SerializeToString, + response_deserializer=sink__pb2.SinkResponse.FromString, + ) self.IsReady = channel.unary_unary( - "/sink.v1.Sink/IsReady", - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=sink__pb2.ReadyResponse.FromString, - ) + '/sink.v1.Sink/IsReady', + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=sink__pb2.ReadyResponse.FromString, + ) class SinkServicer(object): """Missing associated documentation comment in .proto file.""" def SinkFn(self, request_iterator, context): - """SinkFn writes the request to a user defined sink.""" + """SinkFn writes the request to a user defined sink. + """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def IsReady(self, request, context): - """IsReady is the heartbeat endpoint for gRPC.""" + """IsReady is the heartbeat endpoint for gRPC. + """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def add_SinkServicer_to_server(servicer, server): rpc_method_handlers = { - "SinkFn": grpc.stream_stream_rpc_method_handler( - servicer.SinkFn, - request_deserializer=sink__pb2.SinkRequest.FromString, - response_serializer=sink__pb2.SinkResponse.SerializeToString, - ), - "IsReady": grpc.unary_unary_rpc_method_handler( - servicer.IsReady, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=sink__pb2.ReadyResponse.SerializeToString, - ), + 'SinkFn': grpc.stream_stream_rpc_method_handler( + servicer.SinkFn, + request_deserializer=sink__pb2.SinkRequest.FromString, + response_serializer=sink__pb2.SinkResponse.SerializeToString, + ), + 'IsReady': grpc.unary_unary_rpc_method_handler( + servicer.IsReady, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=sink__pb2.ReadyResponse.SerializeToString, + ), } - generic_handler = grpc.method_handlers_generic_handler("sink.v1.Sink", rpc_method_handlers) + generic_handler = grpc.method_handlers_generic_handler( + 'sink.v1.Sink', rpc_method_handlers) server.add_generic_rpc_handlers((generic_handler,)) -# This class is part of an EXPERIMENTAL API. + # This class is part of an EXPERIMENTAL API. class Sink(object): """Missing associated documentation comment in .proto file.""" @staticmethod - def SinkFn( - request_iterator, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.stream_stream( - request_iterator, + def SinkFn(request_iterator, target, - "/sink.v1.Sink/SinkFn", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.stream_stream(request_iterator, target, '/sink.v1.Sink/SinkFn', sink__pb2.SinkRequest.SerializeToString, sink__pb2.SinkResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) @staticmethod - def IsReady( - request, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.unary_unary( - request, + def IsReady(request, target, - "/sink.v1.Sink/IsReady", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.unary_unary(request, target, '/sink.v1.Sink/IsReady', google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, sink__pb2.ReadyResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) diff --git a/pynumaflow/proto/sourcer/source_pb2.py b/pynumaflow/proto/sourcer/source_pb2.py index 6cd6f0b2..e8d62515 100644 --- a/pynumaflow/proto/sourcer/source_pb2.py +++ b/pynumaflow/proto/sourcer/source_pb2.py @@ -7,7 +7,6 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import symbol_database as _symbol_database from google.protobuf.internal import builder as _builder - # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() @@ -17,55 +16,53 @@ from google.protobuf import empty_pb2 as google_dot_protobuf_dot_empty__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x0csource.proto\x12\tsource.v1\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1bgoogle/protobuf/empty.proto"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08"\xb1\x01\n\x0bReadRequest\x12/\n\x07request\x18\x01 \x01(\x0b\x32\x1e.source.v1.ReadRequest.Request\x12,\n\thandshake\x18\x02 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a\x35\n\x07Request\x12\x13\n\x0bnum_records\x18\x01 \x01(\x04\x12\x15\n\rtimeout_in_ms\x18\x02 \x01(\rB\x0c\n\n_handshake"\x81\x05\n\x0cReadResponse\x12.\n\x06result\x18\x01 \x01(\x0b\x32\x1e.source.v1.ReadResponse.Result\x12.\n\x06status\x18\x02 \x01(\x0b\x32\x1e.source.v1.ReadResponse.Status\x12,\n\thandshake\x18\x03 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a\xe8\x01\n\x06Result\x12\x0f\n\x07payload\x18\x01 \x01(\x0c\x12!\n\x06offset\x18\x02 \x01(\x0b\x32\x11.source.v1.Offset\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04keys\x18\x04 \x03(\t\x12<\n\x07headers\x18\x05 \x03(\x0b\x32+.source.v1.ReadResponse.Result.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x1a\xe9\x01\n\x06Status\x12\x0b\n\x03\x65ot\x18\x01 \x01(\x08\x12\x31\n\x04\x63ode\x18\x02 \x01(\x0e\x32#.source.v1.ReadResponse.Status.Code\x12\x38\n\x05\x65rror\x18\x03 \x01(\x0e\x32$.source.v1.ReadResponse.Status.ErrorH\x00\x88\x01\x01\x12\x10\n\x03msg\x18\x04 \x01(\tH\x01\x88\x01\x01" \n\x04\x43ode\x12\x0b\n\x07SUCCESS\x10\x00\x12\x0b\n\x07\x46\x41ILURE\x10\x01"\x1f\n\x05\x45rror\x12\x0b\n\x07UNACKED\x10\x00\x12\t\n\x05OTHER\x10\x01\x42\x08\n\x06_errorB\x06\n\x04_msgB\x0c\n\n_handshake"\xa7\x01\n\nAckRequest\x12.\n\x07request\x18\x01 \x01(\x0b\x32\x1d.source.v1.AckRequest.Request\x12,\n\thandshake\x18\x02 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a-\n\x07Request\x12"\n\x07offsets\x18\x01 \x03(\x0b\x32\x11.source.v1.OffsetB\x0c\n\n_handshake"\xab\x01\n\x0b\x41\x63kResponse\x12-\n\x06result\x18\x01 \x01(\x0b\x32\x1d.source.v1.AckResponse.Result\x12,\n\thandshake\x18\x02 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a\x31\n\x06Result\x12\'\n\x07success\x18\x01 \x01(\x0b\x32\x16.google.protobuf.EmptyB\x0c\n\n_handshake"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08"]\n\x0fPendingResponse\x12\x31\n\x06result\x18\x01 \x01(\x0b\x32!.source.v1.PendingResponse.Result\x1a\x17\n\x06Result\x12\r\n\x05\x63ount\x18\x01 \x01(\x03"h\n\x12PartitionsResponse\x12\x34\n\x06result\x18\x01 \x01(\x0b\x32$.source.v1.PartitionsResponse.Result\x1a\x1c\n\x06Result\x12\x12\n\npartitions\x18\x01 \x03(\x05".\n\x06Offset\x12\x0e\n\x06offset\x18\x01 \x01(\x0c\x12\x14\n\x0cpartition_id\x18\x02 \x01(\x05\x32\xc8\x02\n\x06Source\x12=\n\x06ReadFn\x12\x16.source.v1.ReadRequest\x1a\x17.source.v1.ReadResponse(\x01\x30\x01\x12:\n\x05\x41\x63kFn\x12\x15.source.v1.AckRequest\x1a\x16.source.v1.AckResponse(\x01\x30\x01\x12?\n\tPendingFn\x12\x16.google.protobuf.Empty\x1a\x1a.source.v1.PendingResponse\x12\x45\n\x0cPartitionsFn\x12\x16.google.protobuf.Empty\x1a\x1d.source.v1.PartitionsResponse\x12;\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x18.source.v1.ReadyResponseb\x06proto3' -) +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x0csource.proto\x12\tsource.v1\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1bgoogle/protobuf/empty.proto\"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08\"\xb1\x01\n\x0bReadRequest\x12/\n\x07request\x18\x01 \x01(\x0b\x32\x1e.source.v1.ReadRequest.Request\x12,\n\thandshake\x18\x02 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a\x35\n\x07Request\x12\x13\n\x0bnum_records\x18\x01 \x01(\x04\x12\x15\n\rtimeout_in_ms\x18\x02 \x01(\rB\x0c\n\n_handshake\"\x81\x05\n\x0cReadResponse\x12.\n\x06result\x18\x01 \x01(\x0b\x32\x1e.source.v1.ReadResponse.Result\x12.\n\x06status\x18\x02 \x01(\x0b\x32\x1e.source.v1.ReadResponse.Status\x12,\n\thandshake\x18\x03 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a\xe8\x01\n\x06Result\x12\x0f\n\x07payload\x18\x01 \x01(\x0c\x12!\n\x06offset\x18\x02 \x01(\x0b\x32\x11.source.v1.Offset\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04keys\x18\x04 \x03(\t\x12<\n\x07headers\x18\x05 \x03(\x0b\x32+.source.v1.ReadResponse.Result.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x1a\xe9\x01\n\x06Status\x12\x0b\n\x03\x65ot\x18\x01 \x01(\x08\x12\x31\n\x04\x63ode\x18\x02 \x01(\x0e\x32#.source.v1.ReadResponse.Status.Code\x12\x38\n\x05\x65rror\x18\x03 \x01(\x0e\x32$.source.v1.ReadResponse.Status.ErrorH\x00\x88\x01\x01\x12\x10\n\x03msg\x18\x04 \x01(\tH\x01\x88\x01\x01\" \n\x04\x43ode\x12\x0b\n\x07SUCCESS\x10\x00\x12\x0b\n\x07\x46\x41ILURE\x10\x01\"\x1f\n\x05\x45rror\x12\x0b\n\x07UNACKED\x10\x00\x12\t\n\x05OTHER\x10\x01\x42\x08\n\x06_errorB\x06\n\x04_msgB\x0c\n\n_handshake\"\xa7\x01\n\nAckRequest\x12.\n\x07request\x18\x01 \x01(\x0b\x32\x1d.source.v1.AckRequest.Request\x12,\n\thandshake\x18\x02 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a-\n\x07Request\x12\"\n\x07offsets\x18\x01 \x03(\x0b\x32\x11.source.v1.OffsetB\x0c\n\n_handshake\"\xab\x01\n\x0b\x41\x63kResponse\x12-\n\x06result\x18\x01 \x01(\x0b\x32\x1d.source.v1.AckResponse.Result\x12,\n\thandshake\x18\x02 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a\x31\n\x06Result\x12\'\n\x07success\x18\x01 \x01(\x0b\x32\x16.google.protobuf.EmptyB\x0c\n\n_handshake\"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\"]\n\x0fPendingResponse\x12\x31\n\x06result\x18\x01 \x01(\x0b\x32!.source.v1.PendingResponse.Result\x1a\x17\n\x06Result\x12\r\n\x05\x63ount\x18\x01 \x01(\x03\"h\n\x12PartitionsResponse\x12\x34\n\x06result\x18\x01 \x01(\x0b\x32$.source.v1.PartitionsResponse.Result\x1a\x1c\n\x06Result\x12\x12\n\npartitions\x18\x01 \x03(\x05\".\n\x06Offset\x12\x0e\n\x06offset\x18\x01 \x01(\x0c\x12\x14\n\x0cpartition_id\x18\x02 \x01(\x05\x32\xc8\x02\n\x06Source\x12=\n\x06ReadFn\x12\x16.source.v1.ReadRequest\x1a\x17.source.v1.ReadResponse(\x01\x30\x01\x12:\n\x05\x41\x63kFn\x12\x15.source.v1.AckRequest\x1a\x16.source.v1.AckResponse(\x01\x30\x01\x12?\n\tPendingFn\x12\x16.google.protobuf.Empty\x1a\x1a.source.v1.PendingResponse\x12\x45\n\x0cPartitionsFn\x12\x16.google.protobuf.Empty\x1a\x1d.source.v1.PartitionsResponse\x12;\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x18.source.v1.ReadyResponseb\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) -_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, "source_pb2", _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'source_pb2', _globals) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None - _globals["_READRESPONSE_RESULT_HEADERSENTRY"]._options = None - _globals["_READRESPONSE_RESULT_HEADERSENTRY"]._serialized_options = b"8\001" - _globals["_HANDSHAKE"]._serialized_start = 89 - _globals["_HANDSHAKE"]._serialized_end = 113 - _globals["_READREQUEST"]._serialized_start = 116 - _globals["_READREQUEST"]._serialized_end = 293 - _globals["_READREQUEST_REQUEST"]._serialized_start = 226 - _globals["_READREQUEST_REQUEST"]._serialized_end = 279 - _globals["_READRESPONSE"]._serialized_start = 296 - _globals["_READRESPONSE"]._serialized_end = 937 - _globals["_READRESPONSE_RESULT"]._serialized_start = 455 - _globals["_READRESPONSE_RESULT"]._serialized_end = 687 - _globals["_READRESPONSE_RESULT_HEADERSENTRY"]._serialized_start = 641 - _globals["_READRESPONSE_RESULT_HEADERSENTRY"]._serialized_end = 687 - _globals["_READRESPONSE_STATUS"]._serialized_start = 690 - _globals["_READRESPONSE_STATUS"]._serialized_end = 923 - _globals["_READRESPONSE_STATUS_CODE"]._serialized_start = 840 - _globals["_READRESPONSE_STATUS_CODE"]._serialized_end = 872 - _globals["_READRESPONSE_STATUS_ERROR"]._serialized_start = 874 - _globals["_READRESPONSE_STATUS_ERROR"]._serialized_end = 905 - _globals["_ACKREQUEST"]._serialized_start = 940 - _globals["_ACKREQUEST"]._serialized_end = 1107 - _globals["_ACKREQUEST_REQUEST"]._serialized_start = 1048 - _globals["_ACKREQUEST_REQUEST"]._serialized_end = 1093 - _globals["_ACKRESPONSE"]._serialized_start = 1110 - _globals["_ACKRESPONSE"]._serialized_end = 1281 - _globals["_ACKRESPONSE_RESULT"]._serialized_start = 1218 - _globals["_ACKRESPONSE_RESULT"]._serialized_end = 1267 - _globals["_READYRESPONSE"]._serialized_start = 1283 - _globals["_READYRESPONSE"]._serialized_end = 1313 - _globals["_PENDINGRESPONSE"]._serialized_start = 1315 - _globals["_PENDINGRESPONSE"]._serialized_end = 1408 - _globals["_PENDINGRESPONSE_RESULT"]._serialized_start = 1385 - _globals["_PENDINGRESPONSE_RESULT"]._serialized_end = 1408 - _globals["_PARTITIONSRESPONSE"]._serialized_start = 1410 - _globals["_PARTITIONSRESPONSE"]._serialized_end = 1514 - _globals["_PARTITIONSRESPONSE_RESULT"]._serialized_start = 1486 - _globals["_PARTITIONSRESPONSE_RESULT"]._serialized_end = 1514 - _globals["_OFFSET"]._serialized_start = 1516 - _globals["_OFFSET"]._serialized_end = 1562 - _globals["_SOURCE"]._serialized_start = 1565 - _globals["_SOURCE"]._serialized_end = 1893 + DESCRIPTOR._options = None + _globals['_READRESPONSE_RESULT_HEADERSENTRY']._options = None + _globals['_READRESPONSE_RESULT_HEADERSENTRY']._serialized_options = b'8\001' + _globals['_HANDSHAKE']._serialized_start=89 + _globals['_HANDSHAKE']._serialized_end=113 + _globals['_READREQUEST']._serialized_start=116 + _globals['_READREQUEST']._serialized_end=293 + _globals['_READREQUEST_REQUEST']._serialized_start=226 + _globals['_READREQUEST_REQUEST']._serialized_end=279 + _globals['_READRESPONSE']._serialized_start=296 + _globals['_READRESPONSE']._serialized_end=937 + _globals['_READRESPONSE_RESULT']._serialized_start=455 + _globals['_READRESPONSE_RESULT']._serialized_end=687 + _globals['_READRESPONSE_RESULT_HEADERSENTRY']._serialized_start=641 + _globals['_READRESPONSE_RESULT_HEADERSENTRY']._serialized_end=687 + _globals['_READRESPONSE_STATUS']._serialized_start=690 + _globals['_READRESPONSE_STATUS']._serialized_end=923 + _globals['_READRESPONSE_STATUS_CODE']._serialized_start=840 + _globals['_READRESPONSE_STATUS_CODE']._serialized_end=872 + _globals['_READRESPONSE_STATUS_ERROR']._serialized_start=874 + _globals['_READRESPONSE_STATUS_ERROR']._serialized_end=905 + _globals['_ACKREQUEST']._serialized_start=940 + _globals['_ACKREQUEST']._serialized_end=1107 + _globals['_ACKREQUEST_REQUEST']._serialized_start=1048 + _globals['_ACKREQUEST_REQUEST']._serialized_end=1093 + _globals['_ACKRESPONSE']._serialized_start=1110 + _globals['_ACKRESPONSE']._serialized_end=1281 + _globals['_ACKRESPONSE_RESULT']._serialized_start=1218 + _globals['_ACKRESPONSE_RESULT']._serialized_end=1267 + _globals['_READYRESPONSE']._serialized_start=1283 + _globals['_READYRESPONSE']._serialized_end=1313 + _globals['_PENDINGRESPONSE']._serialized_start=1315 + _globals['_PENDINGRESPONSE']._serialized_end=1408 + _globals['_PENDINGRESPONSE_RESULT']._serialized_start=1385 + _globals['_PENDINGRESPONSE_RESULT']._serialized_end=1408 + _globals['_PARTITIONSRESPONSE']._serialized_start=1410 + _globals['_PARTITIONSRESPONSE']._serialized_end=1514 + _globals['_PARTITIONSRESPONSE_RESULT']._serialized_start=1486 + _globals['_PARTITIONSRESPONSE_RESULT']._serialized_end=1514 + _globals['_OFFSET']._serialized_start=1516 + _globals['_OFFSET']._serialized_end=1562 + _globals['_SOURCE']._serialized_start=1565 + _globals['_SOURCE']._serialized_end=1893 # @@protoc_insertion_point(module_scope) diff --git a/pynumaflow/proto/sourcer/source_pb2.pyi b/pynumaflow/proto/sourcer/source_pb2.pyi index f2cdc70e..52c20928 100644 --- a/pynumaflow/proto/sourcer/source_pb2.pyi +++ b/pynumaflow/proto/sourcer/source_pb2.pyi @@ -4,13 +4,7 @@ from google.protobuf.internal import containers as _containers from google.protobuf.internal import enum_type_wrapper as _enum_type_wrapper from google.protobuf import descriptor as _descriptor from google.protobuf import message as _message -from typing import ( - ClassVar as _ClassVar, - Iterable as _Iterable, - Mapping as _Mapping, - Optional as _Optional, - Union as _Union, -) +from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union DESCRIPTOR: _descriptor.FileDescriptor @@ -22,32 +16,23 @@ class Handshake(_message.Message): class ReadRequest(_message.Message): __slots__ = ("request", "handshake") - class Request(_message.Message): __slots__ = ("num_records", "timeout_in_ms") NUM_RECORDS_FIELD_NUMBER: _ClassVar[int] TIMEOUT_IN_MS_FIELD_NUMBER: _ClassVar[int] num_records: int timeout_in_ms: int - def __init__( - self, num_records: _Optional[int] = ..., timeout_in_ms: _Optional[int] = ... - ) -> None: ... + def __init__(self, num_records: _Optional[int] = ..., timeout_in_ms: _Optional[int] = ...) -> None: ... REQUEST_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] request: ReadRequest.Request handshake: Handshake - def __init__( - self, - request: _Optional[_Union[ReadRequest.Request, _Mapping]] = ..., - handshake: _Optional[_Union[Handshake, _Mapping]] = ..., - ) -> None: ... + def __init__(self, request: _Optional[_Union[ReadRequest.Request, _Mapping]] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ...) -> None: ... class ReadResponse(_message.Message): __slots__ = ("result", "status", "handshake") - class Result(_message.Message): __slots__ = ("payload", "offset", "event_time", "keys", "headers") - class HeadersEntry(_message.Message): __slots__ = ("key", "value") KEY_FIELD_NUMBER: _ClassVar[int] @@ -65,25 +50,15 @@ class ReadResponse(_message.Message): event_time: _timestamp_pb2.Timestamp keys: _containers.RepeatedScalarFieldContainer[str] headers: _containers.ScalarMap[str, str] - def __init__( - self, - payload: _Optional[bytes] = ..., - offset: _Optional[_Union[Offset, _Mapping]] = ..., - event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - keys: _Optional[_Iterable[str]] = ..., - headers: _Optional[_Mapping[str, str]] = ..., - ) -> None: ... - + def __init__(self, payload: _Optional[bytes] = ..., offset: _Optional[_Union[Offset, _Mapping]] = ..., event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., keys: _Optional[_Iterable[str]] = ..., headers: _Optional[_Mapping[str, str]] = ...) -> None: ... class Status(_message.Message): __slots__ = ("eot", "code", "error", "msg") - class Code(int, metaclass=_enum_type_wrapper.EnumTypeWrapper): __slots__ = () SUCCESS: _ClassVar[ReadResponse.Status.Code] FAILURE: _ClassVar[ReadResponse.Status.Code] SUCCESS: ReadResponse.Status.Code FAILURE: ReadResponse.Status.Code - class Error(int, metaclass=_enum_type_wrapper.EnumTypeWrapper): __slots__ = () UNACKED: _ClassVar[ReadResponse.Status.Error] @@ -98,65 +73,40 @@ class ReadResponse(_message.Message): code: ReadResponse.Status.Code error: ReadResponse.Status.Error msg: str - def __init__( - self, - eot: bool = ..., - code: _Optional[_Union[ReadResponse.Status.Code, str]] = ..., - error: _Optional[_Union[ReadResponse.Status.Error, str]] = ..., - msg: _Optional[str] = ..., - ) -> None: ... + def __init__(self, eot: bool = ..., code: _Optional[_Union[ReadResponse.Status.Code, str]] = ..., error: _Optional[_Union[ReadResponse.Status.Error, str]] = ..., msg: _Optional[str] = ...) -> None: ... RESULT_FIELD_NUMBER: _ClassVar[int] STATUS_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] result: ReadResponse.Result status: ReadResponse.Status handshake: Handshake - def __init__( - self, - result: _Optional[_Union[ReadResponse.Result, _Mapping]] = ..., - status: _Optional[_Union[ReadResponse.Status, _Mapping]] = ..., - handshake: _Optional[_Union[Handshake, _Mapping]] = ..., - ) -> None: ... + def __init__(self, result: _Optional[_Union[ReadResponse.Result, _Mapping]] = ..., status: _Optional[_Union[ReadResponse.Status, _Mapping]] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ...) -> None: ... class AckRequest(_message.Message): __slots__ = ("request", "handshake") - class Request(_message.Message): __slots__ = ("offsets",) OFFSETS_FIELD_NUMBER: _ClassVar[int] offsets: _containers.RepeatedCompositeFieldContainer[Offset] - def __init__( - self, offsets: _Optional[_Iterable[_Union[Offset, _Mapping]]] = ... - ) -> None: ... + def __init__(self, offsets: _Optional[_Iterable[_Union[Offset, _Mapping]]] = ...) -> None: ... REQUEST_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] request: AckRequest.Request handshake: Handshake - def __init__( - self, - request: _Optional[_Union[AckRequest.Request, _Mapping]] = ..., - handshake: _Optional[_Union[Handshake, _Mapping]] = ..., - ) -> None: ... + def __init__(self, request: _Optional[_Union[AckRequest.Request, _Mapping]] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ...) -> None: ... class AckResponse(_message.Message): __slots__ = ("result", "handshake") - class Result(_message.Message): __slots__ = ("success",) SUCCESS_FIELD_NUMBER: _ClassVar[int] success: _empty_pb2.Empty - def __init__( - self, success: _Optional[_Union[_empty_pb2.Empty, _Mapping]] = ... - ) -> None: ... + def __init__(self, success: _Optional[_Union[_empty_pb2.Empty, _Mapping]] = ...) -> None: ... RESULT_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] result: AckResponse.Result handshake: Handshake - def __init__( - self, - result: _Optional[_Union[AckResponse.Result, _Mapping]] = ..., - handshake: _Optional[_Union[Handshake, _Mapping]] = ..., - ) -> None: ... + def __init__(self, result: _Optional[_Union[AckResponse.Result, _Mapping]] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ...) -> None: ... class ReadyResponse(_message.Message): __slots__ = ("ready",) @@ -166,7 +116,6 @@ class ReadyResponse(_message.Message): class PendingResponse(_message.Message): __slots__ = ("result",) - class Result(_message.Message): __slots__ = ("count",) COUNT_FIELD_NUMBER: _ClassVar[int] @@ -174,13 +123,10 @@ class PendingResponse(_message.Message): def __init__(self, count: _Optional[int] = ...) -> None: ... RESULT_FIELD_NUMBER: _ClassVar[int] result: PendingResponse.Result - def __init__( - self, result: _Optional[_Union[PendingResponse.Result, _Mapping]] = ... - ) -> None: ... + def __init__(self, result: _Optional[_Union[PendingResponse.Result, _Mapping]] = ...) -> None: ... class PartitionsResponse(_message.Message): __slots__ = ("result",) - class Result(_message.Message): __slots__ = ("partitions",) PARTITIONS_FIELD_NUMBER: _ClassVar[int] @@ -188,9 +134,7 @@ class PartitionsResponse(_message.Message): def __init__(self, partitions: _Optional[_Iterable[int]] = ...) -> None: ... RESULT_FIELD_NUMBER: _ClassVar[int] result: PartitionsResponse.Result - def __init__( - self, result: _Optional[_Union[PartitionsResponse.Result, _Mapping]] = ... - ) -> None: ... + def __init__(self, result: _Optional[_Union[PartitionsResponse.Result, _Mapping]] = ...) -> None: ... class Offset(_message.Message): __slots__ = ("offset", "partition_id") @@ -198,6 +142,4 @@ class Offset(_message.Message): PARTITION_ID_FIELD_NUMBER: _ClassVar[int] offset: bytes partition_id: int - def __init__( - self, offset: _Optional[bytes] = ..., partition_id: _Optional[int] = ... - ) -> None: ... + def __init__(self, offset: _Optional[bytes] = ..., partition_id: _Optional[int] = ...) -> None: ... diff --git a/pynumaflow/proto/sourcer/source_pb2_grpc.py b/pynumaflow/proto/sourcer/source_pb2_grpc.py index d5e09f1b..14ad1d5a 100644 --- a/pynumaflow/proto/sourcer/source_pb2_grpc.py +++ b/pynumaflow/proto/sourcer/source_pb2_grpc.py @@ -16,30 +16,30 @@ def __init__(self, channel): channel: A grpc.Channel. """ self.ReadFn = channel.stream_stream( - "/source.v1.Source/ReadFn", - request_serializer=source__pb2.ReadRequest.SerializeToString, - response_deserializer=source__pb2.ReadResponse.FromString, - ) + '/source.v1.Source/ReadFn', + request_serializer=source__pb2.ReadRequest.SerializeToString, + response_deserializer=source__pb2.ReadResponse.FromString, + ) self.AckFn = channel.stream_stream( - "/source.v1.Source/AckFn", - request_serializer=source__pb2.AckRequest.SerializeToString, - response_deserializer=source__pb2.AckResponse.FromString, - ) + '/source.v1.Source/AckFn', + request_serializer=source__pb2.AckRequest.SerializeToString, + response_deserializer=source__pb2.AckResponse.FromString, + ) self.PendingFn = channel.unary_unary( - "/source.v1.Source/PendingFn", - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=source__pb2.PendingResponse.FromString, - ) + '/source.v1.Source/PendingFn', + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=source__pb2.PendingResponse.FromString, + ) self.PartitionsFn = channel.unary_unary( - "/source.v1.Source/PartitionsFn", - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=source__pb2.PartitionsResponse.FromString, - ) + '/source.v1.Source/PartitionsFn', + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=source__pb2.PartitionsResponse.FromString, + ) self.IsReady = channel.unary_unary( - "/source.v1.Source/IsReady", - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=source__pb2.ReadyResponse.FromString, - ) + '/source.v1.Source/IsReady', + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=source__pb2.ReadyResponse.FromString, + ) class SourceServicer(object): @@ -53,8 +53,8 @@ def ReadFn(self, request_iterator, context): Once it has sent all the datum, the server will send a ReadResponse with the end of transmission flag set to true. """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def AckFn(self, request_iterator, context): """AckFn acknowledges a stream of datum offsets. @@ -65,205 +65,149 @@ def AckFn(self, request_iterator, context): Clients sends n requests and expects n responses. """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def PendingFn(self, request, context): - """PendingFn returns the number of pending records at the user defined source.""" + """PendingFn returns the number of pending records at the user defined source. + """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def PartitionsFn(self, request, context): - """PartitionsFn returns the list of partitions for the user defined source.""" + """PartitionsFn returns the list of partitions for the user defined source. + """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def IsReady(self, request, context): - """IsReady is the heartbeat endpoint for user defined source gRPC.""" + """IsReady is the heartbeat endpoint for user defined source gRPC. + """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def add_SourceServicer_to_server(servicer, server): rpc_method_handlers = { - "ReadFn": grpc.stream_stream_rpc_method_handler( - servicer.ReadFn, - request_deserializer=source__pb2.ReadRequest.FromString, - response_serializer=source__pb2.ReadResponse.SerializeToString, - ), - "AckFn": grpc.stream_stream_rpc_method_handler( - servicer.AckFn, - request_deserializer=source__pb2.AckRequest.FromString, - response_serializer=source__pb2.AckResponse.SerializeToString, - ), - "PendingFn": grpc.unary_unary_rpc_method_handler( - servicer.PendingFn, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=source__pb2.PendingResponse.SerializeToString, - ), - "PartitionsFn": grpc.unary_unary_rpc_method_handler( - servicer.PartitionsFn, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=source__pb2.PartitionsResponse.SerializeToString, - ), - "IsReady": grpc.unary_unary_rpc_method_handler( - servicer.IsReady, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=source__pb2.ReadyResponse.SerializeToString, - ), + 'ReadFn': grpc.stream_stream_rpc_method_handler( + servicer.ReadFn, + request_deserializer=source__pb2.ReadRequest.FromString, + response_serializer=source__pb2.ReadResponse.SerializeToString, + ), + 'AckFn': grpc.stream_stream_rpc_method_handler( + servicer.AckFn, + request_deserializer=source__pb2.AckRequest.FromString, + response_serializer=source__pb2.AckResponse.SerializeToString, + ), + 'PendingFn': grpc.unary_unary_rpc_method_handler( + servicer.PendingFn, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=source__pb2.PendingResponse.SerializeToString, + ), + 'PartitionsFn': grpc.unary_unary_rpc_method_handler( + servicer.PartitionsFn, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=source__pb2.PartitionsResponse.SerializeToString, + ), + 'IsReady': grpc.unary_unary_rpc_method_handler( + servicer.IsReady, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=source__pb2.ReadyResponse.SerializeToString, + ), } - generic_handler = grpc.method_handlers_generic_handler("source.v1.Source", rpc_method_handlers) + generic_handler = grpc.method_handlers_generic_handler( + 'source.v1.Source', rpc_method_handlers) server.add_generic_rpc_handlers((generic_handler,)) -# This class is part of an EXPERIMENTAL API. + # This class is part of an EXPERIMENTAL API. class Source(object): """Missing associated documentation comment in .proto file.""" @staticmethod - def ReadFn( - request_iterator, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.stream_stream( - request_iterator, + def ReadFn(request_iterator, target, - "/source.v1.Source/ReadFn", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.stream_stream(request_iterator, target, '/source.v1.Source/ReadFn', source__pb2.ReadRequest.SerializeToString, source__pb2.ReadResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) @staticmethod - def AckFn( - request_iterator, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.stream_stream( - request_iterator, + def AckFn(request_iterator, target, - "/source.v1.Source/AckFn", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.stream_stream(request_iterator, target, '/source.v1.Source/AckFn', source__pb2.AckRequest.SerializeToString, source__pb2.AckResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) @staticmethod - def PendingFn( - request, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.unary_unary( - request, + def PendingFn(request, target, - "/source.v1.Source/PendingFn", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.unary_unary(request, target, '/source.v1.Source/PendingFn', google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, source__pb2.PendingResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) @staticmethod - def PartitionsFn( - request, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.unary_unary( - request, + def PartitionsFn(request, target, - "/source.v1.Source/PartitionsFn", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.unary_unary(request, target, '/source.v1.Source/PartitionsFn', google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, source__pb2.PartitionsResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) @staticmethod - def IsReady( - request, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.unary_unary( - request, + def IsReady(request, target, - "/source.v1.Source/IsReady", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.unary_unary(request, target, '/source.v1.Source/IsReady', google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, source__pb2.ReadyResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) diff --git a/pynumaflow/proto/sourcetransformer/transform_pb2.py b/pynumaflow/proto/sourcetransformer/transform_pb2.py index a7b08491..8309b7ca 100644 --- a/pynumaflow/proto/sourcetransformer/transform_pb2.py +++ b/pynumaflow/proto/sourcetransformer/transform_pb2.py @@ -7,7 +7,6 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import symbol_database as _symbol_database from google.protobuf.internal import builder as _builder - # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() @@ -17,31 +16,29 @@ from google.protobuf import empty_pb2 as google_dot_protobuf_dot_empty__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x0ftransform.proto\x12\x14sourcetransformer.v1\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1bgoogle/protobuf/empty.proto"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08"\xbe\x03\n\x16SourceTransformRequest\x12\x45\n\x07request\x18\x01 \x01(\x0b\x32\x34.sourcetransformer.v1.SourceTransformRequest.Request\x12\x37\n\thandshake\x18\x02 \x01(\x0b\x32\x1f.sourcetransformer.v1.HandshakeH\x00\x88\x01\x01\x1a\x95\x02\n\x07Request\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12R\n\x07headers\x18\x05 \x03(\x0b\x32\x41.sourcetransformer.v1.SourceTransformRequest.Request.HeadersEntry\x12\n\n\x02id\x18\x06 \x01(\t\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x42\x0c\n\n_handshake"\x98\x02\n\x17SourceTransformResponse\x12\x45\n\x07results\x18\x01 \x03(\x0b\x32\x34.sourcetransformer.v1.SourceTransformResponse.Result\x12\n\n\x02id\x18\x02 \x01(\t\x12\x37\n\thandshake\x18\x03 \x01(\x0b\x32\x1f.sourcetransformer.v1.HandshakeH\x00\x88\x01\x01\x1a\x63\n\x06Result\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04tags\x18\x04 \x03(\tB\x0c\n\n_handshake"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\xcf\x01\n\x0fSourceTransform\x12t\n\x11SourceTransformFn\x12,.sourcetransformer.v1.SourceTransformRequest\x1a-.sourcetransformer.v1.SourceTransformResponse(\x01\x30\x01\x12\x46\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a#.sourcetransformer.v1.ReadyResponseb\x06proto3' -) +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x0ftransform.proto\x12\x14sourcetransformer.v1\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1bgoogle/protobuf/empty.proto\"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08\"\xbe\x03\n\x16SourceTransformRequest\x12\x45\n\x07request\x18\x01 \x01(\x0b\x32\x34.sourcetransformer.v1.SourceTransformRequest.Request\x12\x37\n\thandshake\x18\x02 \x01(\x0b\x32\x1f.sourcetransformer.v1.HandshakeH\x00\x88\x01\x01\x1a\x95\x02\n\x07Request\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12R\n\x07headers\x18\x05 \x03(\x0b\x32\x41.sourcetransformer.v1.SourceTransformRequest.Request.HeadersEntry\x12\n\n\x02id\x18\x06 \x01(\t\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x42\x0c\n\n_handshake\"\x98\x02\n\x17SourceTransformResponse\x12\x45\n\x07results\x18\x01 \x03(\x0b\x32\x34.sourcetransformer.v1.SourceTransformResponse.Result\x12\n\n\x02id\x18\x02 \x01(\t\x12\x37\n\thandshake\x18\x03 \x01(\x0b\x32\x1f.sourcetransformer.v1.HandshakeH\x00\x88\x01\x01\x1a\x63\n\x06Result\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04tags\x18\x04 \x03(\tB\x0c\n\n_handshake\"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\xcf\x01\n\x0fSourceTransform\x12t\n\x11SourceTransformFn\x12,.sourcetransformer.v1.SourceTransformRequest\x1a-.sourcetransformer.v1.SourceTransformResponse(\x01\x30\x01\x12\x46\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a#.sourcetransformer.v1.ReadyResponseb\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) -_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, "transform_pb2", _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'transform_pb2', _globals) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None - _globals["_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY"]._options = None - _globals["_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY"]._serialized_options = b"8\001" - _globals["_HANDSHAKE"]._serialized_start = 103 - _globals["_HANDSHAKE"]._serialized_end = 127 - _globals["_SOURCETRANSFORMREQUEST"]._serialized_start = 130 - _globals["_SOURCETRANSFORMREQUEST"]._serialized_end = 576 - _globals["_SOURCETRANSFORMREQUEST_REQUEST"]._serialized_start = 285 - _globals["_SOURCETRANSFORMREQUEST_REQUEST"]._serialized_end = 562 - _globals["_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY"]._serialized_start = 516 - _globals["_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY"]._serialized_end = 562 - _globals["_SOURCETRANSFORMRESPONSE"]._serialized_start = 579 - _globals["_SOURCETRANSFORMRESPONSE"]._serialized_end = 859 - _globals["_SOURCETRANSFORMRESPONSE_RESULT"]._serialized_start = 746 - _globals["_SOURCETRANSFORMRESPONSE_RESULT"]._serialized_end = 845 - _globals["_READYRESPONSE"]._serialized_start = 861 - _globals["_READYRESPONSE"]._serialized_end = 891 - _globals["_SOURCETRANSFORM"]._serialized_start = 894 - _globals["_SOURCETRANSFORM"]._serialized_end = 1101 + DESCRIPTOR._options = None + _globals['_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY']._options = None + _globals['_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY']._serialized_options = b'8\001' + _globals['_HANDSHAKE']._serialized_start=103 + _globals['_HANDSHAKE']._serialized_end=127 + _globals['_SOURCETRANSFORMREQUEST']._serialized_start=130 + _globals['_SOURCETRANSFORMREQUEST']._serialized_end=576 + _globals['_SOURCETRANSFORMREQUEST_REQUEST']._serialized_start=285 + _globals['_SOURCETRANSFORMREQUEST_REQUEST']._serialized_end=562 + _globals['_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY']._serialized_start=516 + _globals['_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY']._serialized_end=562 + _globals['_SOURCETRANSFORMRESPONSE']._serialized_start=579 + _globals['_SOURCETRANSFORMRESPONSE']._serialized_end=859 + _globals['_SOURCETRANSFORMRESPONSE_RESULT']._serialized_start=746 + _globals['_SOURCETRANSFORMRESPONSE_RESULT']._serialized_end=845 + _globals['_READYRESPONSE']._serialized_start=861 + _globals['_READYRESPONSE']._serialized_end=891 + _globals['_SOURCETRANSFORM']._serialized_start=894 + _globals['_SOURCETRANSFORM']._serialized_end=1101 # @@protoc_insertion_point(module_scope) diff --git a/pynumaflow/proto/sourcetransformer/transform_pb2.pyi b/pynumaflow/proto/sourcetransformer/transform_pb2.pyi index cc8fe420..c7c15079 100644 --- a/pynumaflow/proto/sourcetransformer/transform_pb2.pyi +++ b/pynumaflow/proto/sourcetransformer/transform_pb2.pyi @@ -3,13 +3,7 @@ from google.protobuf import empty_pb2 as _empty_pb2 from google.protobuf.internal import containers as _containers from google.protobuf import descriptor as _descriptor from google.protobuf import message as _message -from typing import ( - ClassVar as _ClassVar, - Iterable as _Iterable, - Mapping as _Mapping, - Optional as _Optional, - Union as _Union, -) +from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union DESCRIPTOR: _descriptor.FileDescriptor @@ -21,10 +15,8 @@ class Handshake(_message.Message): class SourceTransformRequest(_message.Message): __slots__ = ("request", "handshake") - class Request(_message.Message): __slots__ = ("keys", "value", "event_time", "watermark", "headers", "id") - class HeadersEntry(_message.Message): __slots__ = ("key", "value") KEY_FIELD_NUMBER: _ClassVar[int] @@ -44,28 +36,15 @@ class SourceTransformRequest(_message.Message): watermark: _timestamp_pb2.Timestamp headers: _containers.ScalarMap[str, str] id: str - def __init__( - self, - keys: _Optional[_Iterable[str]] = ..., - value: _Optional[bytes] = ..., - event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - headers: _Optional[_Mapping[str, str]] = ..., - id: _Optional[str] = ..., - ) -> None: ... + def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., headers: _Optional[_Mapping[str, str]] = ..., id: _Optional[str] = ...) -> None: ... REQUEST_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] request: SourceTransformRequest.Request handshake: Handshake - def __init__( - self, - request: _Optional[_Union[SourceTransformRequest.Request, _Mapping]] = ..., - handshake: _Optional[_Union[Handshake, _Mapping]] = ..., - ) -> None: ... + def __init__(self, request: _Optional[_Union[SourceTransformRequest.Request, _Mapping]] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ...) -> None: ... class SourceTransformResponse(_message.Message): __slots__ = ("results", "id", "handshake") - class Result(_message.Message): __slots__ = ("keys", "value", "event_time", "tags") KEYS_FIELD_NUMBER: _ClassVar[int] @@ -76,25 +55,14 @@ class SourceTransformResponse(_message.Message): value: bytes event_time: _timestamp_pb2.Timestamp tags: _containers.RepeatedScalarFieldContainer[str] - def __init__( - self, - keys: _Optional[_Iterable[str]] = ..., - value: _Optional[bytes] = ..., - event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., - tags: _Optional[_Iterable[str]] = ..., - ) -> None: ... + def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., tags: _Optional[_Iterable[str]] = ...) -> None: ... RESULTS_FIELD_NUMBER: _ClassVar[int] ID_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] results: _containers.RepeatedCompositeFieldContainer[SourceTransformResponse.Result] id: str handshake: Handshake - def __init__( - self, - results: _Optional[_Iterable[_Union[SourceTransformResponse.Result, _Mapping]]] = ..., - id: _Optional[str] = ..., - handshake: _Optional[_Union[Handshake, _Mapping]] = ..., - ) -> None: ... + def __init__(self, results: _Optional[_Iterable[_Union[SourceTransformResponse.Result, _Mapping]]] = ..., id: _Optional[str] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ...) -> None: ... class ReadyResponse(_message.Message): __slots__ = ("ready",) diff --git a/pynumaflow/proto/sourcetransformer/transform_pb2_grpc.py b/pynumaflow/proto/sourcetransformer/transform_pb2_grpc.py index c6820d39..1a1ea499 100644 --- a/pynumaflow/proto/sourcetransformer/transform_pb2_grpc.py +++ b/pynumaflow/proto/sourcetransformer/transform_pb2_grpc.py @@ -16,15 +16,15 @@ def __init__(self, channel): channel: A grpc.Channel. """ self.SourceTransformFn = channel.stream_stream( - "/sourcetransformer.v1.SourceTransform/SourceTransformFn", - request_serializer=transform__pb2.SourceTransformRequest.SerializeToString, - response_deserializer=transform__pb2.SourceTransformResponse.FromString, - ) + '/sourcetransformer.v1.SourceTransform/SourceTransformFn', + request_serializer=transform__pb2.SourceTransformRequest.SerializeToString, + response_deserializer=transform__pb2.SourceTransformResponse.FromString, + ) self.IsReady = channel.unary_unary( - "/sourcetransformer.v1.SourceTransform/IsReady", - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=transform__pb2.ReadyResponse.FromString, - ) + '/sourcetransformer.v1.SourceTransform/IsReady', + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=transform__pb2.ReadyResponse.FromString, + ) class SourceTransformServicer(object): @@ -36,93 +36,69 @@ def SourceTransformFn(self, request_iterator, context): SourceTransformFn can be used only at source vertex by source data transformer. """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def IsReady(self, request, context): - """IsReady is the heartbeat endpoint for gRPC.""" + """IsReady is the heartbeat endpoint for gRPC. + """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def add_SourceTransformServicer_to_server(servicer, server): rpc_method_handlers = { - "SourceTransformFn": grpc.stream_stream_rpc_method_handler( - servicer.SourceTransformFn, - request_deserializer=transform__pb2.SourceTransformRequest.FromString, - response_serializer=transform__pb2.SourceTransformResponse.SerializeToString, - ), - "IsReady": grpc.unary_unary_rpc_method_handler( - servicer.IsReady, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=transform__pb2.ReadyResponse.SerializeToString, - ), + 'SourceTransformFn': grpc.stream_stream_rpc_method_handler( + servicer.SourceTransformFn, + request_deserializer=transform__pb2.SourceTransformRequest.FromString, + response_serializer=transform__pb2.SourceTransformResponse.SerializeToString, + ), + 'IsReady': grpc.unary_unary_rpc_method_handler( + servicer.IsReady, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=transform__pb2.ReadyResponse.SerializeToString, + ), } generic_handler = grpc.method_handlers_generic_handler( - "sourcetransformer.v1.SourceTransform", rpc_method_handlers - ) + 'sourcetransformer.v1.SourceTransform', rpc_method_handlers) server.add_generic_rpc_handlers((generic_handler,)) -# This class is part of an EXPERIMENTAL API. + # This class is part of an EXPERIMENTAL API. class SourceTransform(object): """Missing associated documentation comment in .proto file.""" @staticmethod - def SourceTransformFn( - request_iterator, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.stream_stream( - request_iterator, + def SourceTransformFn(request_iterator, target, - "/sourcetransformer.v1.SourceTransform/SourceTransformFn", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.stream_stream(request_iterator, target, '/sourcetransformer.v1.SourceTransform/SourceTransformFn', transform__pb2.SourceTransformRequest.SerializeToString, transform__pb2.SourceTransformResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) @staticmethod - def IsReady( - request, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.unary_unary( - request, + def IsReady(request, target, - "/sourcetransformer.v1.SourceTransform/IsReady", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.unary_unary(request, target, '/sourcetransformer.v1.SourceTransform/IsReady', google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, transform__pb2.ReadyResponse.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) diff --git a/tests/accumulator/test_async_accumulator.py b/tests/accumulator/test_async_accumulator.py index 9d529889..292e3687 100644 --- a/tests/accumulator/test_async_accumulator.py +++ b/tests/accumulator/test_async_accumulator.py @@ -341,49 +341,6 @@ def test_accumulate_append_without_open(self) -> None: self.assertEqual(5, count) self.assertEqual(1, eof_count) - def test_accumulate_with_multiple_keys_append_only(self) -> None: - stub = self.__stub() - request = start_request_without_open() - generator_response = None - try: - generator_response = stub.AccumulateFn( - request_iterator=request_generator_append_only( - count=10, request=request, resetkey=True - ) - ) - except grpc.RpcError as e: - LOGGER.error(e) - - count = 0 - eof_count = 0 - key_counts = {} - - # capture the output from the AccumulateFn generator and assert. - for r in generator_response: - # Check for responses with values - if r.payload.value: - count += 1 - # Track count per key - key = r.payload.keys[0] if r.payload.keys else "no_key" - key_counts[key] = key_counts.get(key, 0) + 1 - - # Each key should have its own counter starting from 1 - expected_msg = f"counter:{key_counts[key]}" - self.assertEqual( - bytes(expected_msg, encoding="utf-8"), - r.payload.value, - ) - self.assertEqual(r.EOF, False) - else: - eof_count += 1 - self.assertEqual(r.EOF, True) - - # We should have 10 messages (one for each key) - self.assertEqual(10, count) - self.assertEqual(10, eof_count) - # Each key should appear once - self.assertEqual(len(key_counts), 10) - def test_accumulate_append_mixed(self) -> None: stub = self.__stub() request = start_request() @@ -467,6 +424,14 @@ def test_max_threads(self): server = AccumulatorAsyncServer(accumulator_instance=ExampleClass) self.assertEqual(server.max_threads, 4) + # zero threads + server = AccumulatorAsyncServer(ExampleClass, max_threads=0) + self.assertEqual(server.max_threads, 0) + + # negative threads + server = AccumulatorAsyncServer(ExampleClass, max_threads=-5) + self.assertEqual(server.max_threads, -5) + def test_server_info_file_path_handling(self): """Test AccumulatorAsyncServer with custom server info file path.""" @@ -486,22 +451,6 @@ def test_init_kwargs_none_handling(self): # Should not raise any errors and should work correctly self.assertIsNotNone(server.accumulator_handler) - def test_server_with_zero_max_threads(self): - """Test server creation with max_threads set to 0.""" - - server = AccumulatorAsyncServer(ExampleClass, max_threads=0) - - # Should be set to 0 (minimum) - self.assertEqual(server.max_threads, 0) - - def test_server_with_negative_max_threads(self): - """Test server creation with negative max_threads.""" - - server = AccumulatorAsyncServer(ExampleClass, max_threads=-5) - - # Should be set to -5 (the minimum function will handle this) - self.assertEqual(server.max_threads, -5) - def test_server_start_method_logging(self): """Test server start method includes proper logging.""" from unittest.mock import patch diff --git a/tests/accumulator/test_async_accumulator_err.py b/tests/accumulator/test_async_accumulator_err.py index 6ac6d712..5b39174c 100644 --- a/tests/accumulator/test_async_accumulator_err.py +++ b/tests/accumulator/test_async_accumulator_err.py @@ -143,10 +143,10 @@ def tearDownClass(cls) -> None: LOGGER.error(e) @patch("psutil.Process.kill", mock_terminate_on_stop) - def test_accumulate_error(self) -> None: + def test_accumulate_partial_success(self) -> None: + """Test that the first datum is processed before error occurs""" stub = self.__stub() request = start_request() - generator_response = None try: generator_response = stub.AccumulateFn( @@ -155,32 +155,12 @@ def test_accumulate_error(self) -> None: # Try to consume the generator counter = 0 - logging.info("[TEST_DEBUG] About to iterate through generator_response") for response in generator_response: + self.assertIsInstance(response, accumulator_pb2.AccumulatorResponse) + self.assertTrue(response.payload.value.startswith(b"counter:")) counter += 1 - logging.info(f"[TEST_DEBUG] Received response {counter}: {response}") - logging.info(f"[TEST_DEBUG] Finished iterating, got {counter} responses") - except BaseException as err: - logging.info(f"[TEST_DEBUG] Caught exception: {err}") - self.assertTrue("Simulated error in accumulator handler" in str(err)) - return - self.fail("Expected an exception.") - @patch("psutil.Process.kill", mock_terminate_on_stop) - def test_accumulate_partial_success(self) -> None: - """Test that the first datum is processed before error occurs""" - stub = self.__stub() - request = start_request() - - try: - generator_response = stub.AccumulateFn( - request_iterator=request_generator(count=3, request=request) - ) - - # Try to consume the generator - counter = 0 - for _ in generator_response: - counter += 1 + self.assertEqual(counter, 1, "Expected only one successful response before error") except BaseException as err: self.assertTrue("Simulated error in accumulator handler" in str(err)) return diff --git a/tests/accumulator/test_datatypes.py b/tests/accumulator/test_datatypes.py index e0d94c49..6f2bb1a2 100644 --- a/tests/accumulator/test_datatypes.py +++ b/tests/accumulator/test_datatypes.py @@ -9,13 +9,11 @@ from pynumaflow.accumulator._dtypes import ( IntervalWindow, KeyedWindow, - Metadata, Datum, AccumulatorResult, AccumulatorRequest, WindowOperation, Message, - _AccumulatorBuilderClass, ) from pynumaflow.shared.asynciter import NonBlockingIterator from tests.testing_utils import ( @@ -28,6 +26,7 @@ TEST_KEYS = ["test"] TEST_ID = "test_id" +TEST_HEADERS = {"key1": "value1", "key2": "value2"} class TestDatum(unittest.TestCase): @@ -69,49 +68,21 @@ def test_err_watermark(self): str(context.exception), ) - def test_value(self): - test_headers = {"key1": "value1", "key2": "value2"} + def test_properties(self): d = Datum( keys=TEST_KEYS, value=mock_message(), event_time=mock_event_time(), watermark=mock_watermark(), id_=TEST_ID, - headers=test_headers, + headers=TEST_HEADERS, ) self.assertEqual(mock_message(), d.value) - self.assertEqual(test_headers, d.headers) - self.assertEqual(TEST_ID, d.id) - - def test_key(self): - d = Datum( - keys=TEST_KEYS, - value=mock_message(), - event_time=mock_event_time(), - watermark=mock_watermark(), - id_=TEST_ID, - ) self.assertEqual(TEST_KEYS, d.keys()) - - def test_event_time(self): - d = Datum( - keys=TEST_KEYS, - value=mock_message(), - event_time=mock_event_time(), - watermark=mock_watermark(), - id_=TEST_ID, - ) self.assertEqual(mock_event_time(), d.event_time) - - def test_watermark(self): - d = Datum( - keys=TEST_KEYS, - value=mock_message(), - event_time=mock_event_time(), - watermark=mock_watermark(), - id_=TEST_ID, - ) self.assertEqual(mock_watermark(), d.watermark) + self.assertEqual(TEST_HEADERS, d.headers) + self.assertEqual(TEST_ID, d.id) def test_default_values(self): d = Datum( @@ -158,14 +129,6 @@ def test_window_property(self): self.assertEqual(kw.window.end, mock_end_time()) -class TestMetadata(unittest.TestCase): - def test_interval_window(self): - i = IntervalWindow(start=mock_start_time(), end=mock_end_time()) - m = Metadata(interval_window=i) - self.assertEqual(type(i), type(m.interval_window)) - self.assertEqual(i, m.interval_window) - - class TestAccumulatorResult(unittest.TestCase): def test_create_result(self): # Create mock objects @@ -395,311 +358,5 @@ async def test_datums(): result.close() -class TestAccumulatorBuilderClass(unittest.TestCase): - """Test AccumulatorBuilderClass functionality.""" - - def test_builder_class_creation(self): - """Test AccumulatorBuilderClass creation and instantiation.""" - - class TestAccumulator(Accumulator): - def __init__(self, counter=0): - self.counter = counter - - async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterator): - pass - - builder = _AccumulatorBuilderClass(TestAccumulator, (15,), {}) - instance = builder.create() - - self.assertIsInstance(instance, TestAccumulator) - self.assertEqual(instance.counter, 15) - - def test_builder_class_with_kwargs(self): - """Test AccumulatorBuilderClass with keyword arguments.""" - - class KwargsAccumulator(Accumulator): - def __init__(self, param1, param2, param3=None): - self.param1 = param1 - self.param2 = param2 - self.param3 = param3 - - async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterator): - pass - - builder = _AccumulatorBuilderClass( - KwargsAccumulator, ("arg1", "arg2"), {"param3": "kwarg_value"} - ) - instance = builder.create() - - self.assertIsInstance(instance, KwargsAccumulator) - self.assertEqual(instance.param1, "arg1") - self.assertEqual(instance.param2, "arg2") - self.assertEqual(instance.param3, "kwarg_value") - - def test_builder_class_empty_args(self): - """Test AccumulatorBuilderClass with empty args and kwargs.""" - - class EmptyArgsAccumulator(Accumulator): - def __init__(self): - self.initialized = True - - async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterator): - pass - - builder = _AccumulatorBuilderClass(EmptyArgsAccumulator, (), {}) - instance = builder.create() - - self.assertIsInstance(instance, EmptyArgsAccumulator) - self.assertTrue(instance.initialized) - - -class TestAsyncServerHandlerCoverage(unittest.TestCase): - """Test async server handler function coverage.""" - - def test_get_handler_with_function_and_args_error(self): - """Test get_handler raises TypeError when function handler is passed with init args.""" - from pynumaflow.accumulator.async_server import get_handler - - async def test_func(datums, output): - pass - - with self.assertRaises(TypeError) as context: - get_handler(test_func, init_args=(1, 2)) - - self.assertIn( - "Cannot pass function handler with init args or kwargs", str(context.exception) - ) - - def test_get_handler_with_function_and_kwargs_error(self): - """Test get_handler raises TypeError when function handler is passed with init kwargs.""" - from pynumaflow.accumulator.async_server import get_handler - - async def test_func(datums, output): - pass - - with self.assertRaises(TypeError) as context: - get_handler(test_func, init_kwargs={"test": "value"}) - - self.assertIn( - "Cannot pass function handler with init args or kwargs", str(context.exception) - ) - - def test_get_handler_with_invalid_class(self): - """Test get_handler raises TypeError for invalid class type.""" - from pynumaflow.accumulator.async_server import get_handler - - class InvalidClass: - pass - - with self.assertRaises(TypeError) as context: - get_handler(InvalidClass()) - - # The actual error comes from issubclass() check since we're passing an instance - self.assertIn("issubclass() arg 1 must be a class", str(context.exception)) - - def test_get_handler_with_invalid_class_type(self): - """Test get_handler raises TypeError for invalid Accumulator class type.""" - from pynumaflow.accumulator.async_server import get_handler - - class NonAccumulatorClass: - pass - - with self.assertRaises(TypeError) as context: - get_handler(NonAccumulatorClass) - - # This will hit the 'Invalid Class Type' error path - self.assertIn("Invalid Class Type", str(context.exception)) - - def test_get_handler_with_valid_class(self): - """Test get_handler returns AccumulatorBuilderClass for valid Accumulator subclass.""" - from pynumaflow.accumulator.async_server import get_handler - - class ValidAccumulator(Accumulator): - def __init__(self, counter=0): - self.counter = counter - - async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterator): - pass - - result = get_handler(ValidAccumulator, init_args=(10,), init_kwargs={"counter": 5}) - - self.assertIsInstance(result, _AccumulatorBuilderClass) - - -class TestTaskManagerUtilities(unittest.TestCase): - """Test TaskManager utility functions.""" - - def test_build_window_hash(self): - """Test build_window_hash function.""" - from pynumaflow.accumulator.servicer.task_manager import build_window_hash - from unittest.mock import Mock - - # Create a mock window with ToMilliseconds method - mock_window = Mock() - mock_window.start.ToMilliseconds.return_value = 1000 - mock_window.end.ToMilliseconds.return_value = 2000 - - result = build_window_hash(mock_window) - self.assertEqual(result, "1000:2000") - - def test_build_unique_key_name(self): - """Test build_unique_key_name function.""" - from pynumaflow.accumulator.servicer.task_manager import build_unique_key_name - - keys = ["key1", "key2", "key3"] - result = build_unique_key_name(keys) - - self.assertEqual(result, "key1:key2:key3") - - def test_task_manager_initialization(self): - """Test TaskManager initialization.""" - import asyncio - from pynumaflow.accumulator.servicer.task_manager import TaskManager - from unittest.mock import Mock - - async def run_test(): - handler = Mock() - task_manager = TaskManager(handler) - - self.assertEqual(task_manager._TaskManager__accumulator_handler, handler) - self.assertEqual(len(task_manager.tasks), 0) - self.assertEqual(len(task_manager.background_tasks), 0) - - asyncio.run(run_test()) - - def test_task_manager_get_unique_windows(self): - """Test TaskManager get_unique_windows with empty tasks.""" - import asyncio - from pynumaflow.accumulator.servicer.task_manager import TaskManager - from unittest.mock import Mock - - async def run_test(): - handler = Mock() - task_manager = TaskManager(handler) - - windows = task_manager.get_unique_windows() - self.assertEqual(len(windows), 0) - - asyncio.run(run_test()) - - def test_task_manager_get_tasks(self): - """Test TaskManager get_tasks method.""" - import asyncio - from pynumaflow.accumulator.servicer.task_manager import TaskManager - from unittest.mock import Mock - - async def run_test(): - handler = Mock() - task_manager = TaskManager(handler) - - tasks = task_manager.get_tasks() - self.assertEqual(len(tasks), 0) - - asyncio.run(run_test()) - - def test_task_manager_close_task_not_found(self): - """Test TaskManager close_task when task is not found.""" - import asyncio - from unittest.mock import patch, Mock - from pynumaflow.accumulator.servicer.task_manager import TaskManager - - with patch("pynumaflow.accumulator.servicer.task_manager._LOGGER") as mock_logger: - handler = Mock() - task_manager = TaskManager(handler) - - # Create a mock request with payload that has keys - mock_request = Mock() - mock_datum = Mock() - mock_datum.keys.return_value = ["test_key"] - mock_request.payload = mock_datum - - # Call close_task - should log error and put exception in queue - asyncio.run(task_manager.close_task(mock_request)) - - # Verify logger was called - mock_logger.critical.assert_called_once_with( - "accumulator task not found", exc_info=True - ) - - -class TestAsyncServicerCoverage(unittest.TestCase): - """Test AsyncAccumulatorServicer coverage.""" - - def test_servicer_is_ready_method(self): - """Test AsyncAccumulatorServicer IsReady method.""" - import asyncio - from unittest.mock import Mock - from pynumaflow.accumulator.servicer.async_servicer import AsyncAccumulatorServicer - from pynumaflow.proto.accumulator import accumulator_pb2 - from google.protobuf import empty_pb2 as _empty_pb2 - - async def run_test(): - mock_handler = Mock() - servicer = AsyncAccumulatorServicer(mock_handler) - - # Create mock context and empty request - mock_context = Mock() - empty_request = _empty_pb2.Empty() - - # Call IsReady - response = await servicer.IsReady(empty_request, mock_context) - - # Verify response - self.assertIsInstance(response, accumulator_pb2.ReadyResponse) - self.assertTrue(response.ready) - - asyncio.run(run_test()) - - def test_datum_generator(self): - """Test datum_generator function.""" - import asyncio - from unittest.mock import Mock - from pynumaflow.accumulator.servicer.async_servicer import datum_generator - from tests.testing_utils import ( - get_time_args, - mock_interval_window_start, - mock_interval_window_end, - ) - - async def run_test(): - # Create mock request - event_time_timestamp, watermark_timestamp = get_time_args() - - mock_request = Mock() - mock_request.operation.event = WindowOperation.OPEN.value - mock_request.operation.keyedWindow.start.ToDatetime.return_value = ( - mock_interval_window_start().ToDatetime() - ) - mock_request.operation.keyedWindow.end.ToDatetime.return_value = ( - mock_interval_window_end().ToDatetime() - ) - mock_request.operation.keyedWindow.slot = "slot-0" - mock_request.operation.keyedWindow.keys = ["test_key"] - - mock_request.payload.keys = ["test_key"] - mock_request.payload.value = b"test_value" - mock_request.payload.event_time.ToDatetime.return_value = ( - event_time_timestamp.ToDatetime() - ) - mock_request.payload.watermark.ToDatetime.return_value = ( - watermark_timestamp.ToDatetime() - ) - mock_request.payload.id = "test_id" - mock_request.payload.headers = {"header1": "value1"} - - async def mock_request_iterator(): - yield mock_request - - results = [] - async for result in datum_generator(mock_request_iterator()): - results.append(result) - - self.assertEqual(len(results), 1) - self.assertIsInstance(results[0], AccumulatorRequest) - self.assertEqual(results[0].operation, WindowOperation.OPEN.value) - - asyncio.run(run_test()) - - if __name__ == "__main__": unittest.main() diff --git a/tests/accumulator/test_use_cases.py b/tests/accumulator/test_use_cases.py index 643c3c8b..2f784282 100644 --- a/tests/accumulator/test_use_cases.py +++ b/tests/accumulator/test_use_cases.py @@ -1,3 +1,41 @@ +"""Accumulator Use Case Examples + +These tests demonstrate working patterns for common accumulator use cases. + +Functionality Being Tested: +=========================== + +1. StreamSorterAccumulator: + - Collects incoming Datum objects from async streams + - Sorts events by event_time in chronological order + - Emits sorted events as Message objects + - Manages internal buffer state between processing windows + +2. AccumulatorAsyncServer Integration: + - Server instantiation with custom accumulator classes + - Initialization argument passing to accumulator constructors + - Server configuration with different accumulator types + +Verification Criteria: +===================== + +1. End-to-End Processing: + - Input: Out-of-order events with different timestamps + - Processing: Async stream handling and sorting logic + - Output: Chronologically ordered Message objects + - State Management: Buffer clearing after processing + +2. Server Integration: + - Server creation with various accumulator configurations + - Proper servicer instantiation and lifecycle + - Support for parameterized accumulator constructors + +3. Data Flow Validation: + - Datum to Message conversion patterns + - Async iterator usage with STREAM_EOF handling + - Message format and content verification +""" + import asyncio import unittest from collections.abc import AsyncIterable @@ -11,6 +49,7 @@ ) from pynumaflow.shared.asynciter import NonBlockingIterator from pynumaflow._constants import STREAM_EOF +from tests.accumulator.utils import create_test_datum class StreamSorterAccumulator(Accumulator): @@ -48,619 +87,125 @@ async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterato self.buffer.clear() -class StreamJoinerAccumulator(Accumulator): - """ - Accumulator that joins streams from different sources based on keys. - This demonstrates stream joining use case. - """ - - def __init__(self): - self.streams: dict[str, list[Datum]] = {} - - async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterator): - # Group datums by source (using first key as source identifier) - async for datum in datums: - source_key = datum.keys()[0] if datum.keys() else "default" - if source_key not in self.streams: - self.streams[source_key] = [] - self.streams[source_key].append(datum) - - # Join streams by combining data from different sources - # Sort all data by event_time across all sources - all_datums = [] - for source_datums in self.streams.values(): - all_datums.extend(source_datums) - - # Sort by event_time for temporal ordering - all_datums.sort(key=lambda d: d.event_time) - - # Emit joined and sorted stream - for datum in all_datums: - joined_value = f"joined_{datum.value.decode()}" - await output.put( - Message( - value=joined_value.encode(), - keys=datum.keys(), - tags=["joined"], - ) - ) - - # Clear streams for next window - self.streams.clear() - - -class ReorderingAccumulator(Accumulator): - """ - Accumulator that handles out-of-order events by reordering them. - This demonstrates event reordering use case. - """ - - def __init__(self, max_delay_seconds: int = 5): - self.max_delay_seconds = max_delay_seconds - self.event_buffer: list[Datum] = [] - - async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterator): - # Collect all datums - async for datum in datums: - self.event_buffer.append(datum) - - # Sort by event_time to handle out-of-order events - self.event_buffer.sort(key=lambda d: d.event_time) - - # Process events that are within the acceptable delay window - current_time = datetime.now(timezone.utc) - processed_events = [] - - for datum in self.event_buffer: - # Check if event is within acceptable delay - delay = (current_time - datum.event_time).total_seconds() - if delay <= self.max_delay_seconds: - processed_events.append(datum) - else: - # Event is too old, but we still process it with a tag - reordered_datum = datum - await output.put( - Message( - value=reordered_datum.value, - keys=reordered_datum.keys(), - tags=["reordered", "delayed"], - ) - ) - - # Emit processed events in order - for datum in processed_events: - await output.put( - Message( - value=datum.value, - keys=datum.keys(), - tags=["reordered"], - ) - ) - - # Clear buffer - self.event_buffer.clear() - - -class TimeBasedCorrelationAccumulator(Accumulator): - """ - Accumulator that correlates events from different sources based on timestamps. - This demonstrates time-based correlation use case. - """ - - def __init__(self, correlation_window_seconds: int = 10): - self.correlation_window_seconds = correlation_window_seconds - self.events: list[Datum] = [] - - async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterator): - # Collect all datums - async for datum in datums: - self.events.append(datum) - - # Sort by event_time - self.events.sort(key=lambda d: d.event_time) - - # Correlate events within time windows - correlated_groups = [] - current_group = [] - - for event in self.events: - if not current_group: - current_group.append(event) - else: - # Check if event is within correlation window - time_diff = (event.event_time - current_group[0].event_time).total_seconds() - if time_diff <= self.correlation_window_seconds: - current_group.append(event) - else: - # Start new group - correlated_groups.append(current_group) - current_group = [event] - - # Add the last group - if current_group: - correlated_groups.append(current_group) - - # Emit correlated events - for group_idx, group in enumerate(correlated_groups): - for event in group: - correlation_id = f"corr_group_{group_idx}" - correlated_value = f"correlated_{event.value.decode()}" - await output.put( - Message( - value=correlated_value.encode(), - keys=event.keys() + [correlation_id], - tags=["correlated"], - ) - ) - - # Clear events - self.events.clear() - - -class CustomTriggerAccumulator(Accumulator): - """ - Accumulator that triggers actions based on custom conditions. - This demonstrates custom triggering use case. - """ - - def __init__(self, trigger_count: int = 3): - self.trigger_count = trigger_count - self.accumulated_events: list[Datum] = [] - - async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterator): - # Collect datums - async for datum in datums: - self.accumulated_events.append(datum) - - # Custom trigger: when we have enough events or specific conditions - if len(self.accumulated_events) >= self.trigger_count: - # Trigger action: process all accumulated events - total_value = sum( - int(event.value.decode()) - for event in self.accumulated_events - if event.value.decode().isdigit() - ) +class TestAccumulatorUseCases(unittest.TestCase): + """Test practical accumulator use cases that developers can reference.""" - # Emit triggered result - await output.put( - Message( - value=f"triggered_sum_{total_value}".encode(), - keys=["triggered"], - tags=["custom_trigger"], - ) - ) + def test_stream_sorter_end_to_end_example(self): + """Complete end-to-end example showing how to build a stream sorting accumulator. - # Clear accumulated events - self.accumulated_events.clear() - else: - # Not enough events to trigger, emit individual events - for event in self.accumulated_events: - await output.put( - Message( - value=event.value, - keys=event.keys(), - tags=["pending_trigger"], - ) - ) + This test demonstrates: + 1. How to implement the Accumulator abstract class + 2. How to process async streams of Datum objects + 3. How to emit results as Message objects + 4. Proper resource management (buffer clearing) + """ - -class TestAccumulatorUseCases(unittest.TestCase): - def test_stream_sorter_accumulator(self): - """Test the stream sorting use case""" - sorter = StreamSorterAccumulator() - - # Test that the accumulator sorts by event_time - self.assertIsInstance(sorter, StreamSorterAccumulator) - self.assertEqual(len(sorter.buffer), 0) - - def test_stream_joiner_accumulator(self): - """Test the stream joining use case""" - joiner = StreamJoinerAccumulator() - - # Test that the accumulator can join streams - self.assertIsInstance(joiner, StreamJoinerAccumulator) - self.assertEqual(len(joiner.streams), 0) - - def test_reordering_accumulator(self): - """Test the event reordering use case""" - reorderer = ReorderingAccumulator(max_delay_seconds=10) - - # Test that the accumulator handles reordering - self.assertIsInstance(reorderer, ReorderingAccumulator) - self.assertEqual(reorderer.max_delay_seconds, 10) - self.assertEqual(len(reorderer.event_buffer), 0) - - def test_time_based_correlation_accumulator(self): - """Test the time-based correlation use case""" - correlator = TimeBasedCorrelationAccumulator(correlation_window_seconds=5) - - # Test that the accumulator correlates events - self.assertIsInstance(correlator, TimeBasedCorrelationAccumulator) - self.assertEqual(correlator.correlation_window_seconds, 5) - self.assertEqual(len(correlator.events), 0) - - def test_custom_trigger_accumulator(self): - """Test the custom triggering use case""" - trigger = CustomTriggerAccumulator(trigger_count=5) - - # Test that the accumulator handles custom triggers - self.assertIsInstance(trigger, CustomTriggerAccumulator) - self.assertEqual(trigger.trigger_count, 5) - self.assertEqual(len(trigger.accumulated_events), 0) - - def test_accumulator_server_with_use_cases(self): - """Test that AccumulatorAsyncServer can be created with use case implementations""" - # Test with StreamSorterAccumulator - server1 = AccumulatorAsyncServer(StreamSorterAccumulator) - self.assertIsNotNone(server1) - - # Test with StreamJoinerAccumulator - server2 = AccumulatorAsyncServer(StreamJoinerAccumulator) - self.assertIsNotNone(server2) - - # Test with ReorderingAccumulator with init args - server3 = AccumulatorAsyncServer(ReorderingAccumulator, init_args=(10,)) - self.assertIsNotNone(server3) - - # Test with TimeBasedCorrelationAccumulator with init args - server4 = AccumulatorAsyncServer(TimeBasedCorrelationAccumulator, init_args=(15,)) - self.assertIsNotNone(server4) - - # Test with CustomTriggerAccumulator with init args - server5 = AccumulatorAsyncServer(CustomTriggerAccumulator, init_args=(3,)) - self.assertIsNotNone(server5) - - def test_stream_sorter_functionality(self): - """Test actual sorting functionality""" - - async def _test_stream_sorter_functionality(): + async def _test_end_to_end(): + # Create accumulator instance sorter = StreamSorterAccumulator() output = NonBlockingIterator() - # Create datums with different event times (out of order) - datums = [ - Datum( - keys=["test"], - value=b"event_3", - event_time=datetime.fromtimestamp(1662998460, timezone.utc), - watermark=datetime.fromtimestamp(1662998460, timezone.utc), - id_="3", + # Create test data - intentionally out of chronological order + test_data = [ + create_test_datum( + keys=["sensor_data"], + value=b"temperature:25.5", + event_time=datetime.fromtimestamp(1662998460, timezone.utc), # Latest + id_="temp_3", ), - Datum( - keys=["test"], - value=b"event_1", - event_time=datetime.fromtimestamp(1662998400, timezone.utc), - watermark=datetime.fromtimestamp(1662998400, timezone.utc), - id_="1", + create_test_datum( + keys=["sensor_data"], + value=b"temperature:22.1", + event_time=datetime.fromtimestamp(1662998400, timezone.utc), # Earliest + id_="temp_1", ), - Datum( - keys=["test"], - value=b"event_2", - event_time=datetime.fromtimestamp(1662998430, timezone.utc), - watermark=datetime.fromtimestamp(1662998430, timezone.utc), - id_="2", + create_test_datum( + keys=["sensor_data"], + value=b"temperature:23.8", + event_time=datetime.fromtimestamp(1662998430, timezone.utc), # Middle + id_="temp_2", ), ] - async def datum_generator(): - for datum in datums: + # Create async generator (simulates streaming data) + async def data_stream(): + for datum in test_data: yield datum - # Process the datums - await sorter.handler(datum_generator(), output) + # Process the stream through our accumulator + await sorter.handler(data_stream(), output) - # Send STREAM_EOF to signal completion (this is normally done by TaskManager) - await output.put(STREAM_EOF) - - # Now let's read from the output to see what was written + # Collect and verify results results = [] - reader = output.read_iterator() - async for item in reader: - results.append(item) + await output.put(STREAM_EOF) # Signal end of stream + + async for item in output.read_iterator(): if item == STREAM_EOF: break - # Safety break to avoid infinite loop - if len(results) >= 10: + results.append(item) + if len(results) >= 10: # Safety break break - # Verify the buffer is cleared - self.assertEqual(len(sorter.buffer), 0) - - asyncio.run(_test_stream_sorter_functionality()) - - def test_stream_joiner_functionality(self): - """Test actual joining functionality""" - - async def _test_stream_joiner_functionality(): - joiner = StreamJoinerAccumulator() - output = NonBlockingIterator() - - # Create datums from different sources - datums = [ - Datum( - keys=["source1"], - value=b"data_from_source1", - event_time=datetime.fromtimestamp(1662998400, timezone.utc), - watermark=datetime.fromtimestamp(1662998400, timezone.utc), - id_="s1_1", - ), - Datum( - keys=["source2"], - value=b"data_from_source2", - event_time=datetime.fromtimestamp(1662998430, timezone.utc), - watermark=datetime.fromtimestamp(1662998430, timezone.utc), - id_="s2_1", - ), - ] - - async def datum_generator(): - for datum in datums: - yield datum - - # Process the datums - await joiner.handler(datum_generator(), output) - - # Verify the streams are cleared - self.assertEqual(len(joiner.streams), 0) - - asyncio.run(_test_stream_joiner_functionality()) - - def test_run_async_tests(self): - """Run the async test methods""" - # This test method is no longer needed since the async tests - # are now properly handled within their respective test methods - pass - - def test_error_handling_scenarios(self): - """Test error handling scenarios in accumulator processing.""" - - async def run_test(): - # Test 1: Function handler called directly - async def func_handler(datums: AsyncIterable[Datum], output: NonBlockingIterator): + # Verify we received the expected number of messages + self.assertEqual(len(results), 3) + + # Verify all results are properly formatted Message objects + for i, result in enumerate(results): + self.assertIsInstance(result, Message) + self.assertEqual(result.keys, ["sensor_data"]) + self.assertEqual(result.tags, []) + # Values should be in chronological order due to sorting + if i == 0: + self.assertEqual(result.value, b"temperature:22.1") # Earliest + elif i == 1: + self.assertEqual(result.value, b"temperature:23.8") # Middle + elif i == 2: + self.assertEqual(result.value, b"temperature:25.5") # Latest + + asyncio.run(_test_end_to_end()) + + def test_accumulator_server_integration_example(self): + """Example showing how to integrate custom accumulator with AccumulatorAsyncServer. + + This demonstrates: + 1. Server instantiation with custom accumulator class + 2. Passing initialization arguments to accumulator constructors + 3. Server lifecycle management + """ + # Example 1: Simple accumulator without initialization args + server = AccumulatorAsyncServer(StreamSorterAccumulator) + self.assertIsNotNone(server) + self.assertIsNotNone(server.servicer) + + # Example 2: Accumulator with configuration parameters + class ConfigurableAccumulator(Accumulator): + def __init__(self, max_buffer_size: int = 1000, sort_ascending: bool = True): + self.max_buffer_size = max_buffer_size + self.sort_ascending = sort_ascending + self.buffer = [] + + async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterator): + # Collect datums up to max buffer size async for datum in datums: - await output.put(Message(datum.value, keys=datum.keys())) + if len(self.buffer) < self.max_buffer_size: + self.buffer.append(datum) - from pynumaflow.accumulator.async_server import get_handler + # Sort based on configuration + self.buffer.sort(key=lambda d: d.event_time, reverse=not self.sort_ascending) - handler = get_handler(func_handler) - self.assertEqual(handler, func_handler) + # Emit results + for datum in self.buffer: + await output.put(Message(value=datum.value, keys=datum.keys(), tags=["sorted"])) - # Test 2: Task manager with function handler - from pynumaflow.accumulator.servicer.task_manager import TaskManager - - task_manager = TaskManager(func_handler) - # Create mock request iterator that simulates function handler path - mock_datum = Datum( - keys=["func_key"], - value=b"test_data", - event_time=datetime.now(), - watermark=datetime.now(), - id_="func_test", - ) - - output_queue = NonBlockingIterator() - - # Create an async iterator that yields our mock datum - async def datum_iter(): - yield mock_datum - - # This should cover the function handler path in __invoke_accumulator - await task_manager._TaskManager__invoke_accumulator(datum_iter(), output_queue) - - # Verify the function handler was called by checking for output - results = [] - try: - async for item in output_queue.read_iterator(): - results.append(item) - if item == STREAM_EOF: - break - # Only expect one message for this test - if len(results) >= 3: # Safety break to avoid infinite loop - break - except Exception: - pass # Expected behavior for this simplified test - - # Verify we got at least one result (message was processed) - self.assertGreaterEqual(len(results), 1) - - asyncio.run(run_test()) - - def test_task_manager_error_scenarios(self): - """Test various error scenarios in TaskManager.""" - import asyncio - - async def run_test(): - # Test 1: Unknown window operation - from pynumaflow.accumulator.servicer.task_manager import TaskManager - from unittest.mock import Mock - - handler = Mock() - task_manager = TaskManager(handler) - - # Test 2: Watermark update with AccumulatorResult - from pynumaflow.accumulator._dtypes import AccumulatorResult - - # Create a task to test watermark updating - initial_watermark = datetime.fromtimestamp(500) - task = AccumulatorResult( - _future=Mock(), - _iterator=Mock(), - _key=["test_key"], - _result_queue=Mock(), - _consumer_future=Mock(), - _latest_watermark=initial_watermark, - ) - - # Test update_watermark method directly - new_watermark = datetime.fromtimestamp(2000) - task.update_watermark(new_watermark) - self.assertEqual(task.latest_watermark, new_watermark) - - # Test 3: Test direct instantiation and basic functionality - unified_key = "test_key" - task_manager.tasks[unified_key] = task - - # Verify task was added - self.assertIn(unified_key, task_manager.tasks) - self.assertEqual(task_manager.tasks[unified_key], task) - - asyncio.run(run_test()) - - def test_edge_case_scenarios(self): - """Test edge cases and error conditions.""" - - async def run_test(): - from pynumaflow.accumulator.servicer.task_manager import TaskManager - from unittest.mock import Mock - - # Test 1: Error handling in EOF counting - handler = Mock() - task_manager = TaskManager(handler) - - # Manually set up a scenario where EOF count exceeds expected - task_manager._expected_eof_count = 1 - task_manager._received_eof_count = 2 # More than expected - - # Create mock task for testing - from pynumaflow.accumulator._dtypes import AccumulatorResult - - mock_task = AccumulatorResult( - _future=Mock(), - _iterator=Mock(), - _key=["edge_test"], - _result_queue=Mock(), - _consumer_future=Mock(), - _latest_watermark=datetime.now(), - ) - - unified_key = "edge_test" - task_manager.tasks[unified_key] = mock_task - - # Test watermark handling with None values - input_queue = NonBlockingIterator() - output_queue = NonBlockingIterator() - - # Create message with None watermark and event_time - edge_message = Message( - value=b"edge_test", - keys=["edge_test"], - watermark=None, # Test None watermark handling - event_time=None, # Test None event_time handling - ) - - await input_queue.put(edge_message) - await input_queue.put(STREAM_EOF) - - # This should handle None watermark and event_time without error - await task_manager.write_to_global_queue(input_queue, output_queue, unified_key) - - # Verify output was generated - results = [] - async for item in output_queue.read_iterator(): - results.append(item) - if len(results) >= 2: # Message + EOF response - break + # Clean up + self.buffer.clear() - self.assertTrue(len(results) >= 2) - - asyncio.run(run_test()) - - def test_abstract_method_coverage(self): - """Test abstract method coverage.""" - - # Test calling the abstract handler method directly - class DirectTestAccumulator(Accumulator): - pass # Don't implement handler to test abstract method - - # This should raise TypeError due to abstract method - with self.assertRaises(TypeError): - DirectTestAccumulator() - - def test_servicer_error_handling(self): - """Test error handling in AsyncAccumulatorServicer.""" - - async def run_test(): - from pynumaflow.accumulator.servicer.async_servicer import AsyncAccumulatorServicer - from unittest.mock import Mock, patch - - # Test exception in consumer loop - mock_handler = Mock() - servicer = AsyncAccumulatorServicer(mock_handler) - mock_context = Mock() - - async def failing_request_iterator(): - yield Mock() # Just one request - - # Mock TaskManager to simulate error in consumer - with patch("pynumaflow.accumulator.servicer.async_servicer.TaskManager") as mock_tm: - mock_task_manager = Mock() - mock_tm.return_value = mock_task_manager - - # Mock read_iterator to return an async generator that raises exception - async def failing_reader(): - raise RuntimeError("Consumer error") - - mock_result_queue = Mock() - # Mock the method to return a coroutine when called - mock_result_queue.read_iterator = lambda: failing_reader() - mock_task_manager.global_result_queue = mock_result_queue - - # Mock process_input_stream to return a coroutine - async def mock_process(): - pass - - # Mock the method to return a coroutine when called - mock_task_manager.process_input_stream = lambda x: mock_process() - - # This should handle the consumer exception - with patch( - "pynumaflow.accumulator.servicer.async_servicer.handle_async_error" - ) as mock_handle: - results = [] - async for result in servicer.AccumulateFn( - failing_request_iterator(), mock_context - ): - results.append(result) - - # Should have called error handler - mock_handle.assert_called() - - # Test exception in producer wait - with patch("pynumaflow.accumulator.servicer.async_servicer.TaskManager") as mock_tm2: - mock_task_manager2 = Mock() - mock_tm2.return_value = mock_task_manager2 - - # Mock read_iterator to return an async generator - async def normal_reader(): - return - yield # Empty generator - - mock_result_queue2 = Mock() - # Mock the method to return a coroutine when called - mock_result_queue2.read_iterator = lambda: normal_reader() - mock_task_manager2.global_result_queue = mock_result_queue2 - - # Mock process_input_stream to return a coroutine that raises exception - async def failing_process(): - raise RuntimeError("Producer error") - - # Mock the method to return a coroutine when called - mock_task_manager2.process_input_stream = lambda x: failing_process() - - # This should handle the producer exception - with patch( - "pynumaflow.accumulator.servicer.async_servicer.handle_async_error" - ) as mock_handle2: - results2 = [] - async for result in servicer.AccumulateFn( - failing_request_iterator(), mock_context - ): - results2.append(result) - - # Should have called error handler for producer error - mock_handle2.assert_called() - - asyncio.run(run_test()) + # Create server with custom configuration + configured_server = AccumulatorAsyncServer( + ConfigurableAccumulator, + init_args=(500, False), # max_buffer_size=500, sort_ascending=False + ) + self.assertIsNotNone(configured_server) if __name__ == "__main__": diff --git a/tests/accumulator/utils.py b/tests/accumulator/utils.py index 07cd4703..d0c68fbb 100644 --- a/tests/accumulator/utils.py +++ b/tests/accumulator/utils.py @@ -1,7 +1,5 @@ from datetime import datetime, timezone from pynumaflow.accumulator import Datum -from pynumaflow.proto.accumulator import accumulator_pb2 -from tests.testing_utils import get_time_args def create_test_datum(keys, value, event_time=None, watermark=None, id_=None, headers=None): @@ -23,166 +21,3 @@ def create_test_datum(keys, value, event_time=None, watermark=None, id_=None, he id_=id_, headers=headers, ) - - -def create_accumulator_request(operation, keys, value, event_time=None, watermark=None): - """Create an AccumulatorRequest for testing""" - if event_time is None or watermark is None: - event_time_timestamp, watermark_timestamp = get_time_args() - else: - event_time_timestamp = event_time - watermark_timestamp = watermark - - window = accumulator_pb2.KeyedWindow( - start=event_time_timestamp, - end=watermark_timestamp, - slot="slot-0", - keys=keys, - ) - - payload = accumulator_pb2.Payload( - keys=keys, - value=value, - event_time=event_time_timestamp, - watermark=watermark_timestamp, - id="test_id", - ) - - operation_proto = accumulator_pb2.AccumulatorRequest.WindowOperation( - event=operation, - keyedWindow=window, - ) - - return accumulator_pb2.AccumulatorRequest( - payload=payload, - operation=operation_proto, - ) - - -def create_out_of_order_datums(): - """Create a list of datums that are out of order by event_time""" - return [ - create_test_datum( - keys=["test"], - value=b"event_3", - event_time=datetime.fromtimestamp(1662998460, timezone.utc), - id_="3", - ), - create_test_datum( - keys=["test"], - value=b"event_1", - event_time=datetime.fromtimestamp(1662998400, timezone.utc), - id_="1", - ), - create_test_datum( - keys=["test"], - value=b"event_2", - event_time=datetime.fromtimestamp(1662998430, timezone.utc), - id_="2", - ), - ] - - -def create_multi_source_datums(): - """Create datums from multiple sources for stream joining tests""" - return [ - create_test_datum( - keys=["source1"], - value=b"data_from_source1_1", - event_time=datetime.fromtimestamp(1662998400, timezone.utc), - id_="s1_1", - ), - create_test_datum( - keys=["source2"], - value=b"data_from_source2_1", - event_time=datetime.fromtimestamp(1662998410, timezone.utc), - id_="s2_1", - ), - create_test_datum( - keys=["source1"], - value=b"data_from_source1_2", - event_time=datetime.fromtimestamp(1662998420, timezone.utc), - id_="s1_2", - ), - create_test_datum( - keys=["source2"], - value=b"data_from_source2_2", - event_time=datetime.fromtimestamp(1662998430, timezone.utc), - id_="s2_2", - ), - ] - - -def create_numeric_datums(): - """Create datums with numeric values for trigger tests""" - return [ - create_test_datum( - keys=["test"], - value=b"1", - event_time=datetime.fromtimestamp(1662998400, timezone.utc), - id_="1", - ), - create_test_datum( - keys=["test"], - value=b"2", - event_time=datetime.fromtimestamp(1662998410, timezone.utc), - id_="2", - ), - create_test_datum( - keys=["test"], - value=b"3", - event_time=datetime.fromtimestamp(1662998420, timezone.utc), - id_="3", - ), - create_test_datum( - keys=["test"], - value=b"4", - event_time=datetime.fromtimestamp(1662998430, timezone.utc), - id_="4", - ), - create_test_datum( - keys=["test"], - value=b"5", - event_time=datetime.fromtimestamp(1662998440, timezone.utc), - id_="5", - ), - ] - - -def create_correlation_datums(): - """Create datums for time-based correlation tests""" - base_time = 1662998400 - return [ - # Group 1: events within 5 seconds - create_test_datum( - keys=["correlation"], - value=b"event_a", - event_time=datetime.fromtimestamp(base_time, timezone.utc), - id_="a", - ), - create_test_datum( - keys=["correlation"], - value=b"event_b", - event_time=datetime.fromtimestamp(base_time + 2, timezone.utc), - id_="b", - ), - create_test_datum( - keys=["correlation"], - value=b"event_c", - event_time=datetime.fromtimestamp(base_time + 4, timezone.utc), - id_="c", - ), - # Group 2: events within another 5 second window - create_test_datum( - keys=["correlation"], - value=b"event_d", - event_time=datetime.fromtimestamp(base_time + 15, timezone.utc), - id_="d", - ), - create_test_datum( - keys=["correlation"], - value=b"event_e", - event_time=datetime.fromtimestamp(base_time + 17, timezone.utc), - id_="e", - ), - ] From 53ec8dac467267284583dd33fe8b321b7431ee65 Mon Sep 17 00:00:00 2001 From: srao12 Date: Sat, 26 Jul 2025 19:56:31 -0700 Subject: [PATCH 26/30] fix: remove extra STREAM_EOF Signed-off-by: srao12 --- pynumaflow/accumulator/servicer/task_manager.py | 5 ----- 1 file changed, 5 deletions(-) diff --git a/pynumaflow/accumulator/servicer/task_manager.py b/pynumaflow/accumulator/servicer/task_manager.py index 523a1aee..a7c80968 100644 --- a/pynumaflow/accumulator/servicer/task_manager.py +++ b/pynumaflow/accumulator/servicer/task_manager.py @@ -263,11 +263,6 @@ async def process_input_stream( fut = task.future await fut - # # Send an EOF message to the local result queue - # # This will signal that the task has completed processing - # TODO: remove this and test end to end as we are sending EOF in _invoke_accumulator - await task.result_queue.put(STREAM_EOF) - # Wait for the local queue to write # all the results of this task to the global result queue con_future = task.consumer_future From e1a7c9a2b722951cd6277244b28ad1fcef4db062 Mon Sep 17 00:00:00 2001 From: srao12 Date: Sat, 26 Jul 2025 20:27:46 -0700 Subject: [PATCH 27/30] fix: lint Signed-off-by: srao12 --- .../proto/accumulator/accumulator_pb2.py | 53 +-- .../proto/accumulator/accumulator_pb2.pyi | 49 ++- .../proto/accumulator/accumulator_pb2_grpc.py | 127 +++++--- pynumaflow/proto/mapper/map_pb2.py | 53 +-- pynumaflow/proto/mapper/map_pb2.pyi | 43 ++- pynumaflow/proto/mapper/map_pb2_grpc.py | 127 +++++--- pynumaflow/proto/reducer/reduce_pb2.py | 53 +-- pynumaflow/proto/reducer/reduce_pb2.pyi | 55 +++- pynumaflow/proto/reducer/reduce_pb2_grpc.py | 127 +++++--- pynumaflow/proto/sideinput/sideinput_pb2.py | 21 +- .../proto/sideinput/sideinput_pb2_grpc.py | 127 +++++--- pynumaflow/proto/sinker/sink_pb2.py | 53 +-- pynumaflow/proto/sinker/sink_pb2.pyi | 43 ++- pynumaflow/proto/sinker/sink_pb2_grpc.py | 127 +++++--- pynumaflow/proto/sourcer/source_pb2.py | 93 +++--- pynumaflow/proto/sourcer/source_pb2.pyi | 84 ++++- pynumaflow/proto/sourcer/source_pb2_grpc.py | 304 +++++++++++------- .../proto/sourcetransformer/transform_pb2.py | 45 +-- .../proto/sourcetransformer/transform_pb2.pyi | 42 ++- .../sourcetransformer/transform_pb2_grpc.py | 124 ++++--- 20 files changed, 1099 insertions(+), 651 deletions(-) diff --git a/pynumaflow/proto/accumulator/accumulator_pb2.py b/pynumaflow/proto/accumulator/accumulator_pb2.py index bc29e898..f1e8ec8d 100644 --- a/pynumaflow/proto/accumulator/accumulator_pb2.py +++ b/pynumaflow/proto/accumulator/accumulator_pb2.py @@ -7,6 +7,7 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import symbol_database as _symbol_database from google.protobuf.internal import builder as _builder + # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() @@ -16,32 +17,36 @@ from google.protobuf import timestamp_pb2 as google_dot_protobuf_dot_timestamp__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x11\x61\x63\x63umulator.proto\x12\x0e\x61\x63\x63umulator.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto\"\xf8\x01\n\x07Payload\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\n\n\x02id\x18\x05 \x01(\t\x12\x35\n\x07headers\x18\x06 \x03(\x0b\x32$.accumulator.v1.Payload.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\"\xbe\x02\n\x12\x41\x63\x63umulatorRequest\x12(\n\x07payload\x18\x01 \x01(\x0b\x32\x17.accumulator.v1.Payload\x12\x45\n\toperation\x18\x02 \x01(\x0b\x32\x32.accumulator.v1.AccumulatorRequest.WindowOperation\x1a\xb6\x01\n\x0fWindowOperation\x12G\n\x05\x65vent\x18\x01 \x01(\x0e\x32\x38.accumulator.v1.AccumulatorRequest.WindowOperation.Event\x12\x30\n\x0bkeyedWindow\x18\x02 \x01(\x0b\x32\x1b.accumulator.v1.KeyedWindow\"(\n\x05\x45vent\x12\x08\n\x04OPEN\x10\x00\x12\t\n\x05\x43LOSE\x10\x01\x12\n\n\x06\x41PPEND\x10\x02\"}\n\x0bKeyedWindow\x12)\n\x05start\x18\x01 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\'\n\x03\x65nd\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04slot\x18\x03 \x01(\t\x12\x0c\n\x04keys\x18\x04 \x03(\t\"\x87\x01\n\x13\x41\x63\x63umulatorResponse\x12(\n\x07payload\x18\x01 \x01(\x0b\x32\x17.accumulator.v1.Payload\x12+\n\x06window\x18\x02 \x01(\x0b\x32\x1b.accumulator.v1.KeyedWindow\x12\x0c\n\x04tags\x18\x03 \x03(\t\x12\x0b\n\x03\x45OF\x18\x04 \x01(\x08\"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\xac\x01\n\x0b\x41\x63\x63umulator\x12[\n\x0c\x41\x63\x63umulateFn\x12\".accumulator.v1.AccumulatorRequest\x1a#.accumulator.v1.AccumulatorResponse(\x01\x30\x01\x12@\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x1d.accumulator.v1.ReadyResponseBd\n#io.numaproj.numaflow.accumulator.v1Z=github.com/numaproj/numaflow-go/pkg/apis/proto/accumulator/v1b\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( + b'\n\x11\x61\x63\x63umulator.proto\x12\x0e\x61\x63\x63umulator.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto"\xf8\x01\n\x07Payload\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\n\n\x02id\x18\x05 \x01(\t\x12\x35\n\x07headers\x18\x06 \x03(\x0b\x32$.accumulator.v1.Payload.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01"\xbe\x02\n\x12\x41\x63\x63umulatorRequest\x12(\n\x07payload\x18\x01 \x01(\x0b\x32\x17.accumulator.v1.Payload\x12\x45\n\toperation\x18\x02 \x01(\x0b\x32\x32.accumulator.v1.AccumulatorRequest.WindowOperation\x1a\xb6\x01\n\x0fWindowOperation\x12G\n\x05\x65vent\x18\x01 \x01(\x0e\x32\x38.accumulator.v1.AccumulatorRequest.WindowOperation.Event\x12\x30\n\x0bkeyedWindow\x18\x02 \x01(\x0b\x32\x1b.accumulator.v1.KeyedWindow"(\n\x05\x45vent\x12\x08\n\x04OPEN\x10\x00\x12\t\n\x05\x43LOSE\x10\x01\x12\n\n\x06\x41PPEND\x10\x02"}\n\x0bKeyedWindow\x12)\n\x05start\x18\x01 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\'\n\x03\x65nd\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04slot\x18\x03 \x01(\t\x12\x0c\n\x04keys\x18\x04 \x03(\t"\x87\x01\n\x13\x41\x63\x63umulatorResponse\x12(\n\x07payload\x18\x01 \x01(\x0b\x32\x17.accumulator.v1.Payload\x12+\n\x06window\x18\x02 \x01(\x0b\x32\x1b.accumulator.v1.KeyedWindow\x12\x0c\n\x04tags\x18\x03 \x03(\t\x12\x0b\n\x03\x45OF\x18\x04 \x01(\x08"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\xac\x01\n\x0b\x41\x63\x63umulator\x12[\n\x0c\x41\x63\x63umulateFn\x12".accumulator.v1.AccumulatorRequest\x1a#.accumulator.v1.AccumulatorResponse(\x01\x30\x01\x12@\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x1d.accumulator.v1.ReadyResponseBd\n#io.numaproj.numaflow.accumulator.v1Z=github.com/numaproj/numaflow-go/pkg/apis/proto/accumulator/v1b\x06proto3' +) _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) -_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'accumulator_pb2', _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, "accumulator_pb2", _globals) if _descriptor._USE_C_DESCRIPTORS == False: - _globals['DESCRIPTOR']._options = None - _globals['DESCRIPTOR']._serialized_options = b'\n#io.numaproj.numaflow.accumulator.v1Z=github.com/numaproj/numaflow-go/pkg/apis/proto/accumulator/v1' - _globals['_PAYLOAD_HEADERSENTRY']._options = None - _globals['_PAYLOAD_HEADERSENTRY']._serialized_options = b'8\001' - _globals['_PAYLOAD']._serialized_start=100 - _globals['_PAYLOAD']._serialized_end=348 - _globals['_PAYLOAD_HEADERSENTRY']._serialized_start=302 - _globals['_PAYLOAD_HEADERSENTRY']._serialized_end=348 - _globals['_ACCUMULATORREQUEST']._serialized_start=351 - _globals['_ACCUMULATORREQUEST']._serialized_end=669 - _globals['_ACCUMULATORREQUEST_WINDOWOPERATION']._serialized_start=487 - _globals['_ACCUMULATORREQUEST_WINDOWOPERATION']._serialized_end=669 - _globals['_ACCUMULATORREQUEST_WINDOWOPERATION_EVENT']._serialized_start=629 - _globals['_ACCUMULATORREQUEST_WINDOWOPERATION_EVENT']._serialized_end=669 - _globals['_KEYEDWINDOW']._serialized_start=671 - _globals['_KEYEDWINDOW']._serialized_end=796 - _globals['_ACCUMULATORRESPONSE']._serialized_start=799 - _globals['_ACCUMULATORRESPONSE']._serialized_end=934 - _globals['_READYRESPONSE']._serialized_start=936 - _globals['_READYRESPONSE']._serialized_end=966 - _globals['_ACCUMULATOR']._serialized_start=969 - _globals['_ACCUMULATOR']._serialized_end=1141 + _globals["DESCRIPTOR"]._options = None + _globals[ + "DESCRIPTOR" + ]._serialized_options = b"\n#io.numaproj.numaflow.accumulator.v1Z=github.com/numaproj/numaflow-go/pkg/apis/proto/accumulator/v1" + _globals["_PAYLOAD_HEADERSENTRY"]._options = None + _globals["_PAYLOAD_HEADERSENTRY"]._serialized_options = b"8\001" + _globals["_PAYLOAD"]._serialized_start = 100 + _globals["_PAYLOAD"]._serialized_end = 348 + _globals["_PAYLOAD_HEADERSENTRY"]._serialized_start = 302 + _globals["_PAYLOAD_HEADERSENTRY"]._serialized_end = 348 + _globals["_ACCUMULATORREQUEST"]._serialized_start = 351 + _globals["_ACCUMULATORREQUEST"]._serialized_end = 669 + _globals["_ACCUMULATORREQUEST_WINDOWOPERATION"]._serialized_start = 487 + _globals["_ACCUMULATORREQUEST_WINDOWOPERATION"]._serialized_end = 669 + _globals["_ACCUMULATORREQUEST_WINDOWOPERATION_EVENT"]._serialized_start = 629 + _globals["_ACCUMULATORREQUEST_WINDOWOPERATION_EVENT"]._serialized_end = 669 + _globals["_KEYEDWINDOW"]._serialized_start = 671 + _globals["_KEYEDWINDOW"]._serialized_end = 796 + _globals["_ACCUMULATORRESPONSE"]._serialized_start = 799 + _globals["_ACCUMULATORRESPONSE"]._serialized_end = 934 + _globals["_READYRESPONSE"]._serialized_start = 936 + _globals["_READYRESPONSE"]._serialized_end = 966 + _globals["_ACCUMULATOR"]._serialized_start = 969 + _globals["_ACCUMULATOR"]._serialized_end = 1141 # @@protoc_insertion_point(module_scope) diff --git a/pynumaflow/proto/accumulator/accumulator_pb2.pyi b/pynumaflow/proto/accumulator/accumulator_pb2.pyi index bfa0bbac..d9f0f7a5 100644 --- a/pynumaflow/proto/accumulator/accumulator_pb2.pyi +++ b/pynumaflow/proto/accumulator/accumulator_pb2.pyi @@ -4,12 +4,19 @@ from google.protobuf.internal import containers as _containers from google.protobuf.internal import enum_type_wrapper as _enum_type_wrapper from google.protobuf import descriptor as _descriptor from google.protobuf import message as _message -from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union +from typing import ( + ClassVar as _ClassVar, + Iterable as _Iterable, + Mapping as _Mapping, + Optional as _Optional, + Union as _Union, +) DESCRIPTOR: _descriptor.FileDescriptor class Payload(_message.Message): __slots__ = ("keys", "value", "event_time", "watermark", "id", "headers") + class HeadersEntry(_message.Message): __slots__ = ("key", "value") KEY_FIELD_NUMBER: _ClassVar[int] @@ -29,12 +36,22 @@ class Payload(_message.Message): watermark: _timestamp_pb2.Timestamp id: str headers: _containers.ScalarMap[str, str] - def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., id: _Optional[str] = ..., headers: _Optional[_Mapping[str, str]] = ...) -> None: ... + def __init__( + self, + keys: _Optional[_Iterable[str]] = ..., + value: _Optional[bytes] = ..., + event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + id: _Optional[str] = ..., + headers: _Optional[_Mapping[str, str]] = ..., + ) -> None: ... class AccumulatorRequest(_message.Message): __slots__ = ("payload", "operation") + class WindowOperation(_message.Message): __slots__ = ("event", "keyedWindow") + class Event(int, metaclass=_enum_type_wrapper.EnumTypeWrapper): __slots__ = () OPEN: _ClassVar[AccumulatorRequest.WindowOperation.Event] @@ -47,12 +64,20 @@ class AccumulatorRequest(_message.Message): KEYEDWINDOW_FIELD_NUMBER: _ClassVar[int] event: AccumulatorRequest.WindowOperation.Event keyedWindow: KeyedWindow - def __init__(self, event: _Optional[_Union[AccumulatorRequest.WindowOperation.Event, str]] = ..., keyedWindow: _Optional[_Union[KeyedWindow, _Mapping]] = ...) -> None: ... + def __init__( + self, + event: _Optional[_Union[AccumulatorRequest.WindowOperation.Event, str]] = ..., + keyedWindow: _Optional[_Union[KeyedWindow, _Mapping]] = ..., + ) -> None: ... PAYLOAD_FIELD_NUMBER: _ClassVar[int] OPERATION_FIELD_NUMBER: _ClassVar[int] payload: Payload operation: AccumulatorRequest.WindowOperation - def __init__(self, payload: _Optional[_Union[Payload, _Mapping]] = ..., operation: _Optional[_Union[AccumulatorRequest.WindowOperation, _Mapping]] = ...) -> None: ... + def __init__( + self, + payload: _Optional[_Union[Payload, _Mapping]] = ..., + operation: _Optional[_Union[AccumulatorRequest.WindowOperation, _Mapping]] = ..., + ) -> None: ... class KeyedWindow(_message.Message): __slots__ = ("start", "end", "slot", "keys") @@ -64,7 +89,13 @@ class KeyedWindow(_message.Message): end: _timestamp_pb2.Timestamp slot: str keys: _containers.RepeatedScalarFieldContainer[str] - def __init__(self, start: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., end: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., slot: _Optional[str] = ..., keys: _Optional[_Iterable[str]] = ...) -> None: ... + def __init__( + self, + start: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + end: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + slot: _Optional[str] = ..., + keys: _Optional[_Iterable[str]] = ..., + ) -> None: ... class AccumulatorResponse(_message.Message): __slots__ = ("payload", "window", "tags", "EOF") @@ -76,7 +107,13 @@ class AccumulatorResponse(_message.Message): window: KeyedWindow tags: _containers.RepeatedScalarFieldContainer[str] EOF: bool - def __init__(self, payload: _Optional[_Union[Payload, _Mapping]] = ..., window: _Optional[_Union[KeyedWindow, _Mapping]] = ..., tags: _Optional[_Iterable[str]] = ..., EOF: bool = ...) -> None: ... + def __init__( + self, + payload: _Optional[_Union[Payload, _Mapping]] = ..., + window: _Optional[_Union[KeyedWindow, _Mapping]] = ..., + tags: _Optional[_Iterable[str]] = ..., + EOF: bool = ..., + ) -> None: ... class ReadyResponse(_message.Message): __slots__ = ("ready",) diff --git a/pynumaflow/proto/accumulator/accumulator_pb2_grpc.py b/pynumaflow/proto/accumulator/accumulator_pb2_grpc.py index 4f0f208f..f41606dd 100644 --- a/pynumaflow/proto/accumulator/accumulator_pb2_grpc.py +++ b/pynumaflow/proto/accumulator/accumulator_pb2_grpc.py @@ -19,15 +19,15 @@ def __init__(self, channel): channel: A grpc.Channel. """ self.AccumulateFn = channel.stream_stream( - '/accumulator.v1.Accumulator/AccumulateFn', - request_serializer=accumulator__pb2.AccumulatorRequest.SerializeToString, - response_deserializer=accumulator__pb2.AccumulatorResponse.FromString, - ) + "/accumulator.v1.Accumulator/AccumulateFn", + request_serializer=accumulator__pb2.AccumulatorRequest.SerializeToString, + response_deserializer=accumulator__pb2.AccumulatorResponse.FromString, + ) self.IsReady = channel.unary_unary( - '/accumulator.v1.Accumulator/IsReady', - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=accumulator__pb2.ReadyResponse.FromString, - ) + "/accumulator.v1.Accumulator/IsReady", + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=accumulator__pb2.ReadyResponse.FromString, + ) class AccumulatorServicer(object): @@ -37,39 +37,38 @@ class AccumulatorServicer(object): """ def AccumulateFn(self, request_iterator, context): - """AccumulateFn applies a accumulate function to a request stream. - """ + """AccumulateFn applies a accumulate function to a request stream.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def IsReady(self, request, context): - """IsReady is the heartbeat endpoint for gRPC. - """ + """IsReady is the heartbeat endpoint for gRPC.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def add_AccumulatorServicer_to_server(servicer, server): rpc_method_handlers = { - 'AccumulateFn': grpc.stream_stream_rpc_method_handler( - servicer.AccumulateFn, - request_deserializer=accumulator__pb2.AccumulatorRequest.FromString, - response_serializer=accumulator__pb2.AccumulatorResponse.SerializeToString, - ), - 'IsReady': grpc.unary_unary_rpc_method_handler( - servicer.IsReady, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=accumulator__pb2.ReadyResponse.SerializeToString, - ), + "AccumulateFn": grpc.stream_stream_rpc_method_handler( + servicer.AccumulateFn, + request_deserializer=accumulator__pb2.AccumulatorRequest.FromString, + response_serializer=accumulator__pb2.AccumulatorResponse.SerializeToString, + ), + "IsReady": grpc.unary_unary_rpc_method_handler( + servicer.IsReady, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=accumulator__pb2.ReadyResponse.SerializeToString, + ), } generic_handler = grpc.method_handlers_generic_handler( - 'accumulator.v1.Accumulator', rpc_method_handlers) + "accumulator.v1.Accumulator", rpc_method_handlers + ) server.add_generic_rpc_handlers((generic_handler,)) - # This class is part of an EXPERIMENTAL API. +# This class is part of an EXPERIMENTAL API. class Accumulator(object): """AccumulatorWindow describes a special kind of SessionWindow (similar to Global Window) where output should always have monotonically increasing WM but it can be manipulated through event-time by reordering the messages. @@ -77,35 +76,59 @@ class Accumulator(object): """ @staticmethod - def AccumulateFn(request_iterator, + def AccumulateFn( + request_iterator, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.stream_stream( + request_iterator, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.stream_stream(request_iterator, target, '/accumulator.v1.Accumulator/AccumulateFn', + "/accumulator.v1.Accumulator/AccumulateFn", accumulator__pb2.AccumulatorRequest.SerializeToString, accumulator__pb2.AccumulatorResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) @staticmethod - def IsReady(request, + def IsReady( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.unary_unary(request, target, '/accumulator.v1.Accumulator/IsReady', + "/accumulator.v1.Accumulator/IsReady", google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, accumulator__pb2.ReadyResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) diff --git a/pynumaflow/proto/mapper/map_pb2.py b/pynumaflow/proto/mapper/map_pb2.py index 78abbd69..80472786 100644 --- a/pynumaflow/proto/mapper/map_pb2.py +++ b/pynumaflow/proto/mapper/map_pb2.py @@ -7,6 +7,7 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import symbol_database as _symbol_database from google.protobuf.internal import builder as _builder + # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() @@ -16,32 +17,36 @@ from google.protobuf import timestamp_pb2 as google_dot_protobuf_dot_timestamp__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\tmap.proto\x12\x06map.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto\"\xac\x03\n\nMapRequest\x12+\n\x07request\x18\x01 \x01(\x0b\x32\x1a.map.v1.MapRequest.Request\x12\n\n\x02id\x18\x02 \x01(\t\x12)\n\thandshake\x18\x03 \x01(\x0b\x32\x11.map.v1.HandshakeH\x00\x88\x01\x01\x12/\n\x06status\x18\x04 \x01(\x0b\x32\x1a.map.v1.TransmissionStatusH\x01\x88\x01\x01\x1a\xef\x01\n\x07Request\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x38\n\x07headers\x18\x05 \x03(\x0b\x32\'.map.v1.MapRequest.Request.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x42\x0c\n\n_handshakeB\t\n\x07_status\"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08\"!\n\x12TransmissionStatus\x12\x0b\n\x03\x65ot\x18\x01 \x01(\x08\"\xf0\x01\n\x0bMapResponse\x12+\n\x07results\x18\x01 \x03(\x0b\x32\x1a.map.v1.MapResponse.Result\x12\n\n\x02id\x18\x02 \x01(\t\x12)\n\thandshake\x18\x03 \x01(\x0b\x32\x11.map.v1.HandshakeH\x00\x88\x01\x01\x12/\n\x06status\x18\x04 \x01(\x0b\x32\x1a.map.v1.TransmissionStatusH\x01\x88\x01\x01\x1a\x33\n\x06Result\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12\x0c\n\x04tags\x18\x03 \x03(\tB\x0c\n\n_handshakeB\t\n\x07_status\"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32u\n\x03Map\x12\x34\n\x05MapFn\x12\x12.map.v1.MapRequest\x1a\x13.map.v1.MapResponse(\x01\x30\x01\x12\x38\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x15.map.v1.ReadyResponseB7Z5github.com/numaproj/numaflow-go/pkg/apis/proto/map/v1b\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( + b'\n\tmap.proto\x12\x06map.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto"\xac\x03\n\nMapRequest\x12+\n\x07request\x18\x01 \x01(\x0b\x32\x1a.map.v1.MapRequest.Request\x12\n\n\x02id\x18\x02 \x01(\t\x12)\n\thandshake\x18\x03 \x01(\x0b\x32\x11.map.v1.HandshakeH\x00\x88\x01\x01\x12/\n\x06status\x18\x04 \x01(\x0b\x32\x1a.map.v1.TransmissionStatusH\x01\x88\x01\x01\x1a\xef\x01\n\x07Request\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x38\n\x07headers\x18\x05 \x03(\x0b\x32\'.map.v1.MapRequest.Request.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x42\x0c\n\n_handshakeB\t\n\x07_status"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08"!\n\x12TransmissionStatus\x12\x0b\n\x03\x65ot\x18\x01 \x01(\x08"\xf0\x01\n\x0bMapResponse\x12+\n\x07results\x18\x01 \x03(\x0b\x32\x1a.map.v1.MapResponse.Result\x12\n\n\x02id\x18\x02 \x01(\t\x12)\n\thandshake\x18\x03 \x01(\x0b\x32\x11.map.v1.HandshakeH\x00\x88\x01\x01\x12/\n\x06status\x18\x04 \x01(\x0b\x32\x1a.map.v1.TransmissionStatusH\x01\x88\x01\x01\x1a\x33\n\x06Result\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12\x0c\n\x04tags\x18\x03 \x03(\tB\x0c\n\n_handshakeB\t\n\x07_status"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32u\n\x03Map\x12\x34\n\x05MapFn\x12\x12.map.v1.MapRequest\x1a\x13.map.v1.MapResponse(\x01\x30\x01\x12\x38\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x15.map.v1.ReadyResponseB7Z5github.com/numaproj/numaflow-go/pkg/apis/proto/map/v1b\x06proto3' +) _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) -_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'map_pb2', _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, "map_pb2", _globals) if _descriptor._USE_C_DESCRIPTORS == False: - _globals['DESCRIPTOR']._options = None - _globals['DESCRIPTOR']._serialized_options = b'Z5github.com/numaproj/numaflow-go/pkg/apis/proto/map/v1' - _globals['_MAPREQUEST_REQUEST_HEADERSENTRY']._options = None - _globals['_MAPREQUEST_REQUEST_HEADERSENTRY']._serialized_options = b'8\001' - _globals['_MAPREQUEST']._serialized_start=84 - _globals['_MAPREQUEST']._serialized_end=512 - _globals['_MAPREQUEST_REQUEST']._serialized_start=248 - _globals['_MAPREQUEST_REQUEST']._serialized_end=487 - _globals['_MAPREQUEST_REQUEST_HEADERSENTRY']._serialized_start=441 - _globals['_MAPREQUEST_REQUEST_HEADERSENTRY']._serialized_end=487 - _globals['_HANDSHAKE']._serialized_start=514 - _globals['_HANDSHAKE']._serialized_end=538 - _globals['_TRANSMISSIONSTATUS']._serialized_start=540 - _globals['_TRANSMISSIONSTATUS']._serialized_end=573 - _globals['_MAPRESPONSE']._serialized_start=576 - _globals['_MAPRESPONSE']._serialized_end=816 - _globals['_MAPRESPONSE_RESULT']._serialized_start=740 - _globals['_MAPRESPONSE_RESULT']._serialized_end=791 - _globals['_READYRESPONSE']._serialized_start=818 - _globals['_READYRESPONSE']._serialized_end=848 - _globals['_MAP']._serialized_start=850 - _globals['_MAP']._serialized_end=967 + _globals["DESCRIPTOR"]._options = None + _globals[ + "DESCRIPTOR" + ]._serialized_options = b"Z5github.com/numaproj/numaflow-go/pkg/apis/proto/map/v1" + _globals["_MAPREQUEST_REQUEST_HEADERSENTRY"]._options = None + _globals["_MAPREQUEST_REQUEST_HEADERSENTRY"]._serialized_options = b"8\001" + _globals["_MAPREQUEST"]._serialized_start = 84 + _globals["_MAPREQUEST"]._serialized_end = 512 + _globals["_MAPREQUEST_REQUEST"]._serialized_start = 248 + _globals["_MAPREQUEST_REQUEST"]._serialized_end = 487 + _globals["_MAPREQUEST_REQUEST_HEADERSENTRY"]._serialized_start = 441 + _globals["_MAPREQUEST_REQUEST_HEADERSENTRY"]._serialized_end = 487 + _globals["_HANDSHAKE"]._serialized_start = 514 + _globals["_HANDSHAKE"]._serialized_end = 538 + _globals["_TRANSMISSIONSTATUS"]._serialized_start = 540 + _globals["_TRANSMISSIONSTATUS"]._serialized_end = 573 + _globals["_MAPRESPONSE"]._serialized_start = 576 + _globals["_MAPRESPONSE"]._serialized_end = 816 + _globals["_MAPRESPONSE_RESULT"]._serialized_start = 740 + _globals["_MAPRESPONSE_RESULT"]._serialized_end = 791 + _globals["_READYRESPONSE"]._serialized_start = 818 + _globals["_READYRESPONSE"]._serialized_end = 848 + _globals["_MAP"]._serialized_start = 850 + _globals["_MAP"]._serialized_end = 967 # @@protoc_insertion_point(module_scope) diff --git a/pynumaflow/proto/mapper/map_pb2.pyi b/pynumaflow/proto/mapper/map_pb2.pyi index 22eed7b9..9832bc3e 100644 --- a/pynumaflow/proto/mapper/map_pb2.pyi +++ b/pynumaflow/proto/mapper/map_pb2.pyi @@ -3,14 +3,22 @@ from google.protobuf import timestamp_pb2 as _timestamp_pb2 from google.protobuf.internal import containers as _containers from google.protobuf import descriptor as _descriptor from google.protobuf import message as _message -from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union +from typing import ( + ClassVar as _ClassVar, + Iterable as _Iterable, + Mapping as _Mapping, + Optional as _Optional, + Union as _Union, +) DESCRIPTOR: _descriptor.FileDescriptor class MapRequest(_message.Message): __slots__ = ("request", "id", "handshake", "status") + class Request(_message.Message): __slots__ = ("keys", "value", "event_time", "watermark", "headers") + class HeadersEntry(_message.Message): __slots__ = ("key", "value") KEY_FIELD_NUMBER: _ClassVar[int] @@ -28,7 +36,14 @@ class MapRequest(_message.Message): event_time: _timestamp_pb2.Timestamp watermark: _timestamp_pb2.Timestamp headers: _containers.ScalarMap[str, str] - def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., headers: _Optional[_Mapping[str, str]] = ...) -> None: ... + def __init__( + self, + keys: _Optional[_Iterable[str]] = ..., + value: _Optional[bytes] = ..., + event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + headers: _Optional[_Mapping[str, str]] = ..., + ) -> None: ... REQUEST_FIELD_NUMBER: _ClassVar[int] ID_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] @@ -37,7 +52,13 @@ class MapRequest(_message.Message): id: str handshake: Handshake status: TransmissionStatus - def __init__(self, request: _Optional[_Union[MapRequest.Request, _Mapping]] = ..., id: _Optional[str] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ..., status: _Optional[_Union[TransmissionStatus, _Mapping]] = ...) -> None: ... + def __init__( + self, + request: _Optional[_Union[MapRequest.Request, _Mapping]] = ..., + id: _Optional[str] = ..., + handshake: _Optional[_Union[Handshake, _Mapping]] = ..., + status: _Optional[_Union[TransmissionStatus, _Mapping]] = ..., + ) -> None: ... class Handshake(_message.Message): __slots__ = ("sot",) @@ -53,6 +74,7 @@ class TransmissionStatus(_message.Message): class MapResponse(_message.Message): __slots__ = ("results", "id", "handshake", "status") + class Result(_message.Message): __slots__ = ("keys", "value", "tags") KEYS_FIELD_NUMBER: _ClassVar[int] @@ -61,7 +83,12 @@ class MapResponse(_message.Message): keys: _containers.RepeatedScalarFieldContainer[str] value: bytes tags: _containers.RepeatedScalarFieldContainer[str] - def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., tags: _Optional[_Iterable[str]] = ...) -> None: ... + def __init__( + self, + keys: _Optional[_Iterable[str]] = ..., + value: _Optional[bytes] = ..., + tags: _Optional[_Iterable[str]] = ..., + ) -> None: ... RESULTS_FIELD_NUMBER: _ClassVar[int] ID_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] @@ -70,7 +97,13 @@ class MapResponse(_message.Message): id: str handshake: Handshake status: TransmissionStatus - def __init__(self, results: _Optional[_Iterable[_Union[MapResponse.Result, _Mapping]]] = ..., id: _Optional[str] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ..., status: _Optional[_Union[TransmissionStatus, _Mapping]] = ...) -> None: ... + def __init__( + self, + results: _Optional[_Iterable[_Union[MapResponse.Result, _Mapping]]] = ..., + id: _Optional[str] = ..., + handshake: _Optional[_Union[Handshake, _Mapping]] = ..., + status: _Optional[_Union[TransmissionStatus, _Mapping]] = ..., + ) -> None: ... class ReadyResponse(_message.Message): __slots__ = ("ready",) diff --git a/pynumaflow/proto/mapper/map_pb2_grpc.py b/pynumaflow/proto/mapper/map_pb2_grpc.py index 87fa0b27..97834865 100644 --- a/pynumaflow/proto/mapper/map_pb2_grpc.py +++ b/pynumaflow/proto/mapper/map_pb2_grpc.py @@ -16,87 +16,108 @@ def __init__(self, channel): channel: A grpc.Channel. """ self.MapFn = channel.stream_stream( - '/map.v1.Map/MapFn', - request_serializer=map__pb2.MapRequest.SerializeToString, - response_deserializer=map__pb2.MapResponse.FromString, - ) + "/map.v1.Map/MapFn", + request_serializer=map__pb2.MapRequest.SerializeToString, + response_deserializer=map__pb2.MapResponse.FromString, + ) self.IsReady = channel.unary_unary( - '/map.v1.Map/IsReady', - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=map__pb2.ReadyResponse.FromString, - ) + "/map.v1.Map/IsReady", + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=map__pb2.ReadyResponse.FromString, + ) class MapServicer(object): """Missing associated documentation comment in .proto file.""" def MapFn(self, request_iterator, context): - """MapFn applies a function to each map request element. - """ + """MapFn applies a function to each map request element.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def IsReady(self, request, context): - """IsReady is the heartbeat endpoint for gRPC. - """ + """IsReady is the heartbeat endpoint for gRPC.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def add_MapServicer_to_server(servicer, server): rpc_method_handlers = { - 'MapFn': grpc.stream_stream_rpc_method_handler( - servicer.MapFn, - request_deserializer=map__pb2.MapRequest.FromString, - response_serializer=map__pb2.MapResponse.SerializeToString, - ), - 'IsReady': grpc.unary_unary_rpc_method_handler( - servicer.IsReady, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=map__pb2.ReadyResponse.SerializeToString, - ), + "MapFn": grpc.stream_stream_rpc_method_handler( + servicer.MapFn, + request_deserializer=map__pb2.MapRequest.FromString, + response_serializer=map__pb2.MapResponse.SerializeToString, + ), + "IsReady": grpc.unary_unary_rpc_method_handler( + servicer.IsReady, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=map__pb2.ReadyResponse.SerializeToString, + ), } - generic_handler = grpc.method_handlers_generic_handler( - 'map.v1.Map', rpc_method_handlers) + generic_handler = grpc.method_handlers_generic_handler("map.v1.Map", rpc_method_handlers) server.add_generic_rpc_handlers((generic_handler,)) - # This class is part of an EXPERIMENTAL API. +# This class is part of an EXPERIMENTAL API. class Map(object): """Missing associated documentation comment in .proto file.""" @staticmethod - def MapFn(request_iterator, + def MapFn( + request_iterator, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.stream_stream( + request_iterator, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.stream_stream(request_iterator, target, '/map.v1.Map/MapFn', + "/map.v1.Map/MapFn", map__pb2.MapRequest.SerializeToString, map__pb2.MapResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) @staticmethod - def IsReady(request, + def IsReady( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.unary_unary(request, target, '/map.v1.Map/IsReady', + "/map.v1.Map/IsReady", google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, map__pb2.ReadyResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) diff --git a/pynumaflow/proto/reducer/reduce_pb2.py b/pynumaflow/proto/reducer/reduce_pb2.py index ecb076e3..f5c1ef83 100644 --- a/pynumaflow/proto/reducer/reduce_pb2.py +++ b/pynumaflow/proto/reducer/reduce_pb2.py @@ -7,6 +7,7 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import symbol_database as _symbol_database from google.protobuf.internal import builder as _builder + # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() @@ -16,33 +17,35 @@ from google.protobuf import timestamp_pb2 as google_dot_protobuf_dot_timestamp__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x0creduce.proto\x12\treduce.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto\"\x98\x04\n\rReduceRequest\x12\x31\n\x07payload\x18\x01 \x01(\x0b\x32 .reduce.v1.ReduceRequest.Payload\x12;\n\toperation\x18\x02 \x01(\x0b\x32(.reduce.v1.ReduceRequest.WindowOperation\x1a\x9e\x01\n\x0fWindowOperation\x12=\n\x05\x65vent\x18\x01 \x01(\x0e\x32..reduce.v1.ReduceRequest.WindowOperation.Event\x12\"\n\x07windows\x18\x02 \x03(\x0b\x32\x11.reduce.v1.Window\"(\n\x05\x45vent\x12\x08\n\x04OPEN\x10\x00\x12\t\n\x05\x43LOSE\x10\x01\x12\n\n\x06\x41PPEND\x10\x04\x1a\xf5\x01\n\x07Payload\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12>\n\x07headers\x18\x05 \x03(\x0b\x32-.reduce.v1.ReduceRequest.Payload.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\"j\n\x06Window\x12)\n\x05start\x18\x01 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\'\n\x03\x65nd\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04slot\x18\x03 \x01(\t\"\xa7\x01\n\x0eReduceResponse\x12\x30\n\x06result\x18\x01 \x01(\x0b\x32 .reduce.v1.ReduceResponse.Result\x12!\n\x06window\x18\x02 \x01(\x0b\x32\x11.reduce.v1.Window\x12\x0b\n\x03\x45OF\x18\x03 \x01(\x08\x1a\x33\n\x06Result\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12\x0c\n\x04tags\x18\x03 \x03(\t\"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\x8a\x01\n\x06Reduce\x12\x43\n\x08ReduceFn\x12\x18.reduce.v1.ReduceRequest\x1a\x19.reduce.v1.ReduceResponse(\x01\x30\x01\x12;\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x18.reduce.v1.ReadyResponseb\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( + b'\n\x0creduce.proto\x12\treduce.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto"\x98\x04\n\rReduceRequest\x12\x31\n\x07payload\x18\x01 \x01(\x0b\x32 .reduce.v1.ReduceRequest.Payload\x12;\n\toperation\x18\x02 \x01(\x0b\x32(.reduce.v1.ReduceRequest.WindowOperation\x1a\x9e\x01\n\x0fWindowOperation\x12=\n\x05\x65vent\x18\x01 \x01(\x0e\x32..reduce.v1.ReduceRequest.WindowOperation.Event\x12"\n\x07windows\x18\x02 \x03(\x0b\x32\x11.reduce.v1.Window"(\n\x05\x45vent\x12\x08\n\x04OPEN\x10\x00\x12\t\n\x05\x43LOSE\x10\x01\x12\n\n\x06\x41PPEND\x10\x04\x1a\xf5\x01\n\x07Payload\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12>\n\x07headers\x18\x05 \x03(\x0b\x32-.reduce.v1.ReduceRequest.Payload.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01"j\n\x06Window\x12)\n\x05start\x18\x01 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\'\n\x03\x65nd\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04slot\x18\x03 \x01(\t"\xa7\x01\n\x0eReduceResponse\x12\x30\n\x06result\x18\x01 \x01(\x0b\x32 .reduce.v1.ReduceResponse.Result\x12!\n\x06window\x18\x02 \x01(\x0b\x32\x11.reduce.v1.Window\x12\x0b\n\x03\x45OF\x18\x03 \x01(\x08\x1a\x33\n\x06Result\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12\x0c\n\x04tags\x18\x03 \x03(\t"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\x8a\x01\n\x06Reduce\x12\x43\n\x08ReduceFn\x12\x18.reduce.v1.ReduceRequest\x1a\x19.reduce.v1.ReduceResponse(\x01\x30\x01\x12;\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x18.reduce.v1.ReadyResponseb\x06proto3' +) _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) -_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'reduce_pb2', _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, "reduce_pb2", _globals) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None - _globals['_REDUCEREQUEST_PAYLOAD_HEADERSENTRY']._options = None - _globals['_REDUCEREQUEST_PAYLOAD_HEADERSENTRY']._serialized_options = b'8\001' - _globals['_REDUCEREQUEST']._serialized_start=90 - _globals['_REDUCEREQUEST']._serialized_end=626 - _globals['_REDUCEREQUEST_WINDOWOPERATION']._serialized_start=220 - _globals['_REDUCEREQUEST_WINDOWOPERATION']._serialized_end=378 - _globals['_REDUCEREQUEST_WINDOWOPERATION_EVENT']._serialized_start=338 - _globals['_REDUCEREQUEST_WINDOWOPERATION_EVENT']._serialized_end=378 - _globals['_REDUCEREQUEST_PAYLOAD']._serialized_start=381 - _globals['_REDUCEREQUEST_PAYLOAD']._serialized_end=626 - _globals['_REDUCEREQUEST_PAYLOAD_HEADERSENTRY']._serialized_start=580 - _globals['_REDUCEREQUEST_PAYLOAD_HEADERSENTRY']._serialized_end=626 - _globals['_WINDOW']._serialized_start=628 - _globals['_WINDOW']._serialized_end=734 - _globals['_REDUCERESPONSE']._serialized_start=737 - _globals['_REDUCERESPONSE']._serialized_end=904 - _globals['_REDUCERESPONSE_RESULT']._serialized_start=853 - _globals['_REDUCERESPONSE_RESULT']._serialized_end=904 - _globals['_READYRESPONSE']._serialized_start=906 - _globals['_READYRESPONSE']._serialized_end=936 - _globals['_REDUCE']._serialized_start=939 - _globals['_REDUCE']._serialized_end=1077 + DESCRIPTOR._options = None + _globals["_REDUCEREQUEST_PAYLOAD_HEADERSENTRY"]._options = None + _globals["_REDUCEREQUEST_PAYLOAD_HEADERSENTRY"]._serialized_options = b"8\001" + _globals["_REDUCEREQUEST"]._serialized_start = 90 + _globals["_REDUCEREQUEST"]._serialized_end = 626 + _globals["_REDUCEREQUEST_WINDOWOPERATION"]._serialized_start = 220 + _globals["_REDUCEREQUEST_WINDOWOPERATION"]._serialized_end = 378 + _globals["_REDUCEREQUEST_WINDOWOPERATION_EVENT"]._serialized_start = 338 + _globals["_REDUCEREQUEST_WINDOWOPERATION_EVENT"]._serialized_end = 378 + _globals["_REDUCEREQUEST_PAYLOAD"]._serialized_start = 381 + _globals["_REDUCEREQUEST_PAYLOAD"]._serialized_end = 626 + _globals["_REDUCEREQUEST_PAYLOAD_HEADERSENTRY"]._serialized_start = 580 + _globals["_REDUCEREQUEST_PAYLOAD_HEADERSENTRY"]._serialized_end = 626 + _globals["_WINDOW"]._serialized_start = 628 + _globals["_WINDOW"]._serialized_end = 734 + _globals["_REDUCERESPONSE"]._serialized_start = 737 + _globals["_REDUCERESPONSE"]._serialized_end = 904 + _globals["_REDUCERESPONSE_RESULT"]._serialized_start = 853 + _globals["_REDUCERESPONSE_RESULT"]._serialized_end = 904 + _globals["_READYRESPONSE"]._serialized_start = 906 + _globals["_READYRESPONSE"]._serialized_end = 936 + _globals["_REDUCE"]._serialized_start = 939 + _globals["_REDUCE"]._serialized_end = 1077 # @@protoc_insertion_point(module_scope) diff --git a/pynumaflow/proto/reducer/reduce_pb2.pyi b/pynumaflow/proto/reducer/reduce_pb2.pyi index 80a974ff..88b27d53 100644 --- a/pynumaflow/proto/reducer/reduce_pb2.pyi +++ b/pynumaflow/proto/reducer/reduce_pb2.pyi @@ -4,14 +4,22 @@ from google.protobuf.internal import containers as _containers from google.protobuf.internal import enum_type_wrapper as _enum_type_wrapper from google.protobuf import descriptor as _descriptor from google.protobuf import message as _message -from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union +from typing import ( + ClassVar as _ClassVar, + Iterable as _Iterable, + Mapping as _Mapping, + Optional as _Optional, + Union as _Union, +) DESCRIPTOR: _descriptor.FileDescriptor class ReduceRequest(_message.Message): __slots__ = ("payload", "operation") + class WindowOperation(_message.Message): __slots__ = ("event", "windows") + class Event(int, metaclass=_enum_type_wrapper.EnumTypeWrapper): __slots__ = () OPEN: _ClassVar[ReduceRequest.WindowOperation.Event] @@ -24,9 +32,15 @@ class ReduceRequest(_message.Message): WINDOWS_FIELD_NUMBER: _ClassVar[int] event: ReduceRequest.WindowOperation.Event windows: _containers.RepeatedCompositeFieldContainer[Window] - def __init__(self, event: _Optional[_Union[ReduceRequest.WindowOperation.Event, str]] = ..., windows: _Optional[_Iterable[_Union[Window, _Mapping]]] = ...) -> None: ... + def __init__( + self, + event: _Optional[_Union[ReduceRequest.WindowOperation.Event, str]] = ..., + windows: _Optional[_Iterable[_Union[Window, _Mapping]]] = ..., + ) -> None: ... + class Payload(_message.Message): __slots__ = ("keys", "value", "event_time", "watermark", "headers") + class HeadersEntry(_message.Message): __slots__ = ("key", "value") KEY_FIELD_NUMBER: _ClassVar[int] @@ -44,12 +58,23 @@ class ReduceRequest(_message.Message): event_time: _timestamp_pb2.Timestamp watermark: _timestamp_pb2.Timestamp headers: _containers.ScalarMap[str, str] - def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., headers: _Optional[_Mapping[str, str]] = ...) -> None: ... + def __init__( + self, + keys: _Optional[_Iterable[str]] = ..., + value: _Optional[bytes] = ..., + event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + headers: _Optional[_Mapping[str, str]] = ..., + ) -> None: ... PAYLOAD_FIELD_NUMBER: _ClassVar[int] OPERATION_FIELD_NUMBER: _ClassVar[int] payload: ReduceRequest.Payload operation: ReduceRequest.WindowOperation - def __init__(self, payload: _Optional[_Union[ReduceRequest.Payload, _Mapping]] = ..., operation: _Optional[_Union[ReduceRequest.WindowOperation, _Mapping]] = ...) -> None: ... + def __init__( + self, + payload: _Optional[_Union[ReduceRequest.Payload, _Mapping]] = ..., + operation: _Optional[_Union[ReduceRequest.WindowOperation, _Mapping]] = ..., + ) -> None: ... class Window(_message.Message): __slots__ = ("start", "end", "slot") @@ -59,10 +84,16 @@ class Window(_message.Message): start: _timestamp_pb2.Timestamp end: _timestamp_pb2.Timestamp slot: str - def __init__(self, start: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., end: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., slot: _Optional[str] = ...) -> None: ... + def __init__( + self, + start: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + end: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + slot: _Optional[str] = ..., + ) -> None: ... class ReduceResponse(_message.Message): __slots__ = ("result", "window", "EOF") + class Result(_message.Message): __slots__ = ("keys", "value", "tags") KEYS_FIELD_NUMBER: _ClassVar[int] @@ -71,14 +102,24 @@ class ReduceResponse(_message.Message): keys: _containers.RepeatedScalarFieldContainer[str] value: bytes tags: _containers.RepeatedScalarFieldContainer[str] - def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., tags: _Optional[_Iterable[str]] = ...) -> None: ... + def __init__( + self, + keys: _Optional[_Iterable[str]] = ..., + value: _Optional[bytes] = ..., + tags: _Optional[_Iterable[str]] = ..., + ) -> None: ... RESULT_FIELD_NUMBER: _ClassVar[int] WINDOW_FIELD_NUMBER: _ClassVar[int] EOF_FIELD_NUMBER: _ClassVar[int] result: ReduceResponse.Result window: Window EOF: bool - def __init__(self, result: _Optional[_Union[ReduceResponse.Result, _Mapping]] = ..., window: _Optional[_Union[Window, _Mapping]] = ..., EOF: bool = ...) -> None: ... + def __init__( + self, + result: _Optional[_Union[ReduceResponse.Result, _Mapping]] = ..., + window: _Optional[_Union[Window, _Mapping]] = ..., + EOF: bool = ..., + ) -> None: ... class ReadyResponse(_message.Message): __slots__ = ("ready",) diff --git a/pynumaflow/proto/reducer/reduce_pb2_grpc.py b/pynumaflow/proto/reducer/reduce_pb2_grpc.py index 1fd860b4..5a0a15f6 100644 --- a/pynumaflow/proto/reducer/reduce_pb2_grpc.py +++ b/pynumaflow/proto/reducer/reduce_pb2_grpc.py @@ -16,87 +16,108 @@ def __init__(self, channel): channel: A grpc.Channel. """ self.ReduceFn = channel.stream_stream( - '/reduce.v1.Reduce/ReduceFn', - request_serializer=reduce__pb2.ReduceRequest.SerializeToString, - response_deserializer=reduce__pb2.ReduceResponse.FromString, - ) + "/reduce.v1.Reduce/ReduceFn", + request_serializer=reduce__pb2.ReduceRequest.SerializeToString, + response_deserializer=reduce__pb2.ReduceResponse.FromString, + ) self.IsReady = channel.unary_unary( - '/reduce.v1.Reduce/IsReady', - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=reduce__pb2.ReadyResponse.FromString, - ) + "/reduce.v1.Reduce/IsReady", + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=reduce__pb2.ReadyResponse.FromString, + ) class ReduceServicer(object): """Missing associated documentation comment in .proto file.""" def ReduceFn(self, request_iterator, context): - """ReduceFn applies a reduce function to a request stream. - """ + """ReduceFn applies a reduce function to a request stream.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def IsReady(self, request, context): - """IsReady is the heartbeat endpoint for gRPC. - """ + """IsReady is the heartbeat endpoint for gRPC.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def add_ReduceServicer_to_server(servicer, server): rpc_method_handlers = { - 'ReduceFn': grpc.stream_stream_rpc_method_handler( - servicer.ReduceFn, - request_deserializer=reduce__pb2.ReduceRequest.FromString, - response_serializer=reduce__pb2.ReduceResponse.SerializeToString, - ), - 'IsReady': grpc.unary_unary_rpc_method_handler( - servicer.IsReady, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=reduce__pb2.ReadyResponse.SerializeToString, - ), + "ReduceFn": grpc.stream_stream_rpc_method_handler( + servicer.ReduceFn, + request_deserializer=reduce__pb2.ReduceRequest.FromString, + response_serializer=reduce__pb2.ReduceResponse.SerializeToString, + ), + "IsReady": grpc.unary_unary_rpc_method_handler( + servicer.IsReady, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=reduce__pb2.ReadyResponse.SerializeToString, + ), } - generic_handler = grpc.method_handlers_generic_handler( - 'reduce.v1.Reduce', rpc_method_handlers) + generic_handler = grpc.method_handlers_generic_handler("reduce.v1.Reduce", rpc_method_handlers) server.add_generic_rpc_handlers((generic_handler,)) - # This class is part of an EXPERIMENTAL API. +# This class is part of an EXPERIMENTAL API. class Reduce(object): """Missing associated documentation comment in .proto file.""" @staticmethod - def ReduceFn(request_iterator, + def ReduceFn( + request_iterator, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.stream_stream( + request_iterator, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.stream_stream(request_iterator, target, '/reduce.v1.Reduce/ReduceFn', + "/reduce.v1.Reduce/ReduceFn", reduce__pb2.ReduceRequest.SerializeToString, reduce__pb2.ReduceResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) @staticmethod - def IsReady(request, + def IsReady( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.unary_unary(request, target, '/reduce.v1.Reduce/IsReady', + "/reduce.v1.Reduce/IsReady", google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, reduce__pb2.ReadyResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) diff --git a/pynumaflow/proto/sideinput/sideinput_pb2.py b/pynumaflow/proto/sideinput/sideinput_pb2.py index 0160315f..c0dab051 100644 --- a/pynumaflow/proto/sideinput/sideinput_pb2.py +++ b/pynumaflow/proto/sideinput/sideinput_pb2.py @@ -7,6 +7,7 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import symbol_database as _symbol_database from google.protobuf.internal import builder as _builder + # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() @@ -15,17 +16,19 @@ from google.protobuf import empty_pb2 as google_dot_protobuf_dot_empty__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x0fsideinput.proto\x12\x0csideinput.v1\x1a\x1bgoogle/protobuf/empty.proto\"8\n\x11SideInputResponse\x12\r\n\x05value\x18\x01 \x01(\x0c\x12\x14\n\x0cno_broadcast\x18\x02 \x01(\x08\"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\x99\x01\n\tSideInput\x12L\n\x11RetrieveSideInput\x12\x16.google.protobuf.Empty\x1a\x1f.sideinput.v1.SideInputResponse\x12>\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x1b.sideinput.v1.ReadyResponseb\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( + b'\n\x0fsideinput.proto\x12\x0csideinput.v1\x1a\x1bgoogle/protobuf/empty.proto"8\n\x11SideInputResponse\x12\r\n\x05value\x18\x01 \x01(\x0c\x12\x14\n\x0cno_broadcast\x18\x02 \x01(\x08"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\x99\x01\n\tSideInput\x12L\n\x11RetrieveSideInput\x12\x16.google.protobuf.Empty\x1a\x1f.sideinput.v1.SideInputResponse\x12>\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x1b.sideinput.v1.ReadyResponseb\x06proto3' +) _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) -_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'sideinput_pb2', _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, "sideinput_pb2", _globals) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None - _globals['_SIDEINPUTRESPONSE']._serialized_start=62 - _globals['_SIDEINPUTRESPONSE']._serialized_end=118 - _globals['_READYRESPONSE']._serialized_start=120 - _globals['_READYRESPONSE']._serialized_end=150 - _globals['_SIDEINPUT']._serialized_start=153 - _globals['_SIDEINPUT']._serialized_end=306 + DESCRIPTOR._options = None + _globals["_SIDEINPUTRESPONSE"]._serialized_start = 62 + _globals["_SIDEINPUTRESPONSE"]._serialized_end = 118 + _globals["_READYRESPONSE"]._serialized_start = 120 + _globals["_READYRESPONSE"]._serialized_end = 150 + _globals["_SIDEINPUT"]._serialized_start = 153 + _globals["_SIDEINPUT"]._serialized_end = 306 # @@protoc_insertion_point(module_scope) diff --git a/pynumaflow/proto/sideinput/sideinput_pb2_grpc.py b/pynumaflow/proto/sideinput/sideinput_pb2_grpc.py index 8abe64d2..72ea87ed 100644 --- a/pynumaflow/proto/sideinput/sideinput_pb2_grpc.py +++ b/pynumaflow/proto/sideinput/sideinput_pb2_grpc.py @@ -24,15 +24,15 @@ def __init__(self, channel): channel: A grpc.Channel. """ self.RetrieveSideInput = channel.unary_unary( - '/sideinput.v1.SideInput/RetrieveSideInput', - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=sideinput__pb2.SideInputResponse.FromString, - ) + "/sideinput.v1.SideInput/RetrieveSideInput", + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=sideinput__pb2.SideInputResponse.FromString, + ) self.IsReady = channel.unary_unary( - '/sideinput.v1.SideInput/IsReady', - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=sideinput__pb2.ReadyResponse.FromString, - ) + "/sideinput.v1.SideInput/IsReady", + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=sideinput__pb2.ReadyResponse.FromString, + ) class SideInputServicer(object): @@ -47,39 +47,38 @@ class SideInputServicer(object): """ def RetrieveSideInput(self, request, context): - """RetrieveSideInput is the endpoint to retrieve the latest value of a given Side Input. - """ + """RetrieveSideInput is the endpoint to retrieve the latest value of a given Side Input.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def IsReady(self, request, context): - """IsReady is the health check endpoint to indicate whether the service is ready to be used. - """ + """IsReady is the health check endpoint to indicate whether the service is ready to be used.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def add_SideInputServicer_to_server(servicer, server): rpc_method_handlers = { - 'RetrieveSideInput': grpc.unary_unary_rpc_method_handler( - servicer.RetrieveSideInput, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=sideinput__pb2.SideInputResponse.SerializeToString, - ), - 'IsReady': grpc.unary_unary_rpc_method_handler( - servicer.IsReady, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=sideinput__pb2.ReadyResponse.SerializeToString, - ), + "RetrieveSideInput": grpc.unary_unary_rpc_method_handler( + servicer.RetrieveSideInput, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=sideinput__pb2.SideInputResponse.SerializeToString, + ), + "IsReady": grpc.unary_unary_rpc_method_handler( + servicer.IsReady, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=sideinput__pb2.ReadyResponse.SerializeToString, + ), } generic_handler = grpc.method_handlers_generic_handler( - 'sideinput.v1.SideInput', rpc_method_handlers) + "sideinput.v1.SideInput", rpc_method_handlers + ) server.add_generic_rpc_handlers((generic_handler,)) - # This class is part of an EXPERIMENTAL API. +# This class is part of an EXPERIMENTAL API. class SideInput(object): """SideInput is the gRPC service for user-defined Side Inputs. It is used to propagate changes in the values of the provided Side Inputs @@ -92,35 +91,59 @@ class SideInput(object): """ @staticmethod - def RetrieveSideInput(request, + def RetrieveSideInput( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.unary_unary(request, target, '/sideinput.v1.SideInput/RetrieveSideInput', + "/sideinput.v1.SideInput/RetrieveSideInput", google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, sideinput__pb2.SideInputResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) @staticmethod - def IsReady(request, + def IsReady( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.unary_unary(request, target, '/sideinput.v1.SideInput/IsReady', + "/sideinput.v1.SideInput/IsReady", google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, sideinput__pb2.ReadyResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) diff --git a/pynumaflow/proto/sinker/sink_pb2.py b/pynumaflow/proto/sinker/sink_pb2.py index 67d905c5..27082a0e 100644 --- a/pynumaflow/proto/sinker/sink_pb2.py +++ b/pynumaflow/proto/sinker/sink_pb2.py @@ -7,6 +7,7 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import symbol_database as _symbol_database from google.protobuf.internal import builder as _builder + # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() @@ -16,33 +17,35 @@ from google.protobuf import timestamp_pb2 as google_dot_protobuf_dot_timestamp__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\nsink.proto\x12\x07sink.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto\"\xa3\x03\n\x0bSinkRequest\x12-\n\x07request\x18\x01 \x01(\x0b\x32\x1c.sink.v1.SinkRequest.Request\x12+\n\x06status\x18\x02 \x01(\x0b\x32\x1b.sink.v1.TransmissionStatus\x12*\n\thandshake\x18\x03 \x01(\x0b\x32\x12.sink.v1.HandshakeH\x00\x88\x01\x01\x1a\xfd\x01\n\x07Request\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\n\n\x02id\x18\x05 \x01(\t\x12:\n\x07headers\x18\x06 \x03(\x0b\x32).sink.v1.SinkRequest.Request.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x42\x0c\n\n_handshake\"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08\"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\"!\n\x12TransmissionStatus\x12\x0b\n\x03\x65ot\x18\x01 \x01(\x08\"\xfc\x01\n\x0cSinkResponse\x12-\n\x07results\x18\x01 \x03(\x0b\x32\x1c.sink.v1.SinkResponse.Result\x12*\n\thandshake\x18\x02 \x01(\x0b\x32\x12.sink.v1.HandshakeH\x00\x88\x01\x01\x12\x30\n\x06status\x18\x03 \x01(\x0b\x32\x1b.sink.v1.TransmissionStatusH\x01\x88\x01\x01\x1a\x46\n\x06Result\x12\n\n\x02id\x18\x01 \x01(\t\x12\x1f\n\x06status\x18\x02 \x01(\x0e\x32\x0f.sink.v1.Status\x12\x0f\n\x07\x65rr_msg\x18\x03 \x01(\tB\x0c\n\n_handshakeB\t\n\x07_status*0\n\x06Status\x12\x0b\n\x07SUCCESS\x10\x00\x12\x0b\n\x07\x46\x41ILURE\x10\x01\x12\x0c\n\x08\x46\x41LLBACK\x10\x02\x32|\n\x04Sink\x12\x39\n\x06SinkFn\x12\x14.sink.v1.SinkRequest\x1a\x15.sink.v1.SinkResponse(\x01\x30\x01\x12\x39\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x16.sink.v1.ReadyResponseb\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( + b'\n\nsink.proto\x12\x07sink.v1\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1fgoogle/protobuf/timestamp.proto"\xa3\x03\n\x0bSinkRequest\x12-\n\x07request\x18\x01 \x01(\x0b\x32\x1c.sink.v1.SinkRequest.Request\x12+\n\x06status\x18\x02 \x01(\x0b\x32\x1b.sink.v1.TransmissionStatus\x12*\n\thandshake\x18\x03 \x01(\x0b\x32\x12.sink.v1.HandshakeH\x00\x88\x01\x01\x1a\xfd\x01\n\x07Request\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\n\n\x02id\x18\x05 \x01(\t\x12:\n\x07headers\x18\x06 \x03(\x0b\x32).sink.v1.SinkRequest.Request.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x42\x0c\n\n_handshake"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08"!\n\x12TransmissionStatus\x12\x0b\n\x03\x65ot\x18\x01 \x01(\x08"\xfc\x01\n\x0cSinkResponse\x12-\n\x07results\x18\x01 \x03(\x0b\x32\x1c.sink.v1.SinkResponse.Result\x12*\n\thandshake\x18\x02 \x01(\x0b\x32\x12.sink.v1.HandshakeH\x00\x88\x01\x01\x12\x30\n\x06status\x18\x03 \x01(\x0b\x32\x1b.sink.v1.TransmissionStatusH\x01\x88\x01\x01\x1a\x46\n\x06Result\x12\n\n\x02id\x18\x01 \x01(\t\x12\x1f\n\x06status\x18\x02 \x01(\x0e\x32\x0f.sink.v1.Status\x12\x0f\n\x07\x65rr_msg\x18\x03 \x01(\tB\x0c\n\n_handshakeB\t\n\x07_status*0\n\x06Status\x12\x0b\n\x07SUCCESS\x10\x00\x12\x0b\n\x07\x46\x41ILURE\x10\x01\x12\x0c\n\x08\x46\x41LLBACK\x10\x02\x32|\n\x04Sink\x12\x39\n\x06SinkFn\x12\x14.sink.v1.SinkRequest\x1a\x15.sink.v1.SinkResponse(\x01\x30\x01\x12\x39\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x16.sink.v1.ReadyResponseb\x06proto3' +) _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) -_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'sink_pb2', _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, "sink_pb2", _globals) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None - _globals['_SINKREQUEST_REQUEST_HEADERSENTRY']._options = None - _globals['_SINKREQUEST_REQUEST_HEADERSENTRY']._serialized_options = b'8\001' - _globals['_STATUS']._serialized_start=855 - _globals['_STATUS']._serialized_end=903 - _globals['_SINKREQUEST']._serialized_start=86 - _globals['_SINKREQUEST']._serialized_end=505 - _globals['_SINKREQUEST_REQUEST']._serialized_start=238 - _globals['_SINKREQUEST_REQUEST']._serialized_end=491 - _globals['_SINKREQUEST_REQUEST_HEADERSENTRY']._serialized_start=445 - _globals['_SINKREQUEST_REQUEST_HEADERSENTRY']._serialized_end=491 - _globals['_HANDSHAKE']._serialized_start=507 - _globals['_HANDSHAKE']._serialized_end=531 - _globals['_READYRESPONSE']._serialized_start=533 - _globals['_READYRESPONSE']._serialized_end=563 - _globals['_TRANSMISSIONSTATUS']._serialized_start=565 - _globals['_TRANSMISSIONSTATUS']._serialized_end=598 - _globals['_SINKRESPONSE']._serialized_start=601 - _globals['_SINKRESPONSE']._serialized_end=853 - _globals['_SINKRESPONSE_RESULT']._serialized_start=758 - _globals['_SINKRESPONSE_RESULT']._serialized_end=828 - _globals['_SINK']._serialized_start=905 - _globals['_SINK']._serialized_end=1029 + DESCRIPTOR._options = None + _globals["_SINKREQUEST_REQUEST_HEADERSENTRY"]._options = None + _globals["_SINKREQUEST_REQUEST_HEADERSENTRY"]._serialized_options = b"8\001" + _globals["_STATUS"]._serialized_start = 855 + _globals["_STATUS"]._serialized_end = 903 + _globals["_SINKREQUEST"]._serialized_start = 86 + _globals["_SINKREQUEST"]._serialized_end = 505 + _globals["_SINKREQUEST_REQUEST"]._serialized_start = 238 + _globals["_SINKREQUEST_REQUEST"]._serialized_end = 491 + _globals["_SINKREQUEST_REQUEST_HEADERSENTRY"]._serialized_start = 445 + _globals["_SINKREQUEST_REQUEST_HEADERSENTRY"]._serialized_end = 491 + _globals["_HANDSHAKE"]._serialized_start = 507 + _globals["_HANDSHAKE"]._serialized_end = 531 + _globals["_READYRESPONSE"]._serialized_start = 533 + _globals["_READYRESPONSE"]._serialized_end = 563 + _globals["_TRANSMISSIONSTATUS"]._serialized_start = 565 + _globals["_TRANSMISSIONSTATUS"]._serialized_end = 598 + _globals["_SINKRESPONSE"]._serialized_start = 601 + _globals["_SINKRESPONSE"]._serialized_end = 853 + _globals["_SINKRESPONSE_RESULT"]._serialized_start = 758 + _globals["_SINKRESPONSE_RESULT"]._serialized_end = 828 + _globals["_SINK"]._serialized_start = 905 + _globals["_SINK"]._serialized_end = 1029 # @@protoc_insertion_point(module_scope) diff --git a/pynumaflow/proto/sinker/sink_pb2.pyi b/pynumaflow/proto/sinker/sink_pb2.pyi index 6c0292ca..78926321 100644 --- a/pynumaflow/proto/sinker/sink_pb2.pyi +++ b/pynumaflow/proto/sinker/sink_pb2.pyi @@ -4,7 +4,13 @@ from google.protobuf.internal import containers as _containers from google.protobuf.internal import enum_type_wrapper as _enum_type_wrapper from google.protobuf import descriptor as _descriptor from google.protobuf import message as _message -from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union +from typing import ( + ClassVar as _ClassVar, + Iterable as _Iterable, + Mapping as _Mapping, + Optional as _Optional, + Union as _Union, +) DESCRIPTOR: _descriptor.FileDescriptor @@ -13,14 +19,17 @@ class Status(int, metaclass=_enum_type_wrapper.EnumTypeWrapper): SUCCESS: _ClassVar[Status] FAILURE: _ClassVar[Status] FALLBACK: _ClassVar[Status] + SUCCESS: Status FAILURE: Status FALLBACK: Status class SinkRequest(_message.Message): __slots__ = ("request", "status", "handshake") + class Request(_message.Message): __slots__ = ("keys", "value", "event_time", "watermark", "id", "headers") + class HeadersEntry(_message.Message): __slots__ = ("key", "value") KEY_FIELD_NUMBER: _ClassVar[int] @@ -40,14 +49,27 @@ class SinkRequest(_message.Message): watermark: _timestamp_pb2.Timestamp id: str headers: _containers.ScalarMap[str, str] - def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., id: _Optional[str] = ..., headers: _Optional[_Mapping[str, str]] = ...) -> None: ... + def __init__( + self, + keys: _Optional[_Iterable[str]] = ..., + value: _Optional[bytes] = ..., + event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + id: _Optional[str] = ..., + headers: _Optional[_Mapping[str, str]] = ..., + ) -> None: ... REQUEST_FIELD_NUMBER: _ClassVar[int] STATUS_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] request: SinkRequest.Request status: TransmissionStatus handshake: Handshake - def __init__(self, request: _Optional[_Union[SinkRequest.Request, _Mapping]] = ..., status: _Optional[_Union[TransmissionStatus, _Mapping]] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ...) -> None: ... + def __init__( + self, + request: _Optional[_Union[SinkRequest.Request, _Mapping]] = ..., + status: _Optional[_Union[TransmissionStatus, _Mapping]] = ..., + handshake: _Optional[_Union[Handshake, _Mapping]] = ..., + ) -> None: ... class Handshake(_message.Message): __slots__ = ("sot",) @@ -69,6 +91,7 @@ class TransmissionStatus(_message.Message): class SinkResponse(_message.Message): __slots__ = ("results", "handshake", "status") + class Result(_message.Message): __slots__ = ("id", "status", "err_msg") ID_FIELD_NUMBER: _ClassVar[int] @@ -77,11 +100,21 @@ class SinkResponse(_message.Message): id: str status: Status err_msg: str - def __init__(self, id: _Optional[str] = ..., status: _Optional[_Union[Status, str]] = ..., err_msg: _Optional[str] = ...) -> None: ... + def __init__( + self, + id: _Optional[str] = ..., + status: _Optional[_Union[Status, str]] = ..., + err_msg: _Optional[str] = ..., + ) -> None: ... RESULTS_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] STATUS_FIELD_NUMBER: _ClassVar[int] results: _containers.RepeatedCompositeFieldContainer[SinkResponse.Result] handshake: Handshake status: TransmissionStatus - def __init__(self, results: _Optional[_Iterable[_Union[SinkResponse.Result, _Mapping]]] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ..., status: _Optional[_Union[TransmissionStatus, _Mapping]] = ...) -> None: ... + def __init__( + self, + results: _Optional[_Iterable[_Union[SinkResponse.Result, _Mapping]]] = ..., + handshake: _Optional[_Union[Handshake, _Mapping]] = ..., + status: _Optional[_Union[TransmissionStatus, _Mapping]] = ..., + ) -> None: ... diff --git a/pynumaflow/proto/sinker/sink_pb2_grpc.py b/pynumaflow/proto/sinker/sink_pb2_grpc.py index 4f53a34e..9609c76e 100644 --- a/pynumaflow/proto/sinker/sink_pb2_grpc.py +++ b/pynumaflow/proto/sinker/sink_pb2_grpc.py @@ -16,87 +16,108 @@ def __init__(self, channel): channel: A grpc.Channel. """ self.SinkFn = channel.stream_stream( - '/sink.v1.Sink/SinkFn', - request_serializer=sink__pb2.SinkRequest.SerializeToString, - response_deserializer=sink__pb2.SinkResponse.FromString, - ) + "/sink.v1.Sink/SinkFn", + request_serializer=sink__pb2.SinkRequest.SerializeToString, + response_deserializer=sink__pb2.SinkResponse.FromString, + ) self.IsReady = channel.unary_unary( - '/sink.v1.Sink/IsReady', - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=sink__pb2.ReadyResponse.FromString, - ) + "/sink.v1.Sink/IsReady", + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=sink__pb2.ReadyResponse.FromString, + ) class SinkServicer(object): """Missing associated documentation comment in .proto file.""" def SinkFn(self, request_iterator, context): - """SinkFn writes the request to a user defined sink. - """ + """SinkFn writes the request to a user defined sink.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def IsReady(self, request, context): - """IsReady is the heartbeat endpoint for gRPC. - """ + """IsReady is the heartbeat endpoint for gRPC.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def add_SinkServicer_to_server(servicer, server): rpc_method_handlers = { - 'SinkFn': grpc.stream_stream_rpc_method_handler( - servicer.SinkFn, - request_deserializer=sink__pb2.SinkRequest.FromString, - response_serializer=sink__pb2.SinkResponse.SerializeToString, - ), - 'IsReady': grpc.unary_unary_rpc_method_handler( - servicer.IsReady, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=sink__pb2.ReadyResponse.SerializeToString, - ), + "SinkFn": grpc.stream_stream_rpc_method_handler( + servicer.SinkFn, + request_deserializer=sink__pb2.SinkRequest.FromString, + response_serializer=sink__pb2.SinkResponse.SerializeToString, + ), + "IsReady": grpc.unary_unary_rpc_method_handler( + servicer.IsReady, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=sink__pb2.ReadyResponse.SerializeToString, + ), } - generic_handler = grpc.method_handlers_generic_handler( - 'sink.v1.Sink', rpc_method_handlers) + generic_handler = grpc.method_handlers_generic_handler("sink.v1.Sink", rpc_method_handlers) server.add_generic_rpc_handlers((generic_handler,)) - # This class is part of an EXPERIMENTAL API. +# This class is part of an EXPERIMENTAL API. class Sink(object): """Missing associated documentation comment in .proto file.""" @staticmethod - def SinkFn(request_iterator, + def SinkFn( + request_iterator, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.stream_stream( + request_iterator, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.stream_stream(request_iterator, target, '/sink.v1.Sink/SinkFn', + "/sink.v1.Sink/SinkFn", sink__pb2.SinkRequest.SerializeToString, sink__pb2.SinkResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) @staticmethod - def IsReady(request, + def IsReady( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.unary_unary(request, target, '/sink.v1.Sink/IsReady', + "/sink.v1.Sink/IsReady", google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, sink__pb2.ReadyResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) diff --git a/pynumaflow/proto/sourcer/source_pb2.py b/pynumaflow/proto/sourcer/source_pb2.py index e8d62515..6cd6f0b2 100644 --- a/pynumaflow/proto/sourcer/source_pb2.py +++ b/pynumaflow/proto/sourcer/source_pb2.py @@ -7,6 +7,7 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import symbol_database as _symbol_database from google.protobuf.internal import builder as _builder + # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() @@ -16,53 +17,55 @@ from google.protobuf import empty_pb2 as google_dot_protobuf_dot_empty__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x0csource.proto\x12\tsource.v1\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1bgoogle/protobuf/empty.proto\"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08\"\xb1\x01\n\x0bReadRequest\x12/\n\x07request\x18\x01 \x01(\x0b\x32\x1e.source.v1.ReadRequest.Request\x12,\n\thandshake\x18\x02 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a\x35\n\x07Request\x12\x13\n\x0bnum_records\x18\x01 \x01(\x04\x12\x15\n\rtimeout_in_ms\x18\x02 \x01(\rB\x0c\n\n_handshake\"\x81\x05\n\x0cReadResponse\x12.\n\x06result\x18\x01 \x01(\x0b\x32\x1e.source.v1.ReadResponse.Result\x12.\n\x06status\x18\x02 \x01(\x0b\x32\x1e.source.v1.ReadResponse.Status\x12,\n\thandshake\x18\x03 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a\xe8\x01\n\x06Result\x12\x0f\n\x07payload\x18\x01 \x01(\x0c\x12!\n\x06offset\x18\x02 \x01(\x0b\x32\x11.source.v1.Offset\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04keys\x18\x04 \x03(\t\x12<\n\x07headers\x18\x05 \x03(\x0b\x32+.source.v1.ReadResponse.Result.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x1a\xe9\x01\n\x06Status\x12\x0b\n\x03\x65ot\x18\x01 \x01(\x08\x12\x31\n\x04\x63ode\x18\x02 \x01(\x0e\x32#.source.v1.ReadResponse.Status.Code\x12\x38\n\x05\x65rror\x18\x03 \x01(\x0e\x32$.source.v1.ReadResponse.Status.ErrorH\x00\x88\x01\x01\x12\x10\n\x03msg\x18\x04 \x01(\tH\x01\x88\x01\x01\" \n\x04\x43ode\x12\x0b\n\x07SUCCESS\x10\x00\x12\x0b\n\x07\x46\x41ILURE\x10\x01\"\x1f\n\x05\x45rror\x12\x0b\n\x07UNACKED\x10\x00\x12\t\n\x05OTHER\x10\x01\x42\x08\n\x06_errorB\x06\n\x04_msgB\x0c\n\n_handshake\"\xa7\x01\n\nAckRequest\x12.\n\x07request\x18\x01 \x01(\x0b\x32\x1d.source.v1.AckRequest.Request\x12,\n\thandshake\x18\x02 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a-\n\x07Request\x12\"\n\x07offsets\x18\x01 \x03(\x0b\x32\x11.source.v1.OffsetB\x0c\n\n_handshake\"\xab\x01\n\x0b\x41\x63kResponse\x12-\n\x06result\x18\x01 \x01(\x0b\x32\x1d.source.v1.AckResponse.Result\x12,\n\thandshake\x18\x02 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a\x31\n\x06Result\x12\'\n\x07success\x18\x01 \x01(\x0b\x32\x16.google.protobuf.EmptyB\x0c\n\n_handshake\"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\"]\n\x0fPendingResponse\x12\x31\n\x06result\x18\x01 \x01(\x0b\x32!.source.v1.PendingResponse.Result\x1a\x17\n\x06Result\x12\r\n\x05\x63ount\x18\x01 \x01(\x03\"h\n\x12PartitionsResponse\x12\x34\n\x06result\x18\x01 \x01(\x0b\x32$.source.v1.PartitionsResponse.Result\x1a\x1c\n\x06Result\x12\x12\n\npartitions\x18\x01 \x03(\x05\".\n\x06Offset\x12\x0e\n\x06offset\x18\x01 \x01(\x0c\x12\x14\n\x0cpartition_id\x18\x02 \x01(\x05\x32\xc8\x02\n\x06Source\x12=\n\x06ReadFn\x12\x16.source.v1.ReadRequest\x1a\x17.source.v1.ReadResponse(\x01\x30\x01\x12:\n\x05\x41\x63kFn\x12\x15.source.v1.AckRequest\x1a\x16.source.v1.AckResponse(\x01\x30\x01\x12?\n\tPendingFn\x12\x16.google.protobuf.Empty\x1a\x1a.source.v1.PendingResponse\x12\x45\n\x0cPartitionsFn\x12\x16.google.protobuf.Empty\x1a\x1d.source.v1.PartitionsResponse\x12;\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x18.source.v1.ReadyResponseb\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( + b'\n\x0csource.proto\x12\tsource.v1\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1bgoogle/protobuf/empty.proto"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08"\xb1\x01\n\x0bReadRequest\x12/\n\x07request\x18\x01 \x01(\x0b\x32\x1e.source.v1.ReadRequest.Request\x12,\n\thandshake\x18\x02 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a\x35\n\x07Request\x12\x13\n\x0bnum_records\x18\x01 \x01(\x04\x12\x15\n\rtimeout_in_ms\x18\x02 \x01(\rB\x0c\n\n_handshake"\x81\x05\n\x0cReadResponse\x12.\n\x06result\x18\x01 \x01(\x0b\x32\x1e.source.v1.ReadResponse.Result\x12.\n\x06status\x18\x02 \x01(\x0b\x32\x1e.source.v1.ReadResponse.Status\x12,\n\thandshake\x18\x03 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a\xe8\x01\n\x06Result\x12\x0f\n\x07payload\x18\x01 \x01(\x0c\x12!\n\x06offset\x18\x02 \x01(\x0b\x32\x11.source.v1.Offset\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04keys\x18\x04 \x03(\t\x12<\n\x07headers\x18\x05 \x03(\x0b\x32+.source.v1.ReadResponse.Result.HeadersEntry\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x1a\xe9\x01\n\x06Status\x12\x0b\n\x03\x65ot\x18\x01 \x01(\x08\x12\x31\n\x04\x63ode\x18\x02 \x01(\x0e\x32#.source.v1.ReadResponse.Status.Code\x12\x38\n\x05\x65rror\x18\x03 \x01(\x0e\x32$.source.v1.ReadResponse.Status.ErrorH\x00\x88\x01\x01\x12\x10\n\x03msg\x18\x04 \x01(\tH\x01\x88\x01\x01" \n\x04\x43ode\x12\x0b\n\x07SUCCESS\x10\x00\x12\x0b\n\x07\x46\x41ILURE\x10\x01"\x1f\n\x05\x45rror\x12\x0b\n\x07UNACKED\x10\x00\x12\t\n\x05OTHER\x10\x01\x42\x08\n\x06_errorB\x06\n\x04_msgB\x0c\n\n_handshake"\xa7\x01\n\nAckRequest\x12.\n\x07request\x18\x01 \x01(\x0b\x32\x1d.source.v1.AckRequest.Request\x12,\n\thandshake\x18\x02 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a-\n\x07Request\x12"\n\x07offsets\x18\x01 \x03(\x0b\x32\x11.source.v1.OffsetB\x0c\n\n_handshake"\xab\x01\n\x0b\x41\x63kResponse\x12-\n\x06result\x18\x01 \x01(\x0b\x32\x1d.source.v1.AckResponse.Result\x12,\n\thandshake\x18\x02 \x01(\x0b\x32\x14.source.v1.HandshakeH\x00\x88\x01\x01\x1a\x31\n\x06Result\x12\'\n\x07success\x18\x01 \x01(\x0b\x32\x16.google.protobuf.EmptyB\x0c\n\n_handshake"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08"]\n\x0fPendingResponse\x12\x31\n\x06result\x18\x01 \x01(\x0b\x32!.source.v1.PendingResponse.Result\x1a\x17\n\x06Result\x12\r\n\x05\x63ount\x18\x01 \x01(\x03"h\n\x12PartitionsResponse\x12\x34\n\x06result\x18\x01 \x01(\x0b\x32$.source.v1.PartitionsResponse.Result\x1a\x1c\n\x06Result\x12\x12\n\npartitions\x18\x01 \x03(\x05".\n\x06Offset\x12\x0e\n\x06offset\x18\x01 \x01(\x0c\x12\x14\n\x0cpartition_id\x18\x02 \x01(\x05\x32\xc8\x02\n\x06Source\x12=\n\x06ReadFn\x12\x16.source.v1.ReadRequest\x1a\x17.source.v1.ReadResponse(\x01\x30\x01\x12:\n\x05\x41\x63kFn\x12\x15.source.v1.AckRequest\x1a\x16.source.v1.AckResponse(\x01\x30\x01\x12?\n\tPendingFn\x12\x16.google.protobuf.Empty\x1a\x1a.source.v1.PendingResponse\x12\x45\n\x0cPartitionsFn\x12\x16.google.protobuf.Empty\x1a\x1d.source.v1.PartitionsResponse\x12;\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a\x18.source.v1.ReadyResponseb\x06proto3' +) _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) -_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'source_pb2', _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, "source_pb2", _globals) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None - _globals['_READRESPONSE_RESULT_HEADERSENTRY']._options = None - _globals['_READRESPONSE_RESULT_HEADERSENTRY']._serialized_options = b'8\001' - _globals['_HANDSHAKE']._serialized_start=89 - _globals['_HANDSHAKE']._serialized_end=113 - _globals['_READREQUEST']._serialized_start=116 - _globals['_READREQUEST']._serialized_end=293 - _globals['_READREQUEST_REQUEST']._serialized_start=226 - _globals['_READREQUEST_REQUEST']._serialized_end=279 - _globals['_READRESPONSE']._serialized_start=296 - _globals['_READRESPONSE']._serialized_end=937 - _globals['_READRESPONSE_RESULT']._serialized_start=455 - _globals['_READRESPONSE_RESULT']._serialized_end=687 - _globals['_READRESPONSE_RESULT_HEADERSENTRY']._serialized_start=641 - _globals['_READRESPONSE_RESULT_HEADERSENTRY']._serialized_end=687 - _globals['_READRESPONSE_STATUS']._serialized_start=690 - _globals['_READRESPONSE_STATUS']._serialized_end=923 - _globals['_READRESPONSE_STATUS_CODE']._serialized_start=840 - _globals['_READRESPONSE_STATUS_CODE']._serialized_end=872 - _globals['_READRESPONSE_STATUS_ERROR']._serialized_start=874 - _globals['_READRESPONSE_STATUS_ERROR']._serialized_end=905 - _globals['_ACKREQUEST']._serialized_start=940 - _globals['_ACKREQUEST']._serialized_end=1107 - _globals['_ACKREQUEST_REQUEST']._serialized_start=1048 - _globals['_ACKREQUEST_REQUEST']._serialized_end=1093 - _globals['_ACKRESPONSE']._serialized_start=1110 - _globals['_ACKRESPONSE']._serialized_end=1281 - _globals['_ACKRESPONSE_RESULT']._serialized_start=1218 - _globals['_ACKRESPONSE_RESULT']._serialized_end=1267 - _globals['_READYRESPONSE']._serialized_start=1283 - _globals['_READYRESPONSE']._serialized_end=1313 - _globals['_PENDINGRESPONSE']._serialized_start=1315 - _globals['_PENDINGRESPONSE']._serialized_end=1408 - _globals['_PENDINGRESPONSE_RESULT']._serialized_start=1385 - _globals['_PENDINGRESPONSE_RESULT']._serialized_end=1408 - _globals['_PARTITIONSRESPONSE']._serialized_start=1410 - _globals['_PARTITIONSRESPONSE']._serialized_end=1514 - _globals['_PARTITIONSRESPONSE_RESULT']._serialized_start=1486 - _globals['_PARTITIONSRESPONSE_RESULT']._serialized_end=1514 - _globals['_OFFSET']._serialized_start=1516 - _globals['_OFFSET']._serialized_end=1562 - _globals['_SOURCE']._serialized_start=1565 - _globals['_SOURCE']._serialized_end=1893 + DESCRIPTOR._options = None + _globals["_READRESPONSE_RESULT_HEADERSENTRY"]._options = None + _globals["_READRESPONSE_RESULT_HEADERSENTRY"]._serialized_options = b"8\001" + _globals["_HANDSHAKE"]._serialized_start = 89 + _globals["_HANDSHAKE"]._serialized_end = 113 + _globals["_READREQUEST"]._serialized_start = 116 + _globals["_READREQUEST"]._serialized_end = 293 + _globals["_READREQUEST_REQUEST"]._serialized_start = 226 + _globals["_READREQUEST_REQUEST"]._serialized_end = 279 + _globals["_READRESPONSE"]._serialized_start = 296 + _globals["_READRESPONSE"]._serialized_end = 937 + _globals["_READRESPONSE_RESULT"]._serialized_start = 455 + _globals["_READRESPONSE_RESULT"]._serialized_end = 687 + _globals["_READRESPONSE_RESULT_HEADERSENTRY"]._serialized_start = 641 + _globals["_READRESPONSE_RESULT_HEADERSENTRY"]._serialized_end = 687 + _globals["_READRESPONSE_STATUS"]._serialized_start = 690 + _globals["_READRESPONSE_STATUS"]._serialized_end = 923 + _globals["_READRESPONSE_STATUS_CODE"]._serialized_start = 840 + _globals["_READRESPONSE_STATUS_CODE"]._serialized_end = 872 + _globals["_READRESPONSE_STATUS_ERROR"]._serialized_start = 874 + _globals["_READRESPONSE_STATUS_ERROR"]._serialized_end = 905 + _globals["_ACKREQUEST"]._serialized_start = 940 + _globals["_ACKREQUEST"]._serialized_end = 1107 + _globals["_ACKREQUEST_REQUEST"]._serialized_start = 1048 + _globals["_ACKREQUEST_REQUEST"]._serialized_end = 1093 + _globals["_ACKRESPONSE"]._serialized_start = 1110 + _globals["_ACKRESPONSE"]._serialized_end = 1281 + _globals["_ACKRESPONSE_RESULT"]._serialized_start = 1218 + _globals["_ACKRESPONSE_RESULT"]._serialized_end = 1267 + _globals["_READYRESPONSE"]._serialized_start = 1283 + _globals["_READYRESPONSE"]._serialized_end = 1313 + _globals["_PENDINGRESPONSE"]._serialized_start = 1315 + _globals["_PENDINGRESPONSE"]._serialized_end = 1408 + _globals["_PENDINGRESPONSE_RESULT"]._serialized_start = 1385 + _globals["_PENDINGRESPONSE_RESULT"]._serialized_end = 1408 + _globals["_PARTITIONSRESPONSE"]._serialized_start = 1410 + _globals["_PARTITIONSRESPONSE"]._serialized_end = 1514 + _globals["_PARTITIONSRESPONSE_RESULT"]._serialized_start = 1486 + _globals["_PARTITIONSRESPONSE_RESULT"]._serialized_end = 1514 + _globals["_OFFSET"]._serialized_start = 1516 + _globals["_OFFSET"]._serialized_end = 1562 + _globals["_SOURCE"]._serialized_start = 1565 + _globals["_SOURCE"]._serialized_end = 1893 # @@protoc_insertion_point(module_scope) diff --git a/pynumaflow/proto/sourcer/source_pb2.pyi b/pynumaflow/proto/sourcer/source_pb2.pyi index 52c20928..f2cdc70e 100644 --- a/pynumaflow/proto/sourcer/source_pb2.pyi +++ b/pynumaflow/proto/sourcer/source_pb2.pyi @@ -4,7 +4,13 @@ from google.protobuf.internal import containers as _containers from google.protobuf.internal import enum_type_wrapper as _enum_type_wrapper from google.protobuf import descriptor as _descriptor from google.protobuf import message as _message -from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union +from typing import ( + ClassVar as _ClassVar, + Iterable as _Iterable, + Mapping as _Mapping, + Optional as _Optional, + Union as _Union, +) DESCRIPTOR: _descriptor.FileDescriptor @@ -16,23 +22,32 @@ class Handshake(_message.Message): class ReadRequest(_message.Message): __slots__ = ("request", "handshake") + class Request(_message.Message): __slots__ = ("num_records", "timeout_in_ms") NUM_RECORDS_FIELD_NUMBER: _ClassVar[int] TIMEOUT_IN_MS_FIELD_NUMBER: _ClassVar[int] num_records: int timeout_in_ms: int - def __init__(self, num_records: _Optional[int] = ..., timeout_in_ms: _Optional[int] = ...) -> None: ... + def __init__( + self, num_records: _Optional[int] = ..., timeout_in_ms: _Optional[int] = ... + ) -> None: ... REQUEST_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] request: ReadRequest.Request handshake: Handshake - def __init__(self, request: _Optional[_Union[ReadRequest.Request, _Mapping]] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ...) -> None: ... + def __init__( + self, + request: _Optional[_Union[ReadRequest.Request, _Mapping]] = ..., + handshake: _Optional[_Union[Handshake, _Mapping]] = ..., + ) -> None: ... class ReadResponse(_message.Message): __slots__ = ("result", "status", "handshake") + class Result(_message.Message): __slots__ = ("payload", "offset", "event_time", "keys", "headers") + class HeadersEntry(_message.Message): __slots__ = ("key", "value") KEY_FIELD_NUMBER: _ClassVar[int] @@ -50,15 +65,25 @@ class ReadResponse(_message.Message): event_time: _timestamp_pb2.Timestamp keys: _containers.RepeatedScalarFieldContainer[str] headers: _containers.ScalarMap[str, str] - def __init__(self, payload: _Optional[bytes] = ..., offset: _Optional[_Union[Offset, _Mapping]] = ..., event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., keys: _Optional[_Iterable[str]] = ..., headers: _Optional[_Mapping[str, str]] = ...) -> None: ... + def __init__( + self, + payload: _Optional[bytes] = ..., + offset: _Optional[_Union[Offset, _Mapping]] = ..., + event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + keys: _Optional[_Iterable[str]] = ..., + headers: _Optional[_Mapping[str, str]] = ..., + ) -> None: ... + class Status(_message.Message): __slots__ = ("eot", "code", "error", "msg") + class Code(int, metaclass=_enum_type_wrapper.EnumTypeWrapper): __slots__ = () SUCCESS: _ClassVar[ReadResponse.Status.Code] FAILURE: _ClassVar[ReadResponse.Status.Code] SUCCESS: ReadResponse.Status.Code FAILURE: ReadResponse.Status.Code + class Error(int, metaclass=_enum_type_wrapper.EnumTypeWrapper): __slots__ = () UNACKED: _ClassVar[ReadResponse.Status.Error] @@ -73,40 +98,65 @@ class ReadResponse(_message.Message): code: ReadResponse.Status.Code error: ReadResponse.Status.Error msg: str - def __init__(self, eot: bool = ..., code: _Optional[_Union[ReadResponse.Status.Code, str]] = ..., error: _Optional[_Union[ReadResponse.Status.Error, str]] = ..., msg: _Optional[str] = ...) -> None: ... + def __init__( + self, + eot: bool = ..., + code: _Optional[_Union[ReadResponse.Status.Code, str]] = ..., + error: _Optional[_Union[ReadResponse.Status.Error, str]] = ..., + msg: _Optional[str] = ..., + ) -> None: ... RESULT_FIELD_NUMBER: _ClassVar[int] STATUS_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] result: ReadResponse.Result status: ReadResponse.Status handshake: Handshake - def __init__(self, result: _Optional[_Union[ReadResponse.Result, _Mapping]] = ..., status: _Optional[_Union[ReadResponse.Status, _Mapping]] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ...) -> None: ... + def __init__( + self, + result: _Optional[_Union[ReadResponse.Result, _Mapping]] = ..., + status: _Optional[_Union[ReadResponse.Status, _Mapping]] = ..., + handshake: _Optional[_Union[Handshake, _Mapping]] = ..., + ) -> None: ... class AckRequest(_message.Message): __slots__ = ("request", "handshake") + class Request(_message.Message): __slots__ = ("offsets",) OFFSETS_FIELD_NUMBER: _ClassVar[int] offsets: _containers.RepeatedCompositeFieldContainer[Offset] - def __init__(self, offsets: _Optional[_Iterable[_Union[Offset, _Mapping]]] = ...) -> None: ... + def __init__( + self, offsets: _Optional[_Iterable[_Union[Offset, _Mapping]]] = ... + ) -> None: ... REQUEST_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] request: AckRequest.Request handshake: Handshake - def __init__(self, request: _Optional[_Union[AckRequest.Request, _Mapping]] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ...) -> None: ... + def __init__( + self, + request: _Optional[_Union[AckRequest.Request, _Mapping]] = ..., + handshake: _Optional[_Union[Handshake, _Mapping]] = ..., + ) -> None: ... class AckResponse(_message.Message): __slots__ = ("result", "handshake") + class Result(_message.Message): __slots__ = ("success",) SUCCESS_FIELD_NUMBER: _ClassVar[int] success: _empty_pb2.Empty - def __init__(self, success: _Optional[_Union[_empty_pb2.Empty, _Mapping]] = ...) -> None: ... + def __init__( + self, success: _Optional[_Union[_empty_pb2.Empty, _Mapping]] = ... + ) -> None: ... RESULT_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] result: AckResponse.Result handshake: Handshake - def __init__(self, result: _Optional[_Union[AckResponse.Result, _Mapping]] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ...) -> None: ... + def __init__( + self, + result: _Optional[_Union[AckResponse.Result, _Mapping]] = ..., + handshake: _Optional[_Union[Handshake, _Mapping]] = ..., + ) -> None: ... class ReadyResponse(_message.Message): __slots__ = ("ready",) @@ -116,6 +166,7 @@ class ReadyResponse(_message.Message): class PendingResponse(_message.Message): __slots__ = ("result",) + class Result(_message.Message): __slots__ = ("count",) COUNT_FIELD_NUMBER: _ClassVar[int] @@ -123,10 +174,13 @@ class PendingResponse(_message.Message): def __init__(self, count: _Optional[int] = ...) -> None: ... RESULT_FIELD_NUMBER: _ClassVar[int] result: PendingResponse.Result - def __init__(self, result: _Optional[_Union[PendingResponse.Result, _Mapping]] = ...) -> None: ... + def __init__( + self, result: _Optional[_Union[PendingResponse.Result, _Mapping]] = ... + ) -> None: ... class PartitionsResponse(_message.Message): __slots__ = ("result",) + class Result(_message.Message): __slots__ = ("partitions",) PARTITIONS_FIELD_NUMBER: _ClassVar[int] @@ -134,7 +188,9 @@ class PartitionsResponse(_message.Message): def __init__(self, partitions: _Optional[_Iterable[int]] = ...) -> None: ... RESULT_FIELD_NUMBER: _ClassVar[int] result: PartitionsResponse.Result - def __init__(self, result: _Optional[_Union[PartitionsResponse.Result, _Mapping]] = ...) -> None: ... + def __init__( + self, result: _Optional[_Union[PartitionsResponse.Result, _Mapping]] = ... + ) -> None: ... class Offset(_message.Message): __slots__ = ("offset", "partition_id") @@ -142,4 +198,6 @@ class Offset(_message.Message): PARTITION_ID_FIELD_NUMBER: _ClassVar[int] offset: bytes partition_id: int - def __init__(self, offset: _Optional[bytes] = ..., partition_id: _Optional[int] = ...) -> None: ... + def __init__( + self, offset: _Optional[bytes] = ..., partition_id: _Optional[int] = ... + ) -> None: ... diff --git a/pynumaflow/proto/sourcer/source_pb2_grpc.py b/pynumaflow/proto/sourcer/source_pb2_grpc.py index 14ad1d5a..d5e09f1b 100644 --- a/pynumaflow/proto/sourcer/source_pb2_grpc.py +++ b/pynumaflow/proto/sourcer/source_pb2_grpc.py @@ -16,30 +16,30 @@ def __init__(self, channel): channel: A grpc.Channel. """ self.ReadFn = channel.stream_stream( - '/source.v1.Source/ReadFn', - request_serializer=source__pb2.ReadRequest.SerializeToString, - response_deserializer=source__pb2.ReadResponse.FromString, - ) + "/source.v1.Source/ReadFn", + request_serializer=source__pb2.ReadRequest.SerializeToString, + response_deserializer=source__pb2.ReadResponse.FromString, + ) self.AckFn = channel.stream_stream( - '/source.v1.Source/AckFn', - request_serializer=source__pb2.AckRequest.SerializeToString, - response_deserializer=source__pb2.AckResponse.FromString, - ) + "/source.v1.Source/AckFn", + request_serializer=source__pb2.AckRequest.SerializeToString, + response_deserializer=source__pb2.AckResponse.FromString, + ) self.PendingFn = channel.unary_unary( - '/source.v1.Source/PendingFn', - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=source__pb2.PendingResponse.FromString, - ) + "/source.v1.Source/PendingFn", + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=source__pb2.PendingResponse.FromString, + ) self.PartitionsFn = channel.unary_unary( - '/source.v1.Source/PartitionsFn', - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=source__pb2.PartitionsResponse.FromString, - ) + "/source.v1.Source/PartitionsFn", + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=source__pb2.PartitionsResponse.FromString, + ) self.IsReady = channel.unary_unary( - '/source.v1.Source/IsReady', - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=source__pb2.ReadyResponse.FromString, - ) + "/source.v1.Source/IsReady", + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=source__pb2.ReadyResponse.FromString, + ) class SourceServicer(object): @@ -53,8 +53,8 @@ def ReadFn(self, request_iterator, context): Once it has sent all the datum, the server will send a ReadResponse with the end of transmission flag set to true. """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def AckFn(self, request_iterator, context): """AckFn acknowledges a stream of datum offsets. @@ -65,149 +65,205 @@ def AckFn(self, request_iterator, context): Clients sends n requests and expects n responses. """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def PendingFn(self, request, context): - """PendingFn returns the number of pending records at the user defined source. - """ + """PendingFn returns the number of pending records at the user defined source.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def PartitionsFn(self, request, context): - """PartitionsFn returns the list of partitions for the user defined source. - """ + """PartitionsFn returns the list of partitions for the user defined source.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def IsReady(self, request, context): - """IsReady is the heartbeat endpoint for user defined source gRPC. - """ + """IsReady is the heartbeat endpoint for user defined source gRPC.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def add_SourceServicer_to_server(servicer, server): rpc_method_handlers = { - 'ReadFn': grpc.stream_stream_rpc_method_handler( - servicer.ReadFn, - request_deserializer=source__pb2.ReadRequest.FromString, - response_serializer=source__pb2.ReadResponse.SerializeToString, - ), - 'AckFn': grpc.stream_stream_rpc_method_handler( - servicer.AckFn, - request_deserializer=source__pb2.AckRequest.FromString, - response_serializer=source__pb2.AckResponse.SerializeToString, - ), - 'PendingFn': grpc.unary_unary_rpc_method_handler( - servicer.PendingFn, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=source__pb2.PendingResponse.SerializeToString, - ), - 'PartitionsFn': grpc.unary_unary_rpc_method_handler( - servicer.PartitionsFn, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=source__pb2.PartitionsResponse.SerializeToString, - ), - 'IsReady': grpc.unary_unary_rpc_method_handler( - servicer.IsReady, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=source__pb2.ReadyResponse.SerializeToString, - ), + "ReadFn": grpc.stream_stream_rpc_method_handler( + servicer.ReadFn, + request_deserializer=source__pb2.ReadRequest.FromString, + response_serializer=source__pb2.ReadResponse.SerializeToString, + ), + "AckFn": grpc.stream_stream_rpc_method_handler( + servicer.AckFn, + request_deserializer=source__pb2.AckRequest.FromString, + response_serializer=source__pb2.AckResponse.SerializeToString, + ), + "PendingFn": grpc.unary_unary_rpc_method_handler( + servicer.PendingFn, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=source__pb2.PendingResponse.SerializeToString, + ), + "PartitionsFn": grpc.unary_unary_rpc_method_handler( + servicer.PartitionsFn, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=source__pb2.PartitionsResponse.SerializeToString, + ), + "IsReady": grpc.unary_unary_rpc_method_handler( + servicer.IsReady, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=source__pb2.ReadyResponse.SerializeToString, + ), } - generic_handler = grpc.method_handlers_generic_handler( - 'source.v1.Source', rpc_method_handlers) + generic_handler = grpc.method_handlers_generic_handler("source.v1.Source", rpc_method_handlers) server.add_generic_rpc_handlers((generic_handler,)) - # This class is part of an EXPERIMENTAL API. +# This class is part of an EXPERIMENTAL API. class Source(object): """Missing associated documentation comment in .proto file.""" @staticmethod - def ReadFn(request_iterator, + def ReadFn( + request_iterator, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.stream_stream( + request_iterator, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.stream_stream(request_iterator, target, '/source.v1.Source/ReadFn', + "/source.v1.Source/ReadFn", source__pb2.ReadRequest.SerializeToString, source__pb2.ReadResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) @staticmethod - def AckFn(request_iterator, + def AckFn( + request_iterator, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.stream_stream( + request_iterator, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.stream_stream(request_iterator, target, '/source.v1.Source/AckFn', + "/source.v1.Source/AckFn", source__pb2.AckRequest.SerializeToString, source__pb2.AckResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) @staticmethod - def PendingFn(request, + def PendingFn( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.unary_unary(request, target, '/source.v1.Source/PendingFn', + "/source.v1.Source/PendingFn", google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, source__pb2.PendingResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) @staticmethod - def PartitionsFn(request, + def PartitionsFn( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.unary_unary(request, target, '/source.v1.Source/PartitionsFn', + "/source.v1.Source/PartitionsFn", google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, source__pb2.PartitionsResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) @staticmethod - def IsReady(request, + def IsReady( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.unary_unary(request, target, '/source.v1.Source/IsReady', + "/source.v1.Source/IsReady", google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, source__pb2.ReadyResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) diff --git a/pynumaflow/proto/sourcetransformer/transform_pb2.py b/pynumaflow/proto/sourcetransformer/transform_pb2.py index 8309b7ca..a7b08491 100644 --- a/pynumaflow/proto/sourcetransformer/transform_pb2.py +++ b/pynumaflow/proto/sourcetransformer/transform_pb2.py @@ -7,6 +7,7 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import symbol_database as _symbol_database from google.protobuf.internal import builder as _builder + # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() @@ -16,29 +17,31 @@ from google.protobuf import empty_pb2 as google_dot_protobuf_dot_empty__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x0ftransform.proto\x12\x14sourcetransformer.v1\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1bgoogle/protobuf/empty.proto\"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08\"\xbe\x03\n\x16SourceTransformRequest\x12\x45\n\x07request\x18\x01 \x01(\x0b\x32\x34.sourcetransformer.v1.SourceTransformRequest.Request\x12\x37\n\thandshake\x18\x02 \x01(\x0b\x32\x1f.sourcetransformer.v1.HandshakeH\x00\x88\x01\x01\x1a\x95\x02\n\x07Request\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12R\n\x07headers\x18\x05 \x03(\x0b\x32\x41.sourcetransformer.v1.SourceTransformRequest.Request.HeadersEntry\x12\n\n\x02id\x18\x06 \x01(\t\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x42\x0c\n\n_handshake\"\x98\x02\n\x17SourceTransformResponse\x12\x45\n\x07results\x18\x01 \x03(\x0b\x32\x34.sourcetransformer.v1.SourceTransformResponse.Result\x12\n\n\x02id\x18\x02 \x01(\t\x12\x37\n\thandshake\x18\x03 \x01(\x0b\x32\x1f.sourcetransformer.v1.HandshakeH\x00\x88\x01\x01\x1a\x63\n\x06Result\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04tags\x18\x04 \x03(\tB\x0c\n\n_handshake\"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\xcf\x01\n\x0fSourceTransform\x12t\n\x11SourceTransformFn\x12,.sourcetransformer.v1.SourceTransformRequest\x1a-.sourcetransformer.v1.SourceTransformResponse(\x01\x30\x01\x12\x46\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a#.sourcetransformer.v1.ReadyResponseb\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( + b'\n\x0ftransform.proto\x12\x14sourcetransformer.v1\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1bgoogle/protobuf/empty.proto"\x18\n\tHandshake\x12\x0b\n\x03sot\x18\x01 \x01(\x08"\xbe\x03\n\x16SourceTransformRequest\x12\x45\n\x07request\x18\x01 \x01(\x0b\x32\x34.sourcetransformer.v1.SourceTransformRequest.Request\x12\x37\n\thandshake\x18\x02 \x01(\x0b\x32\x1f.sourcetransformer.v1.HandshakeH\x00\x88\x01\x01\x1a\x95\x02\n\x07Request\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12-\n\twatermark\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12R\n\x07headers\x18\x05 \x03(\x0b\x32\x41.sourcetransformer.v1.SourceTransformRequest.Request.HeadersEntry\x12\n\n\x02id\x18\x06 \x01(\t\x1a.\n\x0cHeadersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x42\x0c\n\n_handshake"\x98\x02\n\x17SourceTransformResponse\x12\x45\n\x07results\x18\x01 \x03(\x0b\x32\x34.sourcetransformer.v1.SourceTransformResponse.Result\x12\n\n\x02id\x18\x02 \x01(\t\x12\x37\n\thandshake\x18\x03 \x01(\x0b\x32\x1f.sourcetransformer.v1.HandshakeH\x00\x88\x01\x01\x1a\x63\n\x06Result\x12\x0c\n\x04keys\x18\x01 \x03(\t\x12\r\n\x05value\x18\x02 \x01(\x0c\x12.\n\nevent_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0c\n\x04tags\x18\x04 \x03(\tB\x0c\n\n_handshake"\x1e\n\rReadyResponse\x12\r\n\x05ready\x18\x01 \x01(\x08\x32\xcf\x01\n\x0fSourceTransform\x12t\n\x11SourceTransformFn\x12,.sourcetransformer.v1.SourceTransformRequest\x1a-.sourcetransformer.v1.SourceTransformResponse(\x01\x30\x01\x12\x46\n\x07IsReady\x12\x16.google.protobuf.Empty\x1a#.sourcetransformer.v1.ReadyResponseb\x06proto3' +) _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) -_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'transform_pb2', _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, "transform_pb2", _globals) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None - _globals['_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY']._options = None - _globals['_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY']._serialized_options = b'8\001' - _globals['_HANDSHAKE']._serialized_start=103 - _globals['_HANDSHAKE']._serialized_end=127 - _globals['_SOURCETRANSFORMREQUEST']._serialized_start=130 - _globals['_SOURCETRANSFORMREQUEST']._serialized_end=576 - _globals['_SOURCETRANSFORMREQUEST_REQUEST']._serialized_start=285 - _globals['_SOURCETRANSFORMREQUEST_REQUEST']._serialized_end=562 - _globals['_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY']._serialized_start=516 - _globals['_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY']._serialized_end=562 - _globals['_SOURCETRANSFORMRESPONSE']._serialized_start=579 - _globals['_SOURCETRANSFORMRESPONSE']._serialized_end=859 - _globals['_SOURCETRANSFORMRESPONSE_RESULT']._serialized_start=746 - _globals['_SOURCETRANSFORMRESPONSE_RESULT']._serialized_end=845 - _globals['_READYRESPONSE']._serialized_start=861 - _globals['_READYRESPONSE']._serialized_end=891 - _globals['_SOURCETRANSFORM']._serialized_start=894 - _globals['_SOURCETRANSFORM']._serialized_end=1101 + DESCRIPTOR._options = None + _globals["_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY"]._options = None + _globals["_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY"]._serialized_options = b"8\001" + _globals["_HANDSHAKE"]._serialized_start = 103 + _globals["_HANDSHAKE"]._serialized_end = 127 + _globals["_SOURCETRANSFORMREQUEST"]._serialized_start = 130 + _globals["_SOURCETRANSFORMREQUEST"]._serialized_end = 576 + _globals["_SOURCETRANSFORMREQUEST_REQUEST"]._serialized_start = 285 + _globals["_SOURCETRANSFORMREQUEST_REQUEST"]._serialized_end = 562 + _globals["_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY"]._serialized_start = 516 + _globals["_SOURCETRANSFORMREQUEST_REQUEST_HEADERSENTRY"]._serialized_end = 562 + _globals["_SOURCETRANSFORMRESPONSE"]._serialized_start = 579 + _globals["_SOURCETRANSFORMRESPONSE"]._serialized_end = 859 + _globals["_SOURCETRANSFORMRESPONSE_RESULT"]._serialized_start = 746 + _globals["_SOURCETRANSFORMRESPONSE_RESULT"]._serialized_end = 845 + _globals["_READYRESPONSE"]._serialized_start = 861 + _globals["_READYRESPONSE"]._serialized_end = 891 + _globals["_SOURCETRANSFORM"]._serialized_start = 894 + _globals["_SOURCETRANSFORM"]._serialized_end = 1101 # @@protoc_insertion_point(module_scope) diff --git a/pynumaflow/proto/sourcetransformer/transform_pb2.pyi b/pynumaflow/proto/sourcetransformer/transform_pb2.pyi index c7c15079..cc8fe420 100644 --- a/pynumaflow/proto/sourcetransformer/transform_pb2.pyi +++ b/pynumaflow/proto/sourcetransformer/transform_pb2.pyi @@ -3,7 +3,13 @@ from google.protobuf import empty_pb2 as _empty_pb2 from google.protobuf.internal import containers as _containers from google.protobuf import descriptor as _descriptor from google.protobuf import message as _message -from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union +from typing import ( + ClassVar as _ClassVar, + Iterable as _Iterable, + Mapping as _Mapping, + Optional as _Optional, + Union as _Union, +) DESCRIPTOR: _descriptor.FileDescriptor @@ -15,8 +21,10 @@ class Handshake(_message.Message): class SourceTransformRequest(_message.Message): __slots__ = ("request", "handshake") + class Request(_message.Message): __slots__ = ("keys", "value", "event_time", "watermark", "headers", "id") + class HeadersEntry(_message.Message): __slots__ = ("key", "value") KEY_FIELD_NUMBER: _ClassVar[int] @@ -36,15 +44,28 @@ class SourceTransformRequest(_message.Message): watermark: _timestamp_pb2.Timestamp headers: _containers.ScalarMap[str, str] id: str - def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., headers: _Optional[_Mapping[str, str]] = ..., id: _Optional[str] = ...) -> None: ... + def __init__( + self, + keys: _Optional[_Iterable[str]] = ..., + value: _Optional[bytes] = ..., + event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + watermark: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + headers: _Optional[_Mapping[str, str]] = ..., + id: _Optional[str] = ..., + ) -> None: ... REQUEST_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] request: SourceTransformRequest.Request handshake: Handshake - def __init__(self, request: _Optional[_Union[SourceTransformRequest.Request, _Mapping]] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ...) -> None: ... + def __init__( + self, + request: _Optional[_Union[SourceTransformRequest.Request, _Mapping]] = ..., + handshake: _Optional[_Union[Handshake, _Mapping]] = ..., + ) -> None: ... class SourceTransformResponse(_message.Message): __slots__ = ("results", "id", "handshake") + class Result(_message.Message): __slots__ = ("keys", "value", "event_time", "tags") KEYS_FIELD_NUMBER: _ClassVar[int] @@ -55,14 +76,25 @@ class SourceTransformResponse(_message.Message): value: bytes event_time: _timestamp_pb2.Timestamp tags: _containers.RepeatedScalarFieldContainer[str] - def __init__(self, keys: _Optional[_Iterable[str]] = ..., value: _Optional[bytes] = ..., event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., tags: _Optional[_Iterable[str]] = ...) -> None: ... + def __init__( + self, + keys: _Optional[_Iterable[str]] = ..., + value: _Optional[bytes] = ..., + event_time: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., + tags: _Optional[_Iterable[str]] = ..., + ) -> None: ... RESULTS_FIELD_NUMBER: _ClassVar[int] ID_FIELD_NUMBER: _ClassVar[int] HANDSHAKE_FIELD_NUMBER: _ClassVar[int] results: _containers.RepeatedCompositeFieldContainer[SourceTransformResponse.Result] id: str handshake: Handshake - def __init__(self, results: _Optional[_Iterable[_Union[SourceTransformResponse.Result, _Mapping]]] = ..., id: _Optional[str] = ..., handshake: _Optional[_Union[Handshake, _Mapping]] = ...) -> None: ... + def __init__( + self, + results: _Optional[_Iterable[_Union[SourceTransformResponse.Result, _Mapping]]] = ..., + id: _Optional[str] = ..., + handshake: _Optional[_Union[Handshake, _Mapping]] = ..., + ) -> None: ... class ReadyResponse(_message.Message): __slots__ = ("ready",) diff --git a/pynumaflow/proto/sourcetransformer/transform_pb2_grpc.py b/pynumaflow/proto/sourcetransformer/transform_pb2_grpc.py index 1a1ea499..c6820d39 100644 --- a/pynumaflow/proto/sourcetransformer/transform_pb2_grpc.py +++ b/pynumaflow/proto/sourcetransformer/transform_pb2_grpc.py @@ -16,15 +16,15 @@ def __init__(self, channel): channel: A grpc.Channel. """ self.SourceTransformFn = channel.stream_stream( - '/sourcetransformer.v1.SourceTransform/SourceTransformFn', - request_serializer=transform__pb2.SourceTransformRequest.SerializeToString, - response_deserializer=transform__pb2.SourceTransformResponse.FromString, - ) + "/sourcetransformer.v1.SourceTransform/SourceTransformFn", + request_serializer=transform__pb2.SourceTransformRequest.SerializeToString, + response_deserializer=transform__pb2.SourceTransformResponse.FromString, + ) self.IsReady = channel.unary_unary( - '/sourcetransformer.v1.SourceTransform/IsReady', - request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, - response_deserializer=transform__pb2.ReadyResponse.FromString, - ) + "/sourcetransformer.v1.SourceTransform/IsReady", + request_serializer=google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, + response_deserializer=transform__pb2.ReadyResponse.FromString, + ) class SourceTransformServicer(object): @@ -36,69 +36,93 @@ def SourceTransformFn(self, request_iterator, context): SourceTransformFn can be used only at source vertex by source data transformer. """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def IsReady(self, request, context): - """IsReady is the heartbeat endpoint for gRPC. - """ + """IsReady is the heartbeat endpoint for gRPC.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def add_SourceTransformServicer_to_server(servicer, server): rpc_method_handlers = { - 'SourceTransformFn': grpc.stream_stream_rpc_method_handler( - servicer.SourceTransformFn, - request_deserializer=transform__pb2.SourceTransformRequest.FromString, - response_serializer=transform__pb2.SourceTransformResponse.SerializeToString, - ), - 'IsReady': grpc.unary_unary_rpc_method_handler( - servicer.IsReady, - request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, - response_serializer=transform__pb2.ReadyResponse.SerializeToString, - ), + "SourceTransformFn": grpc.stream_stream_rpc_method_handler( + servicer.SourceTransformFn, + request_deserializer=transform__pb2.SourceTransformRequest.FromString, + response_serializer=transform__pb2.SourceTransformResponse.SerializeToString, + ), + "IsReady": grpc.unary_unary_rpc_method_handler( + servicer.IsReady, + request_deserializer=google_dot_protobuf_dot_empty__pb2.Empty.FromString, + response_serializer=transform__pb2.ReadyResponse.SerializeToString, + ), } generic_handler = grpc.method_handlers_generic_handler( - 'sourcetransformer.v1.SourceTransform', rpc_method_handlers) + "sourcetransformer.v1.SourceTransform", rpc_method_handlers + ) server.add_generic_rpc_handlers((generic_handler,)) - # This class is part of an EXPERIMENTAL API. +# This class is part of an EXPERIMENTAL API. class SourceTransform(object): """Missing associated documentation comment in .proto file.""" @staticmethod - def SourceTransformFn(request_iterator, + def SourceTransformFn( + request_iterator, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.stream_stream( + request_iterator, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.stream_stream(request_iterator, target, '/sourcetransformer.v1.SourceTransform/SourceTransformFn', + "/sourcetransformer.v1.SourceTransform/SourceTransformFn", transform__pb2.SourceTransformRequest.SerializeToString, transform__pb2.SourceTransformResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) @staticmethod - def IsReady(request, + def IsReady( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.unary_unary(request, target, '/sourcetransformer.v1.SourceTransform/IsReady', + "/sourcetransformer.v1.SourceTransform/IsReady", google_dot_protobuf_dot_empty__pb2.Empty.SerializeToString, transform__pb2.ReadyResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) From 42c0264cc837902e59c62084f047d86c6f753c9c Mon Sep 17 00:00:00 2001 From: srao12 Date: Wed, 30 Jul 2025 00:47:40 -0700 Subject: [PATCH 28/30] fix: remove tests Signed-off-by: srao12 --- tests/accumulator/test_datatypes.py | 22 --- tests/accumulator/test_use_cases.py | 212 ---------------------------- 2 files changed, 234 deletions(-) delete mode 100644 tests/accumulator/test_use_cases.py diff --git a/tests/accumulator/test_datatypes.py b/tests/accumulator/test_datatypes.py index 6f2bb1a2..17b2a7a4 100644 --- a/tests/accumulator/test_datatypes.py +++ b/tests/accumulator/test_datatypes.py @@ -310,28 +310,6 @@ def test_init(self): self.assertEqual(2, r.test2) self.assertEqual(1, r.test3) - def test_deep_copy(self): - """Test that the deepcopy works as expected""" - r = self.ExampleClass(test1=1, test2=2) - # Create a copy of r - r_copy = deepcopy(r) - # Check that the attributes are the same - self.assertEqual(1, r_copy.test1) - self.assertEqual(2, r_copy.test2) - self.assertEqual(1, r_copy.test3) - # Check that the objects are not the same - self.assertNotEqual(id(r), id(r_copy)) - # Update the attributes of r - r.test1 = 5 - r.test3 = 6 - # Check that the other object is not updated - self.assertNotEqual(r.test1, r_copy.test1) - self.assertNotEqual(r.test3, r_copy.test3) - self.assertNotEqual(id(r.test3), id(r_copy.test3)) - # Verify that the instance type is correct - self.assertTrue(isinstance(r_copy, self.ExampleClass)) - self.assertTrue(isinstance(r_copy, Accumulator)) - def test_callable(self): """Test that accumulator instances can be called directly""" r = self.ExampleClass(test1=1, test2=2) diff --git a/tests/accumulator/test_use_cases.py b/tests/accumulator/test_use_cases.py deleted file mode 100644 index 2f784282..00000000 --- a/tests/accumulator/test_use_cases.py +++ /dev/null @@ -1,212 +0,0 @@ -"""Accumulator Use Case Examples - -These tests demonstrate working patterns for common accumulator use cases. - -Functionality Being Tested: -=========================== - -1. StreamSorterAccumulator: - - Collects incoming Datum objects from async streams - - Sorts events by event_time in chronological order - - Emits sorted events as Message objects - - Manages internal buffer state between processing windows - -2. AccumulatorAsyncServer Integration: - - Server instantiation with custom accumulator classes - - Initialization argument passing to accumulator constructors - - Server configuration with different accumulator types - -Verification Criteria: -===================== - -1. End-to-End Processing: - - Input: Out-of-order events with different timestamps - - Processing: Async stream handling and sorting logic - - Output: Chronologically ordered Message objects - - State Management: Buffer clearing after processing - -2. Server Integration: - - Server creation with various accumulator configurations - - Proper servicer instantiation and lifecycle - - Support for parameterized accumulator constructors - -3. Data Flow Validation: - - Datum to Message conversion patterns - - Async iterator usage with STREAM_EOF handling - - Message format and content verification -""" - -import asyncio -import unittest -from collections.abc import AsyncIterable -from datetime import datetime, timezone - -from pynumaflow.accumulator import ( - Message, - Datum, - Accumulator, - AccumulatorAsyncServer, -) -from pynumaflow.shared.asynciter import NonBlockingIterator -from pynumaflow._constants import STREAM_EOF -from tests.accumulator.utils import create_test_datum - - -class StreamSorterAccumulator(Accumulator): - """ - Accumulator that sorts events by event_time and watermark. - This demonstrates custom sorting use case. - """ - - def __init__(self): - self.buffer: list[Datum] = [] - - async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterator): - # Collect all datums - datum_count = 0 - async for datum in datums: - datum_count += 1 - self.buffer.append(datum) - - # Sort by event_time - self.buffer.sort(key=lambda d: d.event_time) - - # Emit sorted datums - message_count = 0 - for datum in self.buffer: - message_count += 1 - await output.put( - Message( - value=datum.value, - keys=datum.keys(), - tags=[], - ) - ) - - # Clear buffer for next window - self.buffer.clear() - - -class TestAccumulatorUseCases(unittest.TestCase): - """Test practical accumulator use cases that developers can reference.""" - - def test_stream_sorter_end_to_end_example(self): - """Complete end-to-end example showing how to build a stream sorting accumulator. - - This test demonstrates: - 1. How to implement the Accumulator abstract class - 2. How to process async streams of Datum objects - 3. How to emit results as Message objects - 4. Proper resource management (buffer clearing) - """ - - async def _test_end_to_end(): - # Create accumulator instance - sorter = StreamSorterAccumulator() - output = NonBlockingIterator() - - # Create test data - intentionally out of chronological order - test_data = [ - create_test_datum( - keys=["sensor_data"], - value=b"temperature:25.5", - event_time=datetime.fromtimestamp(1662998460, timezone.utc), # Latest - id_="temp_3", - ), - create_test_datum( - keys=["sensor_data"], - value=b"temperature:22.1", - event_time=datetime.fromtimestamp(1662998400, timezone.utc), # Earliest - id_="temp_1", - ), - create_test_datum( - keys=["sensor_data"], - value=b"temperature:23.8", - event_time=datetime.fromtimestamp(1662998430, timezone.utc), # Middle - id_="temp_2", - ), - ] - - # Create async generator (simulates streaming data) - async def data_stream(): - for datum in test_data: - yield datum - - # Process the stream through our accumulator - await sorter.handler(data_stream(), output) - - # Collect and verify results - results = [] - await output.put(STREAM_EOF) # Signal end of stream - - async for item in output.read_iterator(): - if item == STREAM_EOF: - break - results.append(item) - if len(results) >= 10: # Safety break - break - - # Verify we received the expected number of messages - self.assertEqual(len(results), 3) - - # Verify all results are properly formatted Message objects - for i, result in enumerate(results): - self.assertIsInstance(result, Message) - self.assertEqual(result.keys, ["sensor_data"]) - self.assertEqual(result.tags, []) - # Values should be in chronological order due to sorting - if i == 0: - self.assertEqual(result.value, b"temperature:22.1") # Earliest - elif i == 1: - self.assertEqual(result.value, b"temperature:23.8") # Middle - elif i == 2: - self.assertEqual(result.value, b"temperature:25.5") # Latest - - asyncio.run(_test_end_to_end()) - - def test_accumulator_server_integration_example(self): - """Example showing how to integrate custom accumulator with AccumulatorAsyncServer. - - This demonstrates: - 1. Server instantiation with custom accumulator class - 2. Passing initialization arguments to accumulator constructors - 3. Server lifecycle management - """ - # Example 1: Simple accumulator without initialization args - server = AccumulatorAsyncServer(StreamSorterAccumulator) - self.assertIsNotNone(server) - self.assertIsNotNone(server.servicer) - - # Example 2: Accumulator with configuration parameters - class ConfigurableAccumulator(Accumulator): - def __init__(self, max_buffer_size: int = 1000, sort_ascending: bool = True): - self.max_buffer_size = max_buffer_size - self.sort_ascending = sort_ascending - self.buffer = [] - - async def handler(self, datums: AsyncIterable[Datum], output: NonBlockingIterator): - # Collect datums up to max buffer size - async for datum in datums: - if len(self.buffer) < self.max_buffer_size: - self.buffer.append(datum) - - # Sort based on configuration - self.buffer.sort(key=lambda d: d.event_time, reverse=not self.sort_ascending) - - # Emit results - for datum in self.buffer: - await output.put(Message(value=datum.value, keys=datum.keys(), tags=["sorted"])) - - # Clean up - self.buffer.clear() - - # Create server with custom configuration - configured_server = AccumulatorAsyncServer( - ConfigurableAccumulator, - init_args=(500, False), # max_buffer_size=500, sort_ascending=False - ) - self.assertIsNotNone(configured_server) - - -if __name__ == "__main__": - unittest.main() From be9c47fea6becaf0f07201ced632754a1857ed35 Mon Sep 17 00:00:00 2001 From: srao12 Date: Wed, 30 Jul 2025 00:50:38 -0700 Subject: [PATCH 29/30] fix: lint Signed-off-by: srao12 --- tests/accumulator/test_datatypes.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/accumulator/test_datatypes.py b/tests/accumulator/test_datatypes.py index 17b2a7a4..a71f3452 100644 --- a/tests/accumulator/test_datatypes.py +++ b/tests/accumulator/test_datatypes.py @@ -1,4 +1,3 @@ -from copy import deepcopy import unittest from collections.abc import AsyncIterable from datetime import datetime, timezone From 597638f572fef787b1b975d46614e910e5941f6f Mon Sep 17 00:00:00 2001 From: srao12 Date: Wed, 30 Jul 2025 01:03:14 -0700 Subject: [PATCH 30/30] fix: add accumulator for e2e tests Signed-off-by: srao12 --- .github/workflows/build-push.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/build-push.yaml b/.github/workflows/build-push.yaml index ab6a122f..a2c1ed8c 100644 --- a/.github/workflows/build-push.yaml +++ b/.github/workflows/build-push.yaml @@ -23,7 +23,7 @@ jobs: "examples/reducestream/counter", "examples/reducestream/sum", "examples/sideinput/simple_sideinput", "examples/sideinput/simple_sideinput/udf", "examples/sink/async_log", "examples/sink/log", "examples/source/simple_source", "examples/sourcetransform/event_time_filter", - "examples/batchmap/flatmap" + "examples/batchmap/flatmap", "examples/accumulator/streamsorter" ] steps: