diff --git a/.gitignore b/.gitignore new file mode 100644 index 00000000..1f6a95a3 --- /dev/null +++ b/.gitignore @@ -0,0 +1,38 @@ +# Compiled class file +*.class + +# Log file +*.log + +# BlueJ files +*.ctxt + +# Mobile Tools for Java (J2ME) +.mtj.tmp/ + +# Package Files # +*.jar +*.war +*.nar +*.ear +*.zip +*.tar.gz +*.rar + +# virtual machine crash logs, see http://www.java.com/en/download/help/error_hotspot.xml +hs_err_pid* + +.DS_Store +*.iml +.idea + +# maven shade plugin +dependency-reduced-pom.xml +target + +# documentation + /_build/ +/venv/ +_build +/dist/html/ +build-linter diff --git a/LICENSE b/LICENSE new file mode 100644 index 00000000..261eeb9e --- /dev/null +++ b/LICENSE @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + 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. diff --git a/README.md b/README.md new file mode 100644 index 00000000..ec3cbe0b --- /dev/null +++ b/README.md @@ -0,0 +1,223 @@ +![Stateful Functions](stateful-functions-docs/images/stateful_functions_logo.png) + +Stateful Functions is a library for distributed applications and services, based on, well, you guessed it: stateful functions. + +The project aims to simplify the development of distributed stateful applications by solving some of the common +challenges in those applications: scaling, consistent state management, reliable interaction between distributed +services, and resource management. + +Stateful Functions uses Apache Flink for distributed coordination, state, and communication. + +This README is meant as a brief walkthrough on the core concepts and how to set things up +to get yourself started with Stateful Functions. For a fully detailed documentation, please +see [https://statefun.io](https://statefun.io). + +## Table of Contents + +- [Core Concepts](#core-concepts) + * [Abstraction](#abstraction) + * [Function modules and extensibility](#modules) +- [Getting Started](#getting-started) + * [Building the project](#build) + * [Running a full example](#greeter) + * [Project setup](#project-setup) + * [Running in the IDE](#ide-harness) +- [Deploying Applications](#deploying) + * [Deploying with a Docker image](#docker) + * [Deploying as a Flink job](#flink) +- [Contributing](#contributing) +- [License](#license) + +## Core Concepts + +### Abstraction + +A Stateful Functions application consists of the following primitives: stateful functions, ingresses, +routers, and egresses. + +#### Stateful functions + +* Stateful functions are the building blocks and namesake of the Stateful Functions framework. +A function is a small piece of logic (currently simple Java functions) that are invoked through a message. + +* Each stateful function exist as uniquely invokable _virtual instances_ of a _function type_. Each instance +is addressed by its type, as well as an unique id (a string) within its type. + +* Stateful functions may be invoked from ingresses or any other stateful function (including itself). +The caller simply needs to know the address of the target function. + +* Function instances are _virtual_, because they are not all active in memory at the same time. +At any point in time, only a small set of functions and their state exists as actual objects. When a +virtual instance receives a message, one of the objects is configured and loaded with the state of that virtual +instance and then processes the message. Similar to virtual memory, the state of many functions might be “swapped out” +at any point in time. + +* Each virtual instance of a function has its own state, which can be accessed in local variables. +That state is private and local to that instance. + +If you know Apache Flink’s DataStream API, you can think of stateful functions a bit like a lightweight +`KeyedProcessFunction`. The function type is process function transformation, while the ID is the key. The difference +is that functions are not assembled in a directed acyclic graph that defines the flow of data (the streaming topology), +but rather send events arbitrarily to all other functions using addresses. + +#### Ingresses and Egresses + +* _Ingresses_ are the way that events initially arrive in a Stateful Functions application. +Ingresses can be message queues, logs, or HTTP servers - anything that produces an event to be +handled by the application. + +* _Routers_ are attached to ingresses to determine which function instance should handle an event initially. + +* _Egresses_ are a way to send events out from the application in a standardized way. +Egresses are optional; it is also possible that no events leave the application and functions sink events or +directly make calls to other applications. + +### Modules and extensibility + +A _module_ is the entry point for adding to a Stateful Functions +application the core building block primitives, i.e. ingresses, egresses, routers, and stateful functions. + +A single application may be a combination of multiple modules, each contributing a part of the whole application. +This allows different parts of the application to be contributed by different modules; for example, +one module may provide ingresses and egresses, while other modules may individually contribute specific parts of the +business logic as stateful functions. This facilitates working in independent teams, but still deploying +into the same larger application. + +This extensibility is achieved by leveraging the [Java Service Loader](https://docs.oracle.com/javase/tutorial/ext/basics/spi.html#the-serviceloader-class). +In this context, each module is essentially a service provider. + +## Getting Started + +Follow the steps here to get started right away with Stateful Functions. + +This guide will walk you through locally building the project, running an existing example, and setup to +start developing and testing your own Stateful Functions application. + +### Building the project + +Prerequisites: + +* Docker +* Maven 3.5.x or above +* Java 8 or above + +Currently, the project does not have any publicly available artifacts or Docker images for use, so you would have to +first build the project yourself before trying it out. + +``` +mvn clean install +``` + +If you want to [deploy your applications using Docker](#docker), you should also build the base Docker image: + +``` +./tools/docker/build-stateful-functions.sh +``` + +### Running a full example + +As a simple demonstration, we will be going through the steps to run the [Greeter example](stateful-functions-examples/stateful-functions-greeter-example). + +Before anything else, make sure that you have locally [built the project as well as the base Stateful Functions Docker image](#build). +Then, follow the next steps to run the example: + +``` +cd stateful-functions-examples/stateful-functions-greeter-example +docker-compose build +docker-compose up +``` + +This example contains a very basic stateful function with a Kafka ingress and a Kafka egress. + +To see the example in action, send some messages to the topic `names`, and see what comes out out of the topic `greetings`: + +``` +KAFKA=$(docker ps -f "name=stateful-functions-greeter-example_kafka-broker_1" --format "{{.ID}}") ; \ +docker exec -it $KAFKA kafka-console-producer.sh \ + --broker-list localhost:9092 \ + --topic names +``` + +``` +KAFKA=$(docker ps -f "name=stateful-functions-greeter-example_kafka-broker_1" --format "{{.ID}}") ; \ +docker exec -it $KAFKA kafka-console-consumer.sh \ + --bootstrap-server localhost:9092 \ + --topic greetings +``` + +### Project setup + +You can quickly get started building Stateful Functions applications using the provided quickstart Maven archetype: + +``` +mvn archetype:generate \ + -DarchetypeGroupId=com.ververica \ + -DarchetypeArtifactId=stateful-functions-quickstart \ + -DarchetypeVersion=1.0-SNAPSHOT +``` + +This allows you to name your newly created project. It will interactively ask you for the groupId, +artifactId, and package name. There will be a new directory with the same name as your artifact id. + +We recommend you import this project into your IDE to develop and test it. +IntelliJ IDEA supports Maven projects out of the box. If you use Eclipse, the `m2e` plugin allows to import +Maven projects. Some Eclipse bundles include that plugin by default, others require you to install it manually. + +### Running from the IDE + +To test out your application, you can directly run it in the IDE without any further packaging or deployments. + +Please see the [Harness example](stateful-functions-examples/stateful-functions-flink-harness-example) on how to do that. + +## Deploying Applications + +Stateful Functions applications can be packaged as either standalone applications or Flink jobs that can be +submitted to a Flink cluster. + +### Deploying with a Docker image + +Below is an example Dockerfile for building an image for an application called `stateful-functions-example`: + +``` +FROM stateful-functions + +RUN mkdir -p /opt/stateful-functions/modules/stateful-functions-example +COPY target/stateful-functions-example*jar /opt/stateful-functions/modules/stateful-functions-example/ +``` + +### Deploying as a Flink job + +If you prefer to package your Stateful Functions application as a Flink job to submit to an existing Flink cluster, +simply include `stateful-functions-flink-distribution` as a dependency to your application. + +``` + + com.ververica + stateful-functions-flink-distribution + 1.0-SNAPSHOT + +``` + +It includes all the runtime dependencies and configures the application's main entry-point. +You do not need to take any action beyond adding the dependency to your Maven pom. + +Bundle the distribution with your application as a fat jar, and then submit it as you normally would +with any other Flink job: + +``` +{$FLINK_DIR}/bin/flink run ./stateful-functions-example.jar +``` + +## Contributing + +If you find these ideas interesting or promising, try Stateful Functions out and get involved! +Check out the example walkthrough or the docs. File an issue if you have an idea how to improve things. + +The project is work-in-progress. We believe we are off to a promising direction, but there is still a +way to go to make all parts of this vision a reality. There are many possible ways to enhance the Stateful +Functions API for different types of applications. Runtime and operations of Stateful Functions +will also evolve with the capabilities of Apache Flink. + +## License + +The code in this repository is under the Apache license. See [license](blob/master/LICENSE). diff --git a/pom.xml b/pom.xml new file mode 100644 index 00000000..37ac068c --- /dev/null +++ b/pom.xml @@ -0,0 +1,148 @@ + + + + + 4.0.0 + + stateful-functions-parent + com.ververica + stateful-functions-parent + 1.0-SNAPSHOT + pom + + + stateful-functions-sdk + stateful-functions-kafka-io + stateful-functions-examples + stateful-functions-flink + stateful-functions-quickstart + stateful-functions-docs + + + + UTF-8 + UTF-8 + 1.8 + 3.1.1 + 1.20.0 + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.8.1 + + 8 + 8 + + + + + + com.github.spotbugs + spotbugs-maven-plugin + ${spotbugs.version} + + + com.github.spotbugs + spotbugs + ${spotbugs.version} + + + + + analyze-compile + verify + + check + + + + + Max + Low + tools/maven/spotbugs-exclude.xml + + + + + + + org.apache.rat + apache-rat-plugin + 0.13 + + + verify + + check + + + + + + + **/.*/** + **/*.prefs + **/*.log + + **/README.md + .github/** + + **/target/** + **/_build/** + stateful-functions-docs/requirements.txt + stateful-functions-docs/runtime.txt + stateful-functions-docs/docs/_templates/** + + **/generated/** + + + + + + + com.diffplug.spotless + spotless-maven-plugin + ${spotless-maven-plugin.version} + + + + 1.7 + + + + + + + + spotless-check + verify + + check + + + + + + + + diff --git a/stateful-functions-docs/Dockerfile b/stateful-functions-docs/Dockerfile new file mode 100644 index 00000000..feaa46b4 --- /dev/null +++ b/stateful-functions-docs/Dockerfile @@ -0,0 +1,25 @@ +# Copyright 2019 Ververica GmbH. +# +# 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. +FROM python:3 + +RUN apt-get update && apt-get -y install texlive-latex-recommended texlive-fonts-recommended texlive-latex-extra latexmk + +COPY requirements.txt /requirements.txt + +RUN set -e; \ + pip install --no-cache-dir -r /requirements.txt + +VOLUME "/build" +EXPOSE 8000 +CMD ["make" "-C", "/build", "html"] diff --git a/stateful-functions-docs/Dockerfile-linter b/stateful-functions-docs/Dockerfile-linter new file mode 100644 index 00000000..d306ae21 --- /dev/null +++ b/stateful-functions-docs/Dockerfile-linter @@ -0,0 +1,23 @@ +# Copyright 2019 Ververica GmbH. +# +# 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. + +FROM alpine:3.8 as docs-linter + +RUN apk add --no-cache \ + python3 \ + nodejs \ + npm + +RUN pip3 install --disable-pip-version-check proselint \ + && npm install --global alex markdown-spellcheck retext-mapbox-standard write-good diff --git a/stateful-functions-docs/Makefile b/stateful-functions-docs/Makefile new file mode 100644 index 00000000..33867769 --- /dev/null +++ b/stateful-functions-docs/Makefile @@ -0,0 +1,168 @@ +# Copyright 2019 Ververica GmbH. +# +# 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. + +# ________________________________________________________ +# _________/\\\___________________________________________ +# ________\/\\\___________________________________________ +# ________\/\\\___________________________________________ +# ________\/\\\______/\\\\\________/\\\\\\\\__/\\\\\\\\\\_ +# ___/\\\\\\\\\____/\\\///\\\____/\\\//////__\/\\\//////__ +# __/\\\////\\\___/\\\__\//\\\__/\\\_________\/\\\\\\\\\\_ +# _\/\\\__\/\\\__\//\\\__/\\\__\//\\\________\////////\\\_ +# _\//\\\\\\\/\\__\///\\\\\/____\///\\\\\\\\__/\\\\\\\\\\_ +# __\///////\//_____\/////________\////////__\//////////__ +# ________________________________________________________ +# +# Makefile for Ververica Platform Docs +# + +BUILDDIR = _build +DOCKER_GROUP = $(or $(shell printenv DOCKER_GROUP), $(GID), 1000) +DOCKER_USER = $(or $(shell printenv DOCKER_USER), $(UID), 1000) +DOCS_FILES = $(shell find . -name \*.rst -print) +SOURCEDIR = docs +SPHINXBUILD = sphinx-build +SPHINXOPTS = -W +SPHINXPROJ = Stateful Functions + + +# Target called by netlify, does some setup tasks specific to netlify +.PHONY: multiversion +multiversion: + sphinx-versioning build "$(SOURCEDIR)" "$(BUILDDIR)/html" + +# Target for building the docs on netlify +.PHONY: netlify +netlify: + ./netlify-build.sh + +# Run sphinx-autobuild to watch and rebuild the docs on changes +.PHONY: autobuild +autobuild: + sphinx-autobuild \ + --host 0.0.0.0 \ + --port 8000 \ + --ignore "*.sw?" \ + --ignore "*.sw??" \ + --ignore "*~" \ + "$(SOURCEDIR)" \ + "$(BUILDDIR)/html" \ + $(SPHINXOPTS) \ + $(0) + +.PHONY: latexpdf +latexpdf: + sphinx-build \ + -M latexpdf \ + "$(SOURCEDIR)" \ + "$(BUILDDIR)/pdf" \ + $(SPHINXOPTS) \ + $(0) + +# Build the base docker image for the docker-% target to use +.PHONY: docker +docker: + docker build -t stateful-functions-docs-builder . + +# Run other make targets in docker... obviously don't call another docker target +.PHONY: docker-% +docker-%: docker + docker \ + run \ + --rm \ + -it \ + -v $(PWD):/build \ + --user $(DOCKER_USER):$(DOCKER_GROUP) \ + -p 8000:8000 \ + stateful-functions-docs-builder \ + make -C /build $* + +# Run sphinx-autobuild in a docker container, much nicer than running locally +.PHONY: auto +auto: docker-autobuild + +.PHONY: html +html: + @$(SPHINXBUILD) -M html "$(SOURCEDIR)" "$(BUILDDIR)" $(SPHINXOPTS) $(O) + +# Legacy build hosted in a docker container, then deployed to kubernetes +.PHONY: dist +dist: docker-html + rsync -a _build/html dist/ + make -C dist + +# Remove/clean the build directory +.PHONY: clean +clean: + rm -rf "$(BUILDDIR)" + +build-linter: Dockerfile-linter + docker build -f Dockerfile-linter -t stateful-functions/stateful-functions-docs-linter . + touch $@ + +.PHONY: docker-linter +docker-linter: build-linter + docker \ + run \ + --rm \ + -it \ + --name docker-linter \ + -v ${PWD}:/run/docs \ + -w /run/docs \ + stateful-functions/stateful-functions-docs-linter:latest \ + $(LINTER) $(or $(LINTER_TARGET), $(DOCS_FILES)) + +.PHONY: linter-alex +linter-alex: + make docker-linter LINTER=alex + +.PHONY: linter-mdspell +linter-mdspell: + make docker-linter LINTER="mdspell --en-us --ignore-numbers --ignore-acronyms --report" + +.PHONY: linter-proselint +linter-proselint: + make docker-linter LINTER=proselint + +.PHONY: linter-retext-mapbox-standard +linter-retext-mapbox-standard: + make docker-linter LINTER=retext-mapbox-standard + +.PHONY: linter-write-good +linter-write-good: + make docker-linter LINTER=write-good + +.PHONY: lint +lint: + make linter-write-good || true + make linter-retext-mapbox-standard || true + make linter-alex || true + make linter-proselint || true + +# This recreates the netlify build environment for local testing and troubleshooting. +# If you're really stumped, try running it with BUILD_ARGS=/bin/bash to get a shell +# and look around the filesystem to see whats up. +.PHONY: netlify-build-local +netlify-build-local: + @mkdir -p _build + @chmod g+w _build + docker \ + run \ + --rm \ + -it \ + -v $(PWD):/opt/buildhome/local_repo \ + -e NETLIFY_REPO_URL=/opt/buildhome/local_repo \ + --user buildbot:$(DOCKER_GROUP) \ + netlify/build \ + build $(or $(BUILD_ARGS), ./netlify-build-local.sh) diff --git a/stateful-functions-docs/README.md b/stateful-functions-docs/README.md new file mode 100644 index 00000000..0e84ca5d --- /dev/null +++ b/stateful-functions-docs/README.md @@ -0,0 +1,18 @@ +Stateful Functions Documentation +------------------------- + +This documentation is using http://www.sphinx-doc.org/ with the https://pypi.org/project/sphinxcontrib-versioning/ plugin. + +Consult the [requirements.txt](requirements.txt) file for the relevant versions of the packages we're using. + +# Build the documentation locally + +Since we're using the [sphinxcontrib-versioning](https://pypi.org/project/sphinxcontrib-versioning/) plugin to build multiple versions of the documentation, building the docs locally will depend on what you want to achieve. + +If you are only concerend with seeing a particular branch/commit that you're working on, then using `make docker-autobuild` will achieve what you want. You can then navigate to http://localhost:8000/index.html to see the resulting pages, and it will auto rebuild the docs on every edit as well. + +If the build fails or you experience any issues, try doing a `make clean` to remove any old build artifacts first. + +# Export the documentation locally as PDF + +`make docker-latexpdf` will export the current brach/commit of the documentation asd PDF. The final documentation will be located under `_build/pdf/latex/ApplicationManager.pdf`. diff --git a/stateful-functions-docs/docs/_static/.gitkeep b/stateful-functions-docs/docs/_static/.gitkeep new file mode 100644 index 00000000..e69de29b diff --git a/stateful-functions-docs/docs/_static/css/customize-theme.css b/stateful-functions-docs/docs/_static/css/customize-theme.css new file mode 100644 index 00000000..ac0d2979 --- /dev/null +++ b/stateful-functions-docs/docs/_static/css/customize-theme.css @@ -0,0 +1,51 @@ +/* Copyright 2019 Ververica GmbH. + + 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. +*/ + +.wy-side-nav-search>a, .wy-side-nav-search .wy-dropdown>a { + display: block; +} + +.wy-nav-top { + background-color: #446D6F; +} + +.wy-side-nav-search>a img.logo, .wy-side-nav-search .wy-dropdown>a img.logo { + width: 100%; + height: auto; +} + +.wy-menu-vertical a:active { + background-color: #446D6F; +} + +.wy-side-nav-search { + background-color: #446D6F; +} + +.wy-nav-content a { + color: #446D6F; +} + +.wy-nav-content a:visited { + color: #7A9E9C; +} + +.wy-nav-content a:hover { + text-decoration: underline; +} + +.scv-banner > a { + color: white ! important; +} \ No newline at end of file diff --git a/stateful-functions-docs/docs/_static/favicon.png b/stateful-functions-docs/docs/_static/favicon.png new file mode 100644 index 00000000..000a1038 Binary files /dev/null and b/stateful-functions-docs/docs/_static/favicon.png differ diff --git a/stateful-functions-docs/docs/_static/images/async_exec.png b/stateful-functions-docs/docs/_static/images/async_exec.png new file mode 100644 index 00000000..c96fa9c3 Binary files /dev/null and b/stateful-functions-docs/docs/_static/images/async_exec.png differ diff --git a/stateful-functions-docs/docs/_static/images/fault_tolerant.png b/stateful-functions-docs/docs/_static/images/fault_tolerant.png new file mode 100644 index 00000000..dd552d0a Binary files /dev/null and b/stateful-functions-docs/docs/_static/images/fault_tolerant.png differ diff --git a/stateful-functions-docs/docs/_static/images/greeter-function.gif b/stateful-functions-docs/docs/_static/images/greeter-function.gif new file mode 100644 index 00000000..c7055ee4 Binary files /dev/null and b/stateful-functions-docs/docs/_static/images/greeter-function.gif differ diff --git a/stateful-functions-docs/docs/_static/images/resource_footprint.png b/stateful-functions-docs/docs/_static/images/resource_footprint.png new file mode 100644 index 00000000..c16e34f2 Binary files /dev/null and b/stateful-functions-docs/docs/_static/images/resource_footprint.png differ diff --git a/stateful-functions-docs/docs/_static/images/state_first.png b/stateful-functions-docs/docs/_static/images/state_first.png new file mode 100644 index 00000000..bede9254 Binary files /dev/null and b/stateful-functions-docs/docs/_static/images/state_first.png differ diff --git a/stateful-functions-docs/docs/_static/images/stateful_functions_overview.png b/stateful-functions-docs/docs/_static/images/stateful_functions_overview.png new file mode 100755 index 00000000..85b4f0d6 Binary files /dev/null and b/stateful-functions-docs/docs/_static/images/stateful_functions_overview.png differ diff --git a/stateful-functions-docs/docs/_static/logo.png b/stateful-functions-docs/docs/_static/logo.png new file mode 100644 index 00000000..b3d80575 Binary files /dev/null and b/stateful-functions-docs/docs/_static/logo.png differ diff --git a/stateful-functions-docs/docs/_templates/breadcrumbs.html b/stateful-functions-docs/docs/_templates/breadcrumbs.html new file mode 100644 index 00000000..ce288078 --- /dev/null +++ b/stateful-functions-docs/docs/_templates/breadcrumbs.html @@ -0,0 +1,22 @@ +{%- extends "sphinx_rtd_theme/breadcrumbs.html" %} + + +{% block breadcrumbs %} {% endblock %} + +{% block breadcrumbs_aside %} + + + + + + + + + + + + + + + +{% endblock %} \ No newline at end of file diff --git a/stateful-functions-docs/docs/_templates/footer.html b/stateful-functions-docs/docs/_templates/footer.html new file mode 100644 index 00000000..6903152e --- /dev/null +++ b/stateful-functions-docs/docs/_templates/footer.html @@ -0,0 +1,9 @@ +{% extends '!footer.html' %} + +{% block extrafooter %} + +

© Copyright 2019, Ververica GmbH.

+

Apache Flink, Flink®, Apache®, the squirrel logo, and the Apache feather logo are either registered trademarks or trademarks of The Apache Software Foundation.

+

Report an issue with this documentation page | Imprint

+ +{% endblock %} \ No newline at end of file diff --git a/stateful-functions-docs/docs/_templates/layout.html b/stateful-functions-docs/docs/_templates/layout.html new file mode 100644 index 00000000..650013a5 --- /dev/null +++ b/stateful-functions-docs/docs/_templates/layout.html @@ -0,0 +1,13 @@ +{% extends '!layout.html' %} + +{%- set favicon = 'favicon.png' %} +{%- set logo = 'logo.png' %} +{%- set theme_logo_only = True %} + +{% block sidebartitle %} + + {{ super() }} + +

A framework for stateful distributed applications by the original creators of Apache Flink®.

+ +{% endblock %} diff --git a/stateful-functions-docs/docs/_templates/searchbox.html b/stateful-functions-docs/docs/_templates/searchbox.html new file mode 100644 index 00000000..e69de29b diff --git a/stateful-functions-docs/docs/api_concepts/index.rst b/stateful-functions-docs/docs/api_concepts/index.rst new file mode 100644 index 00000000..aa71be84 --- /dev/null +++ b/stateful-functions-docs/docs/api_concepts/index.rst @@ -0,0 +1,31 @@ +.. Copyright 2019 Ververica GmbH. + + 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. + +.. _api-concepts: + +############ +API Concepts +############ + +.. toctree:: + :hidden: + + stateful_functions + match_functions + persistence + io_module/index + +Stateful Functions applications are a collection of virtual stateful functions that can send arbitrary messages between each other and external systems. +The execution can happen on a local JVM, or clusters of many machines. +This section walks you through the basic API's you need to be familiar with to get started. diff --git a/stateful-functions-docs/docs/api_concepts/io_module/custom.rst b/stateful-functions-docs/docs/api_concepts/io_module/custom.rst new file mode 100644 index 00000000..0c8a721f --- /dev/null +++ b/stateful-functions-docs/docs/api_concepts/io_module/custom.rst @@ -0,0 +1,86 @@ +.. Copyright 2019 Ververica GmbH. + + 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. + +######################## +User Defined I/O Modules +######################## + +An I/O module provides access to data which is stored in an external system. +If a pre-built I/O module for a particular system does not exist, you can define your own. + +Stateful Functions I/O modules are built on top of {flink} connectors, for details of how to build a custom connector see the official {flink} `documentation `_. + +.. contents:: :local: + +A Two Package Approach +====================== + +Stateful Functions applications are typically modular, containing many modules multiplexed into a single Flink application. +For that reason, I/O modules provide two components, a specification, and an implementation. +That way, multiple modules can depend on the same I/O type while the implementation only needs to be provided once on the classpath. + +Specifications +============== + +Specifications are the user-facing component of an I/O module and only depend on ``stateful-functions-sdk``. +They include an ingress or egress type and spec. + +Ingress and egress types are similar to function types, they provide an namespace and type associated with a class of I/O components. +Specs are what users configure to set properties for a particular instance of an I/O connection. +The only required parameter is the ingress or egress identifier, all other properties will by system specific. + +.. literalinclude:: ../../../src/main/java/com/ververica/statefun/docs/io/custom/MyIngressSpec.java + :language: java + :lines: 16- + +.. literalinclude:: ../../../src/main/java/com/ververica/statefun/docs/io/custom/MyEgressSpec.java + :language: java + :lines: 16- + +Implementations +=============== + +The implementation maps specs to Flink sources and sinks. +They depend on ``stateful-functions-flink-io``, your specifications module, and the underlying Flink connector. + +Source and Sink Providers +""""""""""""""""""""""""" + +Providers take in the ingress and egress specs and return configured Flink sources and sinks. + +.. literalinclude:: ../../../src/main/java/com/ververica/statefun/docs/io/custom/flink/MySourceProvider.java + :language: java + :lines: 16- + +.. literalinclude:: ../../../src/main/java/com/ververica/statefun/docs/io/custom/flink/MySinkProvider.java + :language: java + :lines: 16- + +Flink I/O Module +"""""""""""""""" + +Flink I/O modules are Stateful Functions' top level entry point for accessing Flink connectors. +They define the relationship between ingress and egress types and source and sink providers. +It also provides runtime configurations through the ``globalConfguration`` which is the union of all configurations in the applications ``flink-conf.yaml`` and any command line arguments passed in the form ``--key value``. + +.. literalinclude:: ../../../src/main/java/com/ververica/statefun/docs/io/custom/flink/MyFlinkIoModule.java + :language: java + :lines: 16- + +I/O modules leverage `Java’s Service Provider Interfaces (SPI) `_ for discovery. +This means that every JAR should contain a file ``com.ververica.statefun.flink.io.spi.FlinkIoModule`` in the ``META_INF/services`` resource directory that lists all available modules that it provides. + +.. code-block:: yaml + + com.ververica.statefun.docs.impl.io.MyFlinkIoModule \ No newline at end of file diff --git a/stateful-functions-docs/docs/api_concepts/io_module/index.rst b/stateful-functions-docs/docs/api_concepts/io_module/index.rst new file mode 100644 index 00000000..f273aafa --- /dev/null +++ b/stateful-functions-docs/docs/api_concepts/io_module/index.rst @@ -0,0 +1,100 @@ +.. Copyright 2019 Ververica GmbH. + + 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. + +########## +I/O Module +########## + +.. toctree:: + :hidden: + + kafka + source_sink + custom + +Stateful Functions' I/O modules allow functions to receive and send messages to external systems. +Based on the concept of Ingress (input) and Egress (output) points, and built on top of the {flink} connector ecosystem, I/O modules enable functions to interact with the outside world through the style of message passing. + +.. contents:: :local: + +.. _ingress: + +Ingress +^^^^^^^^ + +An Ingress is an input point where data is consumed from an external system and forwarded to zero or more functions. +An ``IngressIdentifier`` and an ``IngressSpec`` define it. + +An ingress identifier, similar to an function type, uniquely identifies an ingress by specifying its input type, a namespace, and a name. + +.. literalinclude:: ../../../src/main/java/com/ververica/statefun/docs/io/ingress/Identifiers.java + :language: java + :lines: 16- + +The spec defines the details of how to connect to the external system, which is specific to each individual I/O module. +Each identifier-spec pair is bound to the system inside an stateful function module. + +.. literalinclude:: ../../../src/main/java/com/ververica/statefun/docs/io/ingress/ModuleWithIngress.java + :language: java + :lines: 16- + +Router +"""""" + +A router is a stateless operator that takes each record from an ingress and routes it to zero or more functions. + +.. literalinclude:: ../../../src/main/java/com/ververica/statefun/docs/io/ingress/UserRouter.java + :language: java + :lines: 16- + +Routers are bound to the system via a stateful function module. +Unlike other components, an ingress may have any number of routers. + +.. literalinclude:: ../../../src/main/java/com/ververica/statefun/docs/io/ingress/ModuleWithRouter.java + :language: java + :lines: 16- + +.. _egress: + +Egress +^^^^^^ + +Egress is the opposite of ingress; it is a point that takes messages and writes them to external systems. +Each egress is defined using two components, an ``EgressIdentifier`` and an ``EgressSpec``. + +An egress identifier uniquely identifies an egress based on a namespace, name, and producing type. + +.. literalinclude:: ../../../src/main/java/com/ververica/statefun/docs/io/egress/Identifiers.java + :language: java + :lines: 16- + +An egress spec defines the details of how to connect to the external system, the details are specific to each individual I/O module. +Each identifier-spec pair are bound to the system inside a stateful function module. + +.. literalinclude:: ../../../src/main/java/com/ververica/statefun/docs/io/egress/ModuleWithEgress.java + :language: java + :lines: 16- + +Stateful functions may then message an egress the same way they message another function, passing the egress identifier as function type. + +.. literalinclude:: ../../../src/main/java/com/ververica/statefun/docs/io/egress/FnOutputting.java + :language: java + :lines: 16- + +I/O modules leverage `Java’s Service Provider Interfaces (SPI) `_ for discovery. +This means that every JAR should contain a file ``com.ververica.statefun.sdk.spi.StatefulFunctionModule`` in the ``META_INF/services`` resource directory that lists all available modules that it provides. + +.. code-block:: yaml + + com.ververica.statefun.docs.BasicFunctionModule \ No newline at end of file diff --git a/stateful-functions-docs/docs/api_concepts/io_module/kafka.rst b/stateful-functions-docs/docs/api_concepts/io_module/kafka.rst new file mode 100644 index 00000000..9482ba59 --- /dev/null +++ b/stateful-functions-docs/docs/api_concepts/io_module/kafka.rst @@ -0,0 +1,106 @@ +.. Copyright 2019 Ververica GmbH. + + 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. + +############ +Apache Kafka +############ + +Stateful Functions offers an Apache Kafka I/O Module for reading from and writing to Kafka topics. +It is based on Apache Flink's universal `Kafka connector `_ and provides exactly-once processing semantics. + +.. contents:: :local: + +Dependency +=========== + +To use the Kafka I/O Module, please include the following dependency in your pom. + +.. code-block:: xml + + + com.ververica + stateful-functions-kafka-io + {version} + provided + + +Kafka Ingress Builder +===================== + +A ``KafkaIngressBuilder`` declares an ingress spec for consuming from Kafka cluster. + +It accepts the following arguments: + +1) The ingress identifier associated with this ingress +2) The topic name / list of topic names +3) The address of the bootstrap servers +4) A ``KafkaIngressDeserializer`` for deserializing data from Kafka +5) Properties for the Kafka consumer + +.. literalinclude:: ../../../src/main/java/com/ververica/statefun/docs/io/kafka/IngressSpecs.java + :language: java + :lines: 16- + +Please refer to the Kafka `consumer configuration `_ documentation for the full list of available properties. + +Kafka Deserializer +"""""""""""""""""" + +The Kafka ingress needs to know how to turn the binary data in Kafka into Java objects. +The ``KafkaIngressDeserializer`` allows users to specify such a schema. +The ``T deserialize(ConsumerRecord record)`` method gets called for each Kafka message, passing the key, value, and metadata from Kafka. + +.. literalinclude:: ../../../src/main/java/com/ververica/statefun/docs/io/kafka/UserDeserializer.java + :language: java + :lines: 16- + +Kafka Egress Spec +================= + +A ``KafkaEgressBuilder`` declares an egress spec for writing data out to a Kafka cluster. + +It accepts the following arguments: + +1) The egress identifier associated with this egress +2) The address of the bootstrap servers +3) A ``KafkaEgressSerializer`` for serializing data into Kafka +4) The fault tolerance semantic +5) Properties for the Kafka producer + +.. literalinclude:: ../../../src/main/java/com/ververica/statefun/docs/io/kafka/EgressSpecs.java + :language: java + :lines: 16- + +Please refer to the Kafka `producer configuration `_ documentation for the full list of available properties. + +Kafka Egress and Fault Tolerance +"""""""""""""""""""""""""""""""" + +With fault tolerance enabled, the Kafka egress can provide exactly-once delivery guarantees. +You can choose three different modes of operating based through the ``KafkaEgressBuilder``. + +* ``KafkaEgressBuilder#withNoProducerSemantics``: Nothing is guaranteed. Produced records can be lost or duplicated. +* ``KafkaEgressBuilder#withAtLeastOnceProducerSemantics``: Stateful Functions will guarantee that nor records will be lost but they can be duplicated. +* ``KafkaEgressBuilder#withExactlyOnceProducerSemantics``: Stateful Functions uses Kafka transactions to provide exactly-once semantics. + +Kafka Serializer +"""""""""""""""" + +The Kafka egress needs to know how to turn Java objects into binary data. +The ``KafkaEgressSerializer`` allows users to specify such a schema. +The ``ProducerRecord serialize(T out)`` method gets called for each message, allowing users to set a key, value, and other metadata. + +.. literalinclude:: ../../../src/main/java/com/ververica/statefun/docs/io/kafka/UserSerializer.java + :language: java + :lines: 16- diff --git a/stateful-functions-docs/docs/api_concepts/io_module/source_sink.rst b/stateful-functions-docs/docs/api_concepts/io_module/source_sink.rst new file mode 100644 index 00000000..af7f6940 --- /dev/null +++ b/stateful-functions-docs/docs/api_concepts/io_module/source_sink.rst @@ -0,0 +1,52 @@ +.. Copyright 2019 Ververica GmbH. + + 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. + +################ +Flink Connectors +################ + +The source-sink I/O module allows you to plug in existing, or custom, Flink connectors that are not already integrated into a dedicated I/O module. +For details details of how to build a custom connector see the official {flink} `documentation `_. + +Dependency +========== + +To use the Source/Sink I/O Module, please include the following dependency in your pom. + +.. code-block:: xml + + + com.ververica + stateful-functions-flink-io/artifactId> + {version} + provided + + +Source Spec +=========== + +A source function spec creates an ingress from a Flink source function. + +.. literalinclude:: ../../../src/main/java/com/ververica/statefun/docs/io/flink/ModuleWithSourceSpec.java + :language: java + :lines: 16- + +Source Spec +=========== + +A sink function spec creates an ingress from a Flink sink function. + +.. literalinclude:: ../../../src/main/java/com/ververica/statefun/docs/io/flink/ModuleWithSinkSpec.java + :language: java + :lines: 16- diff --git a/stateful-functions-docs/docs/api_concepts/match_functions.rst b/stateful-functions-docs/docs/api_concepts/match_functions.rst new file mode 100644 index 00000000..396984e9 --- /dev/null +++ b/stateful-functions-docs/docs/api_concepts/match_functions.rst @@ -0,0 +1,68 @@ +.. Copyright 2019 Ververica GmbH. + + 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. + +############### +Match Functions +############### + +Stateful functions provide a powerful abstraction for working with events and state, allowing developers to build components that can react to any kind of message. +Commonly, functions only need to handle a known set of message types, and the ``StatefulMatchFunction`` interface provides an opinionated solution to that problem. + +.. contents:: :local: + +Common Patterns +=============== + +Imagine a greeter function that wants to print specialized greeters depending on the type of input. + +.. literalinclude:: ../../src/main/java/com/ververica/statefun/docs/match/FnUserGreeter.java + :language: java + :lines: 16- + +Customers receive one standard message, and employees receive a personalized message depending on whether or not they are managers. +The input is expected to be from a set of known classes. +Certain variants perform some type specific checks and then call the appropriate action. + +Simple Match Function +===================== + +Stateful match functions are an opinionated variant of stateful functions for precisely this pattern. +Developers outline expected types, optional predicates, and well-typed business logic and let the system dispatch each input to the correct action. +Variants are bound inside a ``configure`` method that is executed once the first time an instance is loaded. + +.. literalinclude:: ../../src/main/java/com/ververica/statefun/docs/match/FnMatchGreeter.java + :language: java + :lines: 16- + +Making Your Function Complete +============================= + +Similar to the first example, match functions are partial by default and will throw an ``IllegalStateException`` on any input that does not match any branch. +They can be made complete by providing an ``otherwise`` clause that serves as a catch-all for unmatched input, think of it as a default clause in a Java switch statement. +The ``otherwise`` action takes its message as an untyped ``java.lang.Object``, allowing you to handle any unexpected messages. + +.. literalinclude:: ../../src/main/java/com/ververica/statefun/docs/match/FnMatchGreeterWithCatchAll.java + :language: java + :lines: 16- + :emphasize-lines: 15 + + +Action Resolution Order +======================= + +Match functions will always match actions from most to least specific using the following resolution rules. + +First, find an action that matches the type and predicate. If two predicates will return true for a particular input, the one registered in the binder first wins. +Next, search for an action that matches the type but does not have an associated predicate. +Finally, if a catch-all exists, it will be executed or an ``IllegalStateException`` will be thrown. diff --git a/stateful-functions-docs/docs/api_concepts/persistence.rst b/stateful-functions-docs/docs/api_concepts/persistence.rst new file mode 100644 index 00000000..8e7594aa --- /dev/null +++ b/stateful-functions-docs/docs/api_concepts/persistence.rst @@ -0,0 +1,80 @@ +.. Copyright 2019 Ververica GmbH. + + 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. + +########### +Persistence +########### + +Stateful Functions treats state as a first class citizen and so all stateful functions can easily define state that is automatically made fault tolerant by the runtime. + +.. contents:: :local: + +.. _persisted-value: + +Defining a Persistent Values +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +All stateful functions may contain state by merely defining one or more ``PersistedValue`` fields. +A ``PersistedValue`` is defined by its name and the class of the type that it stores. +The data is always scoped to a specific function type and identifier. +Below is a stateful function that greets users based on the number of times they have been seen. + +.. warning:: + + All ``PersistedValue`` fields must be marked with an ``@Persisted`` annotation or they will not be made fault tolerant by the runtime. + +.. literalinclude:: ../../src/main/java/com/ververica/statefun/docs/FnUserGreeter.java + :language: java + :lines: 16- + +Persisted value comes with the right primitive methods to build powerful stateful applications. +Calling ``PersistedValue#get`` will return the current value of an object stored in state, or ``null`` if nothing is set. +Conversely, ``PersistedValue#set`` will update the value in state and ``PersistedValue#clear`` will delete the value from state. + + +Supported Types +^^^^^^^^^^^^^^^ + +Stateful Functions applications are typically designed to run indefinitely or for long periods of time. +As with all long-running services, the applications need to be updated to adapt to changing requirements. +This goes the same for data schemas that the applications work against; they evolve along with the application. +That is why the system limits the types that can be stored inside a ``PersistedValue`` to all Java primitives and complex types that support well defined schema migration semantics. + +.. note:: + + Schema evolution is supported naturally with protobuf and json, and the project is working on connecting it to Flink’s schema evolution capabilities. + +POJO types +"""""""""" + +Stateful Functions recognizes data types as a POJO type if the following conditions are fulfilled: + +* The class is public and standalone (no non-static inner class) +* The class has a public no-argument constructor +* All non-static, non-transient fields in the class (and all superclasses) are either public (and non-final) or have a public getter- and a setter- method that follows the Java beans naming conventions for getters and setters. + +Apache Avro +""""""""""" + +Stateful Functions can store any Apache Avro class and fully supports evolving schema of Avro type state, as long as the schema change is considered compatible by `Avro’s rules for schema resolution `_. + +Protocol Buffers +"""""""""""""""" + +Stateful Functions can store any `Protocol Buffer `_ class and fully supports schema evolution as long as the schema change is considered compatible by ProtoBuff's rules for schema evolution. + +Json +"""" + +Stateful Functions can store any object that serializes as JSON. diff --git a/stateful-functions-docs/docs/api_concepts/stateful_functions.rst b/stateful-functions-docs/docs/api_concepts/stateful_functions.rst new file mode 100644 index 00000000..074916c2 --- /dev/null +++ b/stateful-functions-docs/docs/api_concepts/stateful_functions.rst @@ -0,0 +1,156 @@ +.. Copyright 2019 Ververica GmbH. + + 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. + +.. _stateful-functions: + +################## +Stateful Functions +################## + +Stateful functions are the building blocks of applications; they are atomic units of isolation, distribution, and persistence. +As objects, they encapsulate the state of a single entity (e.g., a specific use, device, or session) and encode its behavior. +Stateful functions can interact with each other, and external systems, through message passing. + +.. contents:: :local: + +Defining A Stateful Function +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +A stateful function is any class that implements the ``StatefulFunction`` interface. The following is an example of a simple hello world function. + +.. literalinclude:: ../../src/main/java/com/ververica/statefun/docs/FnHelloWorld.java + :language: java + :lines: 16- + +Functions process each incoming message through their ``invoke`` method. +Input's are untyped and passed through the system as a ``java.lang.Object`` so one function can potentially process multiple types of messages. + +The ``Context`` provides metadata about the current message and function, and is how you can call other functions or external systems. +Functions are invoked based on a function type and unique identifier. + +Function Type's and Identifiers +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +In a local environment, the address of an object is the same as a reference to it. +But in a distributed system, objects may be spread across multiple machines and may or may not be active at any given moment. + +In Stateful Functions, function types and identifiers are used to reference specific stateful functions in the system. +A function type is similar to a class in an object-oriented language; it declares what sort of function the address references. +The id is a primary key and scopes the function call to a specific instance of the function type. + +Suppose a Stateful Functions application was tracking metadata about each user account on a website. +The system would contain a user stateful function that accepts and responds to inputs about users and tracks relevant information. +Stateful Functions will create one virtual instance of this stateful function for every user. +Other functions can call the function for any particular user by the user function type and using the current user id as the instance identifier. + +.. literalinclude:: ../../src/main/java/com/ververica/statefun/docs/FnUser.java + :language: java + :emphasize-lines: 10 + :lines: 16- + +.. literalinclude:: ../../src/main/java/com/ververica/statefun/docs/FnCaller.java + :language: java + :emphasize-lines: 14 + :lines: 16- + +Virtual Functions +^^^^^^^^^^^^^^^^^ + +Functions are virtual, which means the system can support an infinite number of active functions while only requiring a static number of physical objects on the JVM heap. +Any function can call any other without ever triggering an allocation. +The system will make it appear as if functions are always available in-memory. + +Stateful Functions applications deploy on Apache Flink's horizontally parallel runtime. +If the user function, seen above, is run on a Flink cluster with a parallelism of 10, then only ten objects will ever be allocated. +Even if the application creates a billion user functions for a billion different users, memory usage will be stable. +Those billion virtual functions will be evenly partitioned and run by the ten underlying objects. +New object creation only occurs the first time a function of that type, regardless of id, is needed. + +Sending Delayed Messages +^^^^^^^^^^^^^^^^^^^^^^^^ + +Functions are able to send messages on a delay so that they will arrive after some duration. +Functions may even send themselves delayed messages that can serve as a callback. +The delayed message is non-blocking so functions will continue to process records between the time a delayed message is sent and received. + +.. literalinclude:: ../../src/main/java/com/ververica/statefun/docs/delay/FnDelayedMessage.java + :language: java + :lines: 16- + +Completing Async Requests +^^^^^^^^^^^^^^^^^^^^^^^^^ + +When interacting with external systems, such as a database or API, one needs to take care that communication delay with the external system does not dominate the application’s total work. +Stateful Functions allows registering a java ``CompletableFuture`` that will resolve to a value at some point in the future. +Future's are registered along with a metadata object that provides additional context about the caller. + +When the future completes, either successfully or exceptionally, the caller function type and id will be invoked with a ``AsyncOperationResult``. +An asynchronous result can complete in one of three states: + +Success +======= + +The asynchronous operation has succeeded, and the produced result can be obtained via ``AsyncOperationResult#value``. + +Failure +======= + +The asynchronous operation has failed, and the cause can be obtained via ``AsyncOperationResult#throwable``. + +Unknown +======= + +The stateful function was restarted, possibly on a different machine, before the ``CompletableFuture`` was completed, therefore it is unknown what is the status of the asynchronous operation. + +.. literalinclude:: ../../src/main/java/com/ververica/statefun/docs/async/EnrichmentFunction.java + :language: java + :lines: 16- + +Function Providers and Dependency Injection +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ +Stateful functions are created across a distributed cluster of nodes. +``StatefulFunctionProvider`` is a factory class for creating a new instance of a stateful function the first time it is activated. + +.. literalinclude:: ../../src/main/java/com/ververica/statefun/docs/CustomProvider.java + :language: java + :lines: 16- + +Providers are called once per type on each parallel worker, not for each id. +If a stateful function requires custom configurations, they can be defined inside a provider and passed to the functions' constructor. +This is also where shared physical resources, such as a database connection, can be created that are used by any number of virtual functions. +Now, tests can quickly provide mock, or test dependencies, without the need for complex dependency injection frameworks. + +.. literalinclude:: ../../src/main/java/com/ververica/statefun/docs/FunctionTest.java + :language: java + :lines: 16- + +.. _module: + +Stateful Function Modules +^^^^^^^^^^^^^^^^^^^^^^^^^ + +Modules define a Stateful Functions application's top-level entry point and are where everything gets tied together. +They offer a single configuration method where stateful functions are bound to the system. +It also provides runtime configurations through the ``globalConfguration`` which is the union of all configurations in the applications ``flink-conf.yaml`` and any command line arguments passed in the form ``--key value``. + +.. literalinclude:: ../../src/main/java/com/ververica/statefun/docs/BasicFunctionModule.java + :language: java + :lines: 16- + +Modules leverage `Java’s Service Provider Interfaces (SPI) `_ for discovery. +This means that every JAR should contain a file ``com.ververica.statefun.sdk.spi.StatefulFunctionModule`` in the ``META_INF/services`` resource directory that lists all available modules that it provides. + +.. code-block:: yaml + + com.ververica.statefun.docs.BasicFunctionModule diff --git a/stateful-functions-docs/docs/conf.py b/stateful-functions-docs/docs/conf.py new file mode 100644 index 00000000..875768ea --- /dev/null +++ b/stateful-functions-docs/docs/conf.py @@ -0,0 +1,242 @@ +# Copyright 2019 Ververica GmbH. +# +# 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. + +# If extensions (or modules to document with autodoc) are in another directory, +# add these directories to sys.path here. If the directory is relative to the +# documentation root, use os.path.abspath to make it absolute, like shown here. +# +# import os +# import sys +# sys.path.insert(0, os.path.abspath('.')) +import re + + +# -- General configuration ------------------------------------------------ + +# If your documentation needs a minimal Sphinx version, state it here. +# +# needs_sphinx = '1.0' + +# Add any Sphinx extension module names here, as strings. They can be +# extensions coming with Sphinx (named 'sphinx.ext.*') or your custom +# ones. +extensions = ['sphinx.ext.ifconfig'] + +# Add any paths that contain templates here, relative to this directory. +templates_path = ['_templates'] + +# The suffix(es) of source filenames. +# You can specify multiple suffix as a list of string: +# +# source_suffix = ['.rst', '.md'] +source_suffix = '.rst' + +# The master toctree document. +master_doc = 'index' + +# Warn about broken references +nitpicky = True + +# General information about the project. +project = u'Stateful Functions' +copyright = u'2019, Ververica GmbH' +author = u'Ververica GmbH' + +# Render it manually in the footer.html +html_show_copyright = False + +# The version info for the project you're documenting, acts as replacement for +# |version| and |release|, also used in various other places throughout the +# built documents. +# +# The short X.Y version. +version = u'1.0-SNAPSHOT' +# The full version, including alpha/beta/rc tags. +release = u'1.0.0' +# The latest Flink version supported by Stateful Functions +flinkVersion = u'1.9' +# The latest full Flink version supported by Stateful Functions +flinkRelease = u'1.9.0' +# The source code repository +repo = "https://github.com/ververica/stateful-functions" + +# The examples directory +example_source = repo + "/tree/master/stateful-functions-examples" + +# hide source link +html_show_sourcelink = False +# required for the search to work +html_copy_source = True + +# The language for content autogenerated by Sphinx. Refer to documentation +# for a list of supported languages. +# +# This is also used if you do content translation via gettext catalogs. +# Usually you set "language" from the command line for these cases. +language = None + +# List of patterns, relative to source directory, that match files and +# directories to ignore when looking for source files. +# This patterns also effect to html_static_path and html_extra_path +exclude_patterns = ['_build', 'Thumbs.db', '.DS_Store', "hidden"] + +# The name of the Pygments (syntax highlighting) style to use. +pygments_style = 'default' + +# If true, `todo` and `todoList` produce output, else they produce nothing. +todo_include_todos = False + + +# -- Options for HTML output ---------------------------------------------- + +# The theme to use for HTML and HTML Help pages. See the documentation for +# a list of builtin themes. +# +html_theme = 'sphinx_rtd_theme' + +theme_prev_next_buttons_location = 'Top' + +# Theme options are theme-specific and customize the look and feel of a theme +# further. For a list of options available for each theme, see the +# documentation. +# +html_theme_options = { + 'titles_only': True, + 'canonical_url': 'https://statefun.io/', + 'display_version': False, + #'prev_next_buttons_location': None +} + +html_show_sphinx = False + + +# Add any paths that contain custom static files (such as style sheets) here, +# relative to this directory. They are copied after the builtin static files, +# so a file named "default.css" will overwrite the builtin "default.css". +html_static_path = ['_static'] + +# Custom sidebar templates, must be a dictionary that maps document names +# to template names. +# +# This is required for the alabaster theme +# refs: http://alabaster.readthedocs.io/en/latest/installation.html#sidebars +html_sidebars = { + '**': [ + 'relations.html', # needs 'show_related': True theme option to display + 'searchbox.html', + ] +} + +# Custom CSS. +html_css_files = [ + 'css/customize-theme.css', +] + +html_context = { + 'css_files': ['_static/css/customize-theme.css'] +} + + +# -- Options for HTMLHelp output ------------------------------------------ + +# Output file base name for HTML help builder. +htmlhelp_basename = 'StatefulFunctionsdoc' + + +# -- Options for LaTeX output --------------------------------------------- + +latex_elements = { + # The paper size ('letterpaper' or 'a4paper'). + # + # 'papersize': 'letterpaper', + + # The font size ('10pt', '11pt' or '12pt'). + # + # 'pointsize': '10pt', + + # Additional stuff for the LaTeX preamble. + # + # 'preamble': '', + + # Latex figure (float) alignment + # + # 'figure_align': 'htbp', +} + +# Grouping the document tree into LaTeX files. List of tuples +# (source start file, target name, title, +# author, documentclass [howto, manual, or own class]). +latex_documents = [ + (master_doc, 'StatefulFunctions.tex', u'Stateful Functions Documentation', + u'Ververica GmbH', 'manual'), +] + + +# -- Options for manual page output --------------------------------------- + +# One entry per manual page. List of tuples +# (source start file, name, description, authors, manual section). +man_pages = [ + (master_doc, 'stateful-functions', u'Stateful Functions Documentation', + [author], 1) +] + + +# -- Options for Texinfo output ------------------------------------------- + +# Grouping the document tree into Texinfo files. List of tuples +# (source start file, target name, title, author, +# dir menu entry, description, category) +texinfo_documents = [ + (master_doc, 'Stateful Functions', u'Stateful Functions Documentation', + author, 'Stateful Functions', 'One line description of project.', + 'Miscellaneous'), +] + + +# -- Settings for sphinxcontrib-versioning -------------------------------- +scv_greatest_tag = True +scv_show_banner = True +scv_banner_greatest_tag = True +scv_sort = ('semver', 'alpha') +scv_whitelist_branches = (re.compile(r'^v\d+\.\d+$'), + re.compile(r'^v\d+\.\d+\.\d+$'), + 'latest') +scv_whitelist_tags = (re.compile(r'^v\d+\.\d+$'), + re.compile(r'^v\d+\.\d+\.\d+$'), + 'latest') + +## Sphynx macros do not work inside of +## code blocks, so we include these custom +## replacements to autofill in versions in +## all places. +def customReplace(app, docname, source): + result = source[0] + for key in app.config.custom_replacements: + result = result.replace(key, app.config.custom_replacements[key]) + source[0] = result + +custom_replacements = { + "{release}" : release, + "{version}" : version, + "{flink}" : u'Apache Flink®', + "{flinkVersion}" : flinkVersion, + "{flinkRelease}" : flinkRelease, + "{repo}" : repo, + "{examples}" : example_source +} + +def setup(app): + app.add_config_value('custom_replacements', {}, True) + app.connect('source-read', customReplace) diff --git a/stateful-functions-docs/docs/contribute.rst b/stateful-functions-docs/docs/contribute.rst new file mode 100644 index 00000000..3db200e2 --- /dev/null +++ b/stateful-functions-docs/docs/contribute.rst @@ -0,0 +1,19 @@ +.. Copyright 2019 Ververica GmbH. + + 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. + +.. _contribute: + +################# +How to Contribute +################# \ No newline at end of file diff --git a/stateful-functions-docs/docs/deployment_operations/configurations.rst b/stateful-functions-docs/docs/deployment_operations/configurations.rst new file mode 100644 index 00000000..81ce9193 --- /dev/null +++ b/stateful-functions-docs/docs/deployment_operations/configurations.rst @@ -0,0 +1,73 @@ +.. Copyright 2019 Ververica GmbH. + + 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. + +############## +Configurations +############## + +Stateful Functions includes a small number of SDK specific configurations. + +Command Line Arguments +^^^^^^^^^^^^^^^^^^^^^^ + +The following may be set as flags in the form ``--key value``. + +stateful-functions.state.checkpointing-interval-ms +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +Description: Flink checkpoint interval in milliseconds, -1 to disable. + +Default: 30 s + +stateful-functions.message.serializer +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +Description: The serializer to use for on the wire messages. + +Default: ``WITH_PROTOBUF_PAYLOADS`` + +Options: ``WITH_PROTOBUF_PAYLOADS``, ``WITH_KRYO_PAYLOADS``, ``WITH_RAW_PAYLOADS``, + +stateful-functions.flink-job-name +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +Description: The name to display in the Flink-UI + +Default: StatefulFunctions + +Flink Configuration Options +^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +These may be set through your job's ``flink-conf.yaml``. + +stateful-functions.feedback.memory.bytes +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +Description: The number of bytes to use for in memory buffering of the feedback channel, before spilling to disk. + +Default: 32 MB + +stateful-functions.state.multiplex-flink-state +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +Description: Use a single MapState to multiplex different function types and persisted values instead of using a ValueState for each combination. + +Default: true + +.. note:: + + When using RocksDB each registered state is backed by a column family. + By default column family's require 2x64 MB of state. + This could prevent loading many functions with a small resource footprint. + diff --git a/stateful-functions-docs/docs/deployment_operations/index.rst b/stateful-functions-docs/docs/deployment_operations/index.rst new file mode 100644 index 00000000..2cf401d4 --- /dev/null +++ b/stateful-functions-docs/docs/deployment_operations/index.rst @@ -0,0 +1,28 @@ +.. Copyright 2019 Ververica GmbH. + + 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. + +######################### +Deployment and Operations +######################### + +.. toctree:: + :hidden: + + packaging + configurations + metrics + +Stateful Functions is a framework built on top of the {flink} runtime, which means it inherits Flink's deployment and operations model, and there are no new concepts you need to learn. +Read through the official `Apache Flink documentation `_ to learn how to run and maintain an application in production. +The following pages outline Stateful Functions' specific configurations and metrics. \ No newline at end of file diff --git a/stateful-functions-docs/docs/deployment_operations/metrics.rst b/stateful-functions-docs/docs/deployment_operations/metrics.rst new file mode 100644 index 00000000..7598d842 --- /dev/null +++ b/stateful-functions-docs/docs/deployment_operations/metrics.rst @@ -0,0 +1,59 @@ +.. Copyright 2019 Ververica GmbH. + + 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. + +####### +Metrics +####### + +Stateful Functions includes a number of SDK specific metrics, scoped on a per function basis, one level below operator scope. + +..in +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ +The number of incoming messages. + +..inRate +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ +The average number of incoming messages per second. + +..out-local +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ +The number of messages sent to a function on the same task slot. + +..out-localRate +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ +The average number of messages sent to a function on the same task slot per second. + +..out-remote +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ +The number of messages sent to a function on another same task slot. + +..out-remoteRate +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ +The average number of messages sent to a function on another task slot per second. + +..out-egress +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ +The number of messages sent to an egress. + +..out-egressRate +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ +The average number of messages sent to an egress per second. + +.writeback.produced +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ +The number of messages read from the write back channel. + +.writeback.produced +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ +The average number of messages read from the write back channel per second. \ No newline at end of file diff --git a/stateful-functions-docs/docs/deployment_operations/packaging.rst b/stateful-functions-docs/docs/deployment_operations/packaging.rst new file mode 100644 index 00000000..b53f48d9 --- /dev/null +++ b/stateful-functions-docs/docs/deployment_operations/packaging.rst @@ -0,0 +1,61 @@ +.. Copyright 2019 Ververica GmbH. + + 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. + +######################## +Packaging For Deployment +######################## + +Stateful Functions applications can be packaged as either standalone applications or Flink jobs that can be submitted to a cluster. + +.. contents:: :local: + +Images +^^^^^^ + +The recommended deployment mode for Stateful Functions applications is to build a Docker image. +This way, user code does not need to package any Apache Flink components. +The provided base image allows teams to package their applications with all the necessary runtime dependencies quickly. + +Below is an example Dockerfile for building a Stateful Functions image for an application called ``stateful-functions-example``. + +.. code-block:: java + + FROM stateful-functions + + RUN mkdir -p /opt/stateful-functions/modules/stateful-functions-example + COPY target/stateful-functions-example*jar /opt/stateful-functions/modules/stateful-functions-example/ + +Flink Jar +^^^^^^^^^ + +If you prefer to package your job to submit to an existing Flink cluster, simply include ``stateful-functions-flink-distribution`` as a dependency to your application. + +.. code-block:: xml + + + com.ververica + stateful-functions-flink-distribution + {version} + + +It includes all of Stateful Functions' runtime dependencies and configure's the applications main entry-point. +You do not need to take any action beyond adding the dependency to your pom. + +.. note:: + + The distribution must be bundled in your application fat jar so that it is on Flink's `user code class loader `_. + +.. code-block:: bash + + ./bin/flink run ./stateful-functions-example.jar \ No newline at end of file diff --git a/stateful-functions-docs/docs/getting_started/index.rst b/stateful-functions-docs/docs/getting_started/index.rst new file mode 100644 index 00000000..7879681c --- /dev/null +++ b/stateful-functions-docs/docs/getting_started/index.rst @@ -0,0 +1,48 @@ +.. Copyright 2019 Ververica GmbH. + + 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. + +############### +Getting Started +############### + +.. toctree:: + :hidden: + + project_setup + walkthrough + +There are many ways to get started with Stateful Functions. +Which one is the best for you depends on your goals and prior experience. +Whether you prefer a more theoretical or a practical approach, we hope you’ll find this section helpful. + +.. contents:: :local: + +Learn By Doing +============== + +If you prefer to learn by doing, start with our code :ref:`walkthrough `. +It provides a step by step introduction to the API and guides you through real applications. + +Learn Concepts Step By Step +=========================== + +If you prefer to learn concepts step by step, start with our guide to :ref:`main concepts `. +It will walk you through all the API's and concepts to build advanced stateful systems. + + +Start A New Project +=================== + +The :ref:`project setup ` instructions show you how to create a project for a new Stateful Functions application in just a few steps. + diff --git a/stateful-functions-docs/docs/getting_started/project_setup.rst b/stateful-functions-docs/docs/getting_started/project_setup.rst new file mode 100644 index 00000000..e12ccdee --- /dev/null +++ b/stateful-functions-docs/docs/getting_started/project_setup.rst @@ -0,0 +1,87 @@ +.. Copyright 2019 Ververica GmbH. + + 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. + +.. _project_setup: + +############# +Project Setup +############# + +You can quickly get started building a Stateful Functions applications by adding the ``stateful-functions-sdk`` to an existing project or using the provided maven archetype. + +.. toctree:: + :hidden: + +Dependency +========== + +.. code-block:: xml + + + com.ververica + stateful-functions-sdk + {version} + provided + + + +Maven Archetype +=============== + +.. code-block:: bash + + $ mvn archetype:generate \ + -DarchetypeGroupId=com.ververica \ + -DarchetypeArtifactId=stateful-functions-quickstart \ + -DarchetypeVersion={version} + +This allows you to name your newly created project. +It will interactively ask you for the groupId, artifactId, and package name. +There will be a new directory with the same name as your artifact id. + +.. code-block:: bash + + $ tree stateful-functions-quickstart/ + stateful-functions-quickstart/ + ├── Dockerfile + ├── pom.xml + └── src + └── main + ├── java + │ └── com + │ └── ververica + │ └── statefun + │ └── Module.java + └── resources + └── META-INF + └── services + └── com.ververica.statefun.sdk.spi.StatefulFunctionModule + +The project contains four files: + +* ``pom.xml``: A pom file with the basic dependencies to start building a Stateful Functions application. +* ``Module``: The entry point for the application. +* ``com.ververica.statefun.sdk.spi.StatefulFunctionModule``: A service entry for the runtime to find the module. +* ``Dockerfile``: A Dockerfile to quickly build a Stateful Functions image ready to deploy. + +We recommend you import this project into your IDE to develop and test it. +IntelliJ IDEA supports Maven projects out of the box. +If you use Eclipse, the m2e plugin allows to import Maven projects. +Some Eclipse bundles include that plugin by default, others require you to install it manually. + +Build Project +============= + +If you want to build/package your project, go to your project directory and run the ``mvn clean package`` command. +You will find a JAR file that contains your application, plus any libraries that you may have added as dependencies to the application: ``target/-.jar``. diff --git a/stateful-functions-docs/docs/getting_started/walkthrough.rst b/stateful-functions-docs/docs/getting_started/walkthrough.rst new file mode 100644 index 00000000..ef28a6b9 --- /dev/null +++ b/stateful-functions-docs/docs/getting_started/walkthrough.rst @@ -0,0 +1,204 @@ +.. Copyright 2019 Ververica GmbH. + + 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. + +.. _walkthrough: + +########### +Walkthrough +########### + +Like all great introductions in software, this walkthrough will start at the beginning: saying hello. +The application will run a simple function that accepts a request and responds with a greeting. +It will not attempt to cover all the complexities of application development, but instead focus on building a stateful function — which is where you will implement your business logic. + +.. contents:: :local: + +A Basic Hello +^^^^^^^^^^^^^ + +Greeting actions are triggered by consuming, routing and passing messages that are defined using ProtoBuf. + +.. code-block:: proto + + syntax = "proto3"; + + message GreetRequest { + string who = 1; + } + + message GreetResponse { + string who = 1; + string greeting = 2; + } + + +Under the hood, messages are processed using :ref:`stateful functions `, by definition any class that implements the ``StatefulFunction`` interface. + +.. code-block:: java + + package com.ververica.statefun.examples.greeter; + + import com.ververica.statefun.sdk.Context; + import com.ververica.statefun.sdk.StatefulFunction; + + public final class GreetFunction implements StatefulFunction { + + @Override + public void invoke(Context context, Object input) { + GreetRequest greetMessage = (GreetRequest) input; + + GreetResponse response = GreetResponse.newBuilder() + .setWho(greetMessage.getWho()) + .setGreeting("Hello " + greetMessage.getWho()) + .build(); + + context.send(GreetingConstants.GREETING_EGRESS_ID, response); + } + } + +This function takes in a request and sends a response to an external system (or :ref:`egress `). +While this is nice, it does not show off the real power of stateful functions: handling state. + +A Stateful Hello +^^^^^^^^^^^^^^^^ + +Suppose you want to generate a personalized response for each user depending on how many times they have sent a request. + +.. code-block:: java + + private static String greetText(String name, int seen) { + switch (seen) { + case 0: + return String.format("Hello %s !", name); + case 1: + return String.format("Hello again %s !", name); + case 2: + return String.format("Third times the charm! %s!", name); + case 3: + return String.format("Happy to see you once again %s !", name); + default: + return String.format("Hello at the %d-th time %s", seen + 1, name); + } + +Routing Messages +================ + +To send a user a personalized greeting, the system needs to keep track of how many times it has seen each user so far. +Speaking in general terms, the simplest solution would be to create one function for every user and independently track the number of times they have been seen. Using most frameworks, this would be prohibitively expensive. +However, stateful functions are virtual and do not consume any CPU or memory when not actively being invoked. +That means your application can create as many functions as necessary — in this case, users — without worrying about resource consumption. + +Whenever data is consumed from an external system (or :ref:`ingress `), it is routed to a specific function based on a given function type and identifier. +The function type represents the Class of function to be invoked, such as the Greeter function, while the identifier (``GreetRequest#getWho``) scopes the call to a specific virtual instance based on some key. + +.. code-block:: java + + package com.ververica.statefun.examples.greeter; + + import com.ververica.statefun.examples.kafka.generated.GreetRequest; + import com.ververica.statefun.sdk.io.Router; + + final class GreetRouter implements Router { + + @Override + public void route(GreetRequest message, Downstream downstream) { + downstream.forward(GreetingConstants.GREETER_FUNCTION_TYPE, message.getWho(), message); + } + } + +So, if a message for a user named John comes in, it will be shipped to John’s dedicated Greeter function. +In case there is a following message for a user named Jane, a new instance of the Greeter function will be spawned. + +Persistence +=========== + +:ref:`Persisted value ` is a special data type that enables stateful functions to maintain fault-tolerant state scoped to their identifiers, so that each instance of a function can track state independently. +To “remember” information across multiple greeting messages, you then need to associate a persisted value field (``count``) to the Greet function. For each user, functions can now track how many times they have been seen. + +.. code-block:: java + + package com.ververica.statefun.examples.greeter; + + import com.ververica.statefun.sdk.Context; + import com.ververica.statefun.sdk.StatefulFunction; + import com.ververica.statefun.sdk.annotations.Persisted; + import com.ververica.statefun.sdk.state.PersistedValue; + + public final class GreetFunction implements StatefulFunction { + + @Persisted + private final PersistedValue count = PersistedValue.of("count", Integer.class); + + @Override + public void invoke(Context context, Object input) { + GreetRequest greetMessage = (GreetRequest) input; + + GreetResponse response = computePersonalizedGreeting(greetMessage); + + context.send(GreetingConstants.GREETING_EGRESS_ID, response); + } + + private GreetResponse computePersonalizedGreeting(GreetRequest greetMessage) { + final String name = greetMessage.getWho(); + final int seen = count.getOrDefault(0); + count.set(seen + 1); + + String greeting = greetText(name, seen); + + return GreetResponse.newBuilder() + .setWho(name) + .setGreeting(greeting) + .build(); + } + } + +Each time a message is processed, the function computes a personalized message for that user. +It reads and updates the number of times that user has been seen and sends a greeting to the egress. + +You can check the full code for the application described in this walkthrough `here <{examples}/stateful-functions-greeter-example>`_. +In particular, take a look at the :ref:`module ` GreetingModule, which is the main entry point for the full application, to see how everything gets tied together. +You can run this example locally using the provided Docker setup. + +.. code-block:: bash + + $ docker-compose build + $ docker-compose up + +Then, send some messages to the topic "names", and observe what comes out of "greetings". + +.. code-block:: bash + + $ KAFKA=$(docker ps -f "name=stateful-functions-greeter-example_kafka-broker_1" --format "{{.ID}}") + $ docker exec -it $KAFKA kafka-console-producer.sh \ + --broker-list localhost:9092 \ + --topic names + +.. code-block:: bash + + $ KAFKA=$(docker ps -f "name=stateful-functions-greeter-example-broker_1" --format "{{.ID}}") + $ docker exec -it $KAFKA kafka-console-consumer.sh \ + --bootstrap-server localhost:9092 \ + --topic greetings + +.. image:: ../_static/images/greeter-function.gif + :align: center + +Want To Go Further? +^^^^^^^^^^^^^^^^^^^ + +This Greeter never forgets a user. +Try and modify the function so that it will reset the ``count`` for any user that spends more than 60 seconds without interacting with the system. + +**Hint:** sending messages with a delay is supported, using ``Context#sendAfter``. +How could you use this to implement a periodic check? diff --git a/stateful-functions-docs/docs/index.rst b/stateful-functions-docs/docs/index.rst new file mode 100644 index 00000000..ea9a6482 --- /dev/null +++ b/stateful-functions-docs/docs/index.rst @@ -0,0 +1,27 @@ +.. Copyright 2019 Ververica GmbH. + + 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. + +.. toctree:: + :hidden: + + overview/index + getting_started/index + api_concepts/index + deployment_operations/index + roadmap + contribute + +############################################ +Lightweight, Stateful Applications at Scale +############################################ \ No newline at end of file diff --git a/stateful-functions-docs/docs/overview/benefits_grid.html b/stateful-functions-docs/docs/overview/benefits_grid.html new file mode 100644 index 00000000..798f1a71 --- /dev/null +++ b/stateful-functions-docs/docs/overview/benefits_grid.html @@ -0,0 +1,76 @@ + + + + + + + +
+ + + + + + + + + +
+ + \ No newline at end of file diff --git a/stateful-functions-docs/docs/overview/consistency_model.rst b/stateful-functions-docs/docs/overview/consistency_model.rst new file mode 100644 index 00000000..a1ec4ca8 --- /dev/null +++ b/stateful-functions-docs/docs/overview/consistency_model.rst @@ -0,0 +1,19 @@ +.. Copyright 2019 Ververica GmbH. + + 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. + +.. _consistency_model: + +################# +Consistency Model +################# diff --git a/stateful-functions-docs/docs/overview/execution_model.rst b/stateful-functions-docs/docs/overview/execution_model.rst new file mode 100644 index 00000000..146ce11d --- /dev/null +++ b/stateful-functions-docs/docs/overview/execution_model.rst @@ -0,0 +1,19 @@ +.. Copyright 2019 Ververica GmbH. + + 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. + +.. _execution_model: + +########################### +Distributed Execution Model +########################### diff --git a/stateful-functions-docs/docs/overview/index.rst b/stateful-functions-docs/docs/overview/index.rst new file mode 100644 index 00000000..b23f54ab --- /dev/null +++ b/stateful-functions-docs/docs/overview/index.rst @@ -0,0 +1,70 @@ +.. Copyright 2019 Ververica GmbH. + + 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. + +######## +Overview +######## + +**Stateful Functions** is a framework for building and orchestrating distributed stateful applications at scale that aims to solve some of the most significant operational challenges for developers: consistent state management, reliable interaction between distributed services and resource management. + +Key Benefits +############ + +.. raw:: html + :file: benefits_grid.html + +.. toctree:: + :hidden: + + stateful_functions + consistency_model + execution_model + tech_space + +Architecture Overview +##################### + +The framework is based on :ref:`functions with persistent state ` that can share a pool of resources and interact arbitrarily with strong consistency guarantees. **Stateful Functions** uses a state-of-the-art runtime built on `Apache Flink `_ for distributed coordination, communication and state management. + +.. topic:: Stateful Functions API + + The API is based on, well, stateful functions: small snippets of functionality that encapsulate business logic, somewhat similar to `actors `_. These functions exist as virtual instances — typically, one per entity in the application (for example, per user or stock item) — and are distributed across shards, making applications **scalable out-of-the-box**. Each function has persistent user-defined state in local variables and can message other functions (including itself!). This model makes **computing with state natural and uncomplicated**. + +.. topic:: Stream Processing Runtime + + The runtime that powers **Stateful Functions** is based on stream processing with Apache Flink. State is kept in the stream processing engine, co-located with the computation, giving you fast and consistent state access. **State durability and fault tolerance** build on Flink’s robust `distributed snapshots model `_. + +.. topic:: Messaging Model + + In **Stateful Functions** applications, everything is inherently strongly consistent: state modifications and messaging are integrated to create the effect of **consistent state and reliable messaging** within all interacting functions. Care about consistency needs to be taken only when interacting with the "outside world”. Event Ingresses and Egresses — optionally with transactional semantics — support interaction with the “outside world” via event streams. + +.. topic:: Consistency Model + + Interactions flow between functions as event streams, in the style of message passing. Apache Flink’s snapshot-based fault tolerance model was extended to support cyclic data flow graphs while ensuring **exactly-once messaging guarantees** (yay!). As a result, you can have functions messaging each other arbitrarily, efficiently, and reliably. + +**Stateful Functions** splits compute and storage differently to the classical two-tier architecture: one ephemeral state/compute tier and a simple persistent blob storage tier. This approach **eliminates the need to provision additional databases, key-value stores or message brokers** and effectively offloads application state management from the shoulders of developers. + +Technology Space +################ + +Stateful Functions is heavily inspired by multiple existing technologies for stateless application development and orchestration. Other than Apache Flink, also Function-as-a-Service (FaaS) systems such as AWS Lambda and the `virtual stateful actor model `_ from Microsoft Orleans served as inspiration for this project. + +The framework is mostly implemented in Java and runs on the JVM. Extending the API to be cross-language compatible and support languages like Python, Go or NodeJS is part of the :ref:`Roadmap `. + +Get Involved! +############# + +If you find these ideas interesting, give **Stateful Functions** a try and get involved! Check out the Getting Started section for introduction walkthroughs. File an issue if you have an idea how to improve things. + +The project is a work-in-progress. We believe we are off to a promising direction, but there is still a way to go to make all parts of this vision a reality. There are many possible ways to enhance **Stateful Functions** for different types of applications. Possibilities for enhancements to the runtime and operations will also evolve with the evolution of capabilities of Apache Flink. \ No newline at end of file diff --git a/stateful-functions-docs/docs/overview/stateful_functions.rst b/stateful-functions-docs/docs/overview/stateful_functions.rst new file mode 100644 index 00000000..d654ff2e --- /dev/null +++ b/stateful-functions-docs/docs/overview/stateful_functions.rst @@ -0,0 +1,19 @@ +.. Copyright 2019 Ververica GmbH. + + 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. + +.. _stateful_functions: + +############################## +Stateful Functions Abstraction +############################## \ No newline at end of file diff --git a/stateful-functions-docs/docs/overview/tech_space.rst b/stateful-functions-docs/docs/overview/tech_space.rst new file mode 100644 index 00000000..f9726cd3 --- /dev/null +++ b/stateful-functions-docs/docs/overview/tech_space.rst @@ -0,0 +1,19 @@ +.. Copyright 2019 Ververica GmbH. + + 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. + +.. _tech_space: + +########################## +How Does It Compare To...? +########################## diff --git a/stateful-functions-docs/docs/roadmap.rst b/stateful-functions-docs/docs/roadmap.rst new file mode 100644 index 00000000..1249b1fe --- /dev/null +++ b/stateful-functions-docs/docs/roadmap.rst @@ -0,0 +1,19 @@ +.. Copyright 2019 Ververica GmbH. + + 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. + +.. _roadmap: + +####################### +Roadmap and Future Work +####################### \ No newline at end of file diff --git a/stateful-functions-docs/images/stateful_functions_logo.png b/stateful-functions-docs/images/stateful_functions_logo.png new file mode 100644 index 00000000..677c4d73 Binary files /dev/null and b/stateful-functions-docs/images/stateful_functions_logo.png differ diff --git a/stateful-functions-docs/pom.xml b/stateful-functions-docs/pom.xml new file mode 100644 index 00000000..addfde4b --- /dev/null +++ b/stateful-functions-docs/pom.xml @@ -0,0 +1,143 @@ + + + + + 4.0.0 + + + stateful-functions-parent + com.ververica + 1.0-SNAPSHOT + + + stateful-functions-docs + stateful-functions-docs + 1.0-SNAPSHOT + pom + + + + com.ververica + stateful-functions-sdk + ${project.version} + provided + + + com.ververica + stateful-functions-flink-io + ${project.version} + provided + + + com.ververica + stateful-functions-kafka-io + ${project.version} + provided + + + com.fasterxml.jackson.core + jackson-databind + 2.9.6 + provided + + + junit + junit + 4.12 + provided + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + 2.12.4 + + true + + + + org.apache.maven.plugins + maven-jar-plugin + 2.6 + + + default-jar + none + + + + + org.apache.maven.plugins + maven-install-plugin + 2.5.2 + + + default-install + none + + + + + org.apache.maven.plugins + maven-deploy-plugin + 2.8.2 + + + default-deploy + none + + + + + org.apache.rat + apache-rat-plugin + 0.13 + + + verify + + check + + + + + + + **/.*/** + **/*.prefs + **/*.log + + **/README.md + .github/** + + **/target/** + **/_build/** + requirements.txt + runtime.txt + /docs/_templates/** + + **/generated/** + + + + + + \ No newline at end of file diff --git a/stateful-functions-docs/requirements.txt b/stateful-functions-docs/requirements.txt new file mode 100644 index 00000000..97882e2b --- /dev/null +++ b/stateful-functions-docs/requirements.txt @@ -0,0 +1,5 @@ +six==1.11.0 +Sphinx==1.7.9 +sphinx-autobuild==0.7.1 +sphinx_rtd_theme==0.4.1 +sphinxcontrib-versioning==2.2.1 diff --git a/stateful-functions-docs/runtime.txt b/stateful-functions-docs/runtime.txt new file mode 100644 index 00000000..475ba515 --- /dev/null +++ b/stateful-functions-docs/runtime.txt @@ -0,0 +1 @@ +3.7 diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/BasicFunctionModule.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/BasicFunctionModule.java new file mode 100644 index 00000000..2ccd9064 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/BasicFunctionModule.java @@ -0,0 +1,33 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs; + +import com.ververica.statefun.sdk.spi.StatefulFunctionModule; +import java.util.Map; + +public class BasicFunctionModule implements StatefulFunctionModule { + + public void configure(Map globalConfiguration, Binder binder) { + + // Declare the user function and bind it to its type + binder.bindFunctionProvider(FnWithDependency.TYPE, new CustomProvider()); + + // Stateful functions that do not require any configuration + // can declare their provider using java 8 lambda syntax + binder.bindFunctionProvider(FnUser.TYPE, unused -> new FnUser()); + } +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/CustomProvider.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/CustomProvider.java new file mode 100644 index 00000000..c7a0cf50 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/CustomProvider.java @@ -0,0 +1,31 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs; + +import com.ververica.statefun.docs.dependency.ProductionDependency; +import com.ververica.statefun.docs.dependency.RuntimeDependency; +import com.ververica.statefun.sdk.FunctionType; +import com.ververica.statefun.sdk.StatefulFunction; +import com.ververica.statefun.sdk.StatefulFunctionProvider; + +public class CustomProvider implements StatefulFunctionProvider { + + public StatefulFunction functionOfType(FunctionType type) { + RuntimeDependency dependency = new ProductionDependency(); + return new FnWithDependency(dependency); + } +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/FnCaller.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/FnCaller.java new file mode 100644 index 00000000..11574081 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/FnCaller.java @@ -0,0 +1,32 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs; + +import com.ververica.statefun.sdk.Context; +import com.ververica.statefun.sdk.FunctionType; +import com.ververica.statefun.sdk.StatefulFunction; + +/** A simple stateful function that sends a message to the user with id "user1" */ +public class FnCaller implements StatefulFunction { + + public static final FunctionType TYPE = new FunctionType("ververica", "caller"); + + @Override + public void invoke(Context context, Object input) { + context.send(FnUser.TYPE, "user1", new MyUserMessage()); + } +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/FnHelloWorld.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/FnHelloWorld.java new file mode 100644 index 00000000..260dbb52 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/FnHelloWorld.java @@ -0,0 +1,28 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs; + +import com.ververica.statefun.sdk.Context; +import com.ververica.statefun.sdk.StatefulFunction; + +public class FnHelloWorld implements StatefulFunction { + + @Override + public void invoke(Context context, Object input) { + System.out.println("Hello " + input.toString()); + } +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/FnUser.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/FnUser.java new file mode 100644 index 00000000..0e79ebad --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/FnUser.java @@ -0,0 +1,33 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs; + +import com.ververica.statefun.sdk.Context; +import com.ververica.statefun.sdk.FunctionType; +import com.ververica.statefun.sdk.StatefulFunction; + +/** A simple function that says hello. */ +public class FnUser implements StatefulFunction { + + public static final FunctionType TYPE = new FunctionType("ververica", "user"); + + @Override + public void invoke(Context context, Object message) { + String userId = context.self().id(); + System.out.println("Hello " + userId); + } +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/FnUserGreeter.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/FnUserGreeter.java new file mode 100644 index 00000000..3e85abbf --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/FnUserGreeter.java @@ -0,0 +1,52 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs; + +import com.ververica.statefun.sdk.Context; +import com.ververica.statefun.sdk.FunctionType; +import com.ververica.statefun.sdk.StatefulFunction; +import com.ververica.statefun.sdk.annotations.Persisted; +import com.ververica.statefun.sdk.state.PersistedValue; + +public class FnUserGreeter implements StatefulFunction { + + public static FunctionType TYPE = new FunctionType("ververica", "greeter"); + + @Persisted + private final PersistedValue count = PersistedValue.of("count", Integer.class); + + public void invoke(Context context, Object input) { + String userId = context.self().id(); + int seen = count.getOrDefault(0); + + switch (seen) { + case 0: + System.out.println(String.format("Hello %s!", userId)); + break; + case 1: + System.out.println("Hello Again!"); + break; + case 2: + System.out.println("Third time is the charm :)"); + break; + default: + System.out.println(String.format("Hello for the %d-th time", seen + 1)); + } + + count.set(seen + 1); + } +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/FnWithDependency.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/FnWithDependency.java new file mode 100644 index 00000000..77f5b51f --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/FnWithDependency.java @@ -0,0 +1,36 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs; + +import com.ververica.statefun.docs.dependency.RuntimeDependency; +import com.ververica.statefun.sdk.Context; +import com.ververica.statefun.sdk.FunctionType; +import com.ververica.statefun.sdk.StatefulFunction; + +public class FnWithDependency implements StatefulFunction { + + static final FunctionType TYPE = new FunctionType("ververica", "fn-with-dep"); + + private final RuntimeDependency dependency; + + public FnWithDependency(RuntimeDependency dependency) { + this.dependency = dependency; + } + + @Override + public void invoke(Context context, Object input) {} +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/FunctionTest.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/FunctionTest.java new file mode 100644 index 00000000..9f614c90 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/FunctionTest.java @@ -0,0 +1,33 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs; + +import com.ververica.statefun.docs.dependency.RuntimeDependency; +import com.ververica.statefun.docs.dependency.TestDependency; +import org.junit.Assert; +import org.junit.Test; + +public class FunctionTest { + + @Test + public void testFunctionWithCustomDependency() { + RuntimeDependency dependency = new TestDependency(); + FnWithDependency function = new FnWithDependency(dependency); + + Assert.assertEquals("It appears math is broken", 1 + 1, 2); + } +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/MyUserMessage.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/MyUserMessage.java new file mode 100644 index 00000000..1466e630 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/MyUserMessage.java @@ -0,0 +1,19 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs; + +class MyUserMessage {} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/async/EnrichmentFunction.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/async/EnrichmentFunction.java new file mode 100644 index 00000000..9f65dd0c --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/async/EnrichmentFunction.java @@ -0,0 +1,58 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.async; + +import com.ververica.statefun.sdk.AsyncOperationResult; +import com.ververica.statefun.sdk.Context; +import com.ververica.statefun.sdk.StatefulFunction; +import java.util.concurrent.CompletableFuture; + +@SuppressWarnings("unchecked") +public class EnrichmentFunction implements StatefulFunction { + + private final QueryService client; + + public EnrichmentFunction(QueryService client) { + this.client = client; + } + + @Override + public void invoke(Context context, Object input) { + if (input instanceof User) { + onUser(context, (User) input); + } else if (input instanceof AsyncOperationResult) { + onAsyncResult((AsyncOperationResult) input); + } + } + + private void onUser(Context context, User user) { + CompletableFuture future = client.getDataAsync(user.getUserId()); + context.registerAsyncOperation(user, future); + } + + private void onAsyncResult(AsyncOperationResult result) { + if (result.successful()) { + User metadata = result.metadata(); + UserEnrichment value = result.value(); + System.out.println(String.format("Successfully completed future: %s %s", metadata, value)); + } else if (result.failure()) { + System.out.println(String.format("Something has gone terribly wrong %s", result.throwable())); + } else { + System.out.println("Not sure what happened, maybe retry"); + } + } +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/async/QueryService.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/async/QueryService.java new file mode 100644 index 00000000..acd536b2 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/async/QueryService.java @@ -0,0 +1,23 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.async; + +import java.util.concurrent.CompletableFuture; + +interface QueryService { + CompletableFuture getDataAsync(String userId); +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/async/User.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/async/User.java new file mode 100644 index 00000000..b7fcde54 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/async/User.java @@ -0,0 +1,23 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.async; + +final class User { + String getUserId() { + return ""; + } +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/async/UserEnrichment.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/async/UserEnrichment.java new file mode 100644 index 00000000..a37beb8a --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/async/UserEnrichment.java @@ -0,0 +1,19 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.async; + +final class UserEnrichment {} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/delay/DelayedMessage.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/delay/DelayedMessage.java new file mode 100644 index 00000000..f2e6af26 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/delay/DelayedMessage.java @@ -0,0 +1,19 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.delay; + +class DelayedMessage {} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/delay/FnDelayedMessage.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/delay/FnDelayedMessage.java new file mode 100644 index 00000000..2f18e7da --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/delay/FnDelayedMessage.java @@ -0,0 +1,36 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.delay; + +import com.ververica.statefun.sdk.Context; +import com.ververica.statefun.sdk.StatefulFunction; +import java.time.Duration; + +public class FnDelayedMessage implements StatefulFunction { + + @Override + public void invoke(Context context, Object input) { + if (input instanceof Message) { + System.out.println("Hello"); + context.sendAfter(Duration.ofMinutes(1), context.self(), new DelayedMessage()); + } + + if (input instanceof DelayedMessage) { + System.out.println("Welcome to the future!"); + } + } +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/delay/Message.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/delay/Message.java new file mode 100644 index 00000000..b0d92311 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/delay/Message.java @@ -0,0 +1,19 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.delay; + +class Message {} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/dependency/ProductionDependency.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/dependency/ProductionDependency.java new file mode 100644 index 00000000..649be2a8 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/dependency/ProductionDependency.java @@ -0,0 +1,19 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.dependency; + +public class ProductionDependency implements RuntimeDependency {} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/dependency/RuntimeDependency.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/dependency/RuntimeDependency.java new file mode 100644 index 00000000..21f5d801 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/dependency/RuntimeDependency.java @@ -0,0 +1,19 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.dependency; + +public interface RuntimeDependency {} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/dependency/TestDependency.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/dependency/TestDependency.java new file mode 100644 index 00000000..8166f937 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/dependency/TestDependency.java @@ -0,0 +1,19 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.dependency; + +public class TestDependency implements RuntimeDependency {} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/MissingImplementationException.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/MissingImplementationException.java new file mode 100644 index 00000000..4d2f71b0 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/MissingImplementationException.java @@ -0,0 +1,24 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.io; + +public class MissingImplementationException extends RuntimeException { + + public MissingImplementationException(String message) { + super(message); + } +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/custom/MyEgressSpec.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/custom/MyEgressSpec.java new file mode 100644 index 00000000..6616550f --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/custom/MyEgressSpec.java @@ -0,0 +1,42 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.io.custom; + +import com.ververica.statefun.sdk.EgressType; +import com.ververica.statefun.sdk.io.EgressIdentifier; +import com.ververica.statefun.sdk.io.EgressSpec; + +public class MyEgressSpec implements EgressSpec { + + public static final EgressType TYPE = new EgressType("ververica", "my-egress"); + + private final EgressIdentifier identifier; + + public MyEgressSpec(EgressIdentifier identifier) { + this.identifier = identifier; + } + + @Override + public EgressType type() { + return TYPE; + } + + @Override + public EgressIdentifier id() { + return identifier; + } +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/custom/MyIngressSpec.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/custom/MyIngressSpec.java new file mode 100644 index 00000000..6911355e --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/custom/MyIngressSpec.java @@ -0,0 +1,42 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.io.custom; + +import com.ververica.statefun.sdk.IngressType; +import com.ververica.statefun.sdk.io.IngressIdentifier; +import com.ververica.statefun.sdk.io.IngressSpec; + +public class MyIngressSpec implements IngressSpec { + + public static final IngressType TYPE = new IngressType("ververica", "my-ingress"); + + private final IngressIdentifier identifier; + + public MyIngressSpec(IngressIdentifier identifier) { + this.identifier = identifier; + } + + @Override + public IngressType type() { + return TYPE; + } + + @Override + public IngressIdentifier id() { + return identifier; + } +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/custom/flink/MyFlinkIoModule.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/custom/flink/MyFlinkIoModule.java new file mode 100644 index 00000000..afe5ed02 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/custom/flink/MyFlinkIoModule.java @@ -0,0 +1,31 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.io.custom.flink; + +import com.ververica.statefun.docs.io.custom.MyEgressSpec; +import com.ververica.statefun.docs.io.custom.MyIngressSpec; +import com.ververica.statefun.flink.io.spi.FlinkIoModule; +import java.util.Map; + +public final class MyFlinkIoModule implements FlinkIoModule { + + @Override + public void configure(Map globalConfiguration, Binder binder) { + binder.bindSourceProvider(MyIngressSpec.TYPE, new MySourceProvider()); + binder.bindSinkProvider(MyEgressSpec.TYPE, new MySinkProvider()); + } +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/custom/flink/MySinkProvider.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/custom/flink/MySinkProvider.java new file mode 100644 index 00000000..b02bfd13 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/custom/flink/MySinkProvider.java @@ -0,0 +1,47 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.io.custom.flink; + +import com.ververica.statefun.docs.io.custom.MyEgressSpec; +import com.ververica.statefun.docs.io.custom.flink.sink.MySinkFunction; +import com.ververica.statefun.flink.io.spi.SinkProvider; +import com.ververica.statefun.sdk.io.EgressSpec; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; + +public class MySinkProvider implements SinkProvider { + + @Override + public SinkFunction forSpec(EgressSpec egressSpec) { + MyEgressSpec spec = asMyEgressSpec(egressSpec); + MySinkFunction sink = new MySinkFunction<>(); + + // configure the sink based on the provided spec + return sink; + } + + private static MyEgressSpec asMyEgressSpec(EgressSpec egressSpec) { + if (egressSpec == null) { + throw new NullPointerException("Unable to translate a NULL spec"); + } + + if (egressSpec instanceof MyEgressSpec) { + return (MyEgressSpec) egressSpec; + } + + throw new IllegalArgumentException(String.format("Wrong type %s", egressSpec.type())); + } +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/custom/flink/MySourceProvider.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/custom/flink/MySourceProvider.java new file mode 100644 index 00000000..56cf65a1 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/custom/flink/MySourceProvider.java @@ -0,0 +1,47 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.io.custom.flink; + +import com.ververica.statefun.docs.io.custom.MyIngressSpec; +import com.ververica.statefun.docs.io.custom.flink.source.MySourceFunction; +import com.ververica.statefun.flink.io.spi.SourceProvider; +import com.ververica.statefun.sdk.io.IngressSpec; +import org.apache.flink.streaming.api.functions.source.SourceFunction; + +public class MySourceProvider implements SourceProvider { + + @Override + public SourceFunction forSpec(IngressSpec ingressSpec) { + MyIngressSpec spec = asMyIngressSpec(ingressSpec); + MySourceFunction source = new MySourceFunction(); + + // configure the source based on the provided spec + return source; + } + + private static MyIngressSpec asMyIngressSpec(IngressSpec ingressSpec) { + if (ingressSpec == null) { + throw new NullPointerException("Unable to translate a NULL spec"); + } + + if (ingressSpec instanceof MyIngressSpec) { + return (MyIngressSpec) ingressSpec; + } + + throw new IllegalArgumentException(String.format("Wrong type %s", ingressSpec.type())); + } +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/custom/flink/sink/MySinkFunction.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/custom/flink/sink/MySinkFunction.java new file mode 100644 index 00000000..53f2e9c1 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/custom/flink/sink/MySinkFunction.java @@ -0,0 +1,25 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.io.custom.flink.sink; + +import org.apache.flink.streaming.api.functions.sink.SinkFunction; + +public class MySinkFunction implements SinkFunction { + + @Override + public void invoke(T value, Context context) {} +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/custom/flink/source/MySourceFunction.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/custom/flink/source/MySourceFunction.java new file mode 100644 index 00000000..fd1e4292 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/custom/flink/source/MySourceFunction.java @@ -0,0 +1,28 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.io.custom.flink.source; + +import org.apache.flink.streaming.api.functions.source.SourceFunction; + +public class MySourceFunction implements SourceFunction { + + @Override + public void run(SourceContext ctx) throws Exception {} + + @Override + public void cancel() {} +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/egress/FnOutputting.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/egress/FnOutputting.java new file mode 100644 index 00000000..1dd5f61c --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/egress/FnOutputting.java @@ -0,0 +1,30 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.io.egress; + +import com.ververica.statefun.docs.models.User; +import com.ververica.statefun.sdk.Context; +import com.ververica.statefun.sdk.StatefulFunction; + +/** A simple function that outputs messages to an egress. */ +public class FnOutputting implements StatefulFunction { + + @Override + public void invoke(Context context, Object input) { + context.send(Identifiers.EGRESS, new User()); + } +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/egress/Identifiers.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/egress/Identifiers.java new file mode 100644 index 00000000..d744b835 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/egress/Identifiers.java @@ -0,0 +1,26 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.io.egress; + +import com.ververica.statefun.docs.models.User; +import com.ververica.statefun.sdk.io.EgressIdentifier; + +public final class Identifiers { + + public static final EgressIdentifier EGRESS = + new EgressIdentifier<>("ververica", "egress", User.class); +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/egress/ModuleWithEgress.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/egress/ModuleWithEgress.java new file mode 100644 index 00000000..88264c45 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/egress/ModuleWithEgress.java @@ -0,0 +1,37 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.io.egress; + +import com.ververica.statefun.docs.io.MissingImplementationException; +import com.ververica.statefun.docs.models.User; +import com.ververica.statefun.sdk.io.EgressIdentifier; +import com.ververica.statefun.sdk.io.EgressSpec; +import com.ververica.statefun.sdk.spi.StatefulFunctionModule; +import java.util.Map; + +public class ModuleWithEgress implements StatefulFunctionModule { + + @Override + public void configure(Map globalConfiguration, Binder binder) { + EgressSpec spec = createEgress(Identifiers.EGRESS); + binder.bindEgress(spec); + } + + public EgressSpec createEgress(EgressIdentifier identifier) { + throw new MissingImplementationException("Replace with your specific egress"); + } +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/flink/FlinkSink.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/flink/FlinkSink.java new file mode 100644 index 00000000..ef61ebcb --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/flink/FlinkSink.java @@ -0,0 +1,21 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.io.flink; + +import org.apache.flink.streaming.api.functions.sink.SinkFunction; + +public class FlinkSink implements SinkFunction {} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/flink/FlinkSource.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/flink/FlinkSource.java new file mode 100644 index 00000000..0e34e3b7 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/flink/FlinkSource.java @@ -0,0 +1,27 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.io.flink; + +import org.apache.flink.streaming.api.functions.source.SourceFunction; + +public class FlinkSource implements SourceFunction { + @Override + public void run(SourceContext ctx) throws Exception {} + + @Override + public void cancel() {} +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/flink/ModuleWithSinkSpec.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/flink/ModuleWithSinkSpec.java new file mode 100644 index 00000000..e0cdba52 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/flink/ModuleWithSinkSpec.java @@ -0,0 +1,34 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.io.flink; + +import com.ververica.statefun.docs.models.User; +import com.ververica.statefun.flink.io.datastream.SinkFunctionSpec; +import com.ververica.statefun.sdk.io.EgressIdentifier; +import com.ververica.statefun.sdk.io.EgressSpec; +import com.ververica.statefun.sdk.spi.StatefulFunctionModule; +import java.util.Map; + +public class ModuleWithSinkSpec implements StatefulFunctionModule { + + @Override + public void configure(Map globalConfiguration, Binder binder) { + EgressIdentifier id = new EgressIdentifier<>("ververica", "user", User.class); + EgressSpec spec = new SinkFunctionSpec<>(id, new FlinkSink<>()); + binder.bindEgress(spec); + } +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/flink/ModuleWithSourceSpec.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/flink/ModuleWithSourceSpec.java new file mode 100644 index 00000000..60ab7cdc --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/flink/ModuleWithSourceSpec.java @@ -0,0 +1,34 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.io.flink; + +import com.ververica.statefun.docs.models.User; +import com.ververica.statefun.flink.io.datastream.SourceFunctionSpec; +import com.ververica.statefun.sdk.io.IngressIdentifier; +import com.ververica.statefun.sdk.io.IngressSpec; +import com.ververica.statefun.sdk.spi.StatefulFunctionModule; +import java.util.Map; + +public class ModuleWithSourceSpec implements StatefulFunctionModule { + + @Override + public void configure(Map globalConfiguration, Binder binder) { + IngressIdentifier id = new IngressIdentifier<>(User.class, "ververica", "users"); + IngressSpec spec = new SourceFunctionSpec<>(id, new FlinkSource<>()); + binder.bindIngress(spec); + } +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/ingress/Identifiers.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/ingress/Identifiers.java new file mode 100644 index 00000000..8b921267 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/ingress/Identifiers.java @@ -0,0 +1,26 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.io.ingress; + +import com.ververica.statefun.docs.models.User; +import com.ververica.statefun.sdk.io.IngressIdentifier; + +public final class Identifiers { + + public static final IngressIdentifier INGRESS = + new IngressIdentifier<>(User.class, "ververica", "user-ingress"); +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/ingress/ModuleWithIngress.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/ingress/ModuleWithIngress.java new file mode 100644 index 00000000..2a7d2502 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/ingress/ModuleWithIngress.java @@ -0,0 +1,37 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.io.ingress; + +import com.ververica.statefun.docs.io.MissingImplementationException; +import com.ververica.statefun.docs.models.User; +import com.ververica.statefun.sdk.io.IngressIdentifier; +import com.ververica.statefun.sdk.io.IngressSpec; +import com.ververica.statefun.sdk.spi.StatefulFunctionModule; +import java.util.Map; + +public class ModuleWithIngress implements StatefulFunctionModule { + + @Override + public void configure(Map globalConfiguration, Binder binder) { + IngressSpec spec = createIngress(Identifiers.INGRESS); + binder.bindIngress(spec); + } + + private IngressSpec createIngress(IngressIdentifier identifier) { + throw new MissingImplementationException("Replace with your specific ingress"); + } +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/ingress/ModuleWithRouter.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/ingress/ModuleWithRouter.java new file mode 100644 index 00000000..e34bd620 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/ingress/ModuleWithRouter.java @@ -0,0 +1,40 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.io.ingress; + +import com.ververica.statefun.docs.models.User; +import com.ververica.statefun.sdk.io.IngressIdentifier; +import com.ververica.statefun.sdk.io.IngressSpec; +import com.ververica.statefun.sdk.io.Router; +import com.ververica.statefun.sdk.spi.StatefulFunctionModule; +import java.util.Map; + +public class ModuleWithRouter implements StatefulFunctionModule { + + @Override + public void configure(Map globalConfiguration, Binder binder) { + IngressSpec spec = createIngressSpec(Identifiers.INGRESS); + Router router = new UserRouter(); + + binder.bindIngress(spec); + binder.bindIngressRouter(Identifiers.INGRESS, router); + } + + private IngressSpec createIngressSpec(IngressIdentifier identifier) { + throw new MissingImplementationException("Replace with your specific ingress"); + } +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/ingress/UserRouter.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/ingress/UserRouter.java new file mode 100644 index 00000000..b155a8d3 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/ingress/UserRouter.java @@ -0,0 +1,29 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.io.ingress; + +import com.ververica.statefun.docs.FnUser; +import com.ververica.statefun.docs.models.User; +import com.ververica.statefun.sdk.io.Router; + +public class UserRouter implements Router { + + @Override + public void route(User message, Downstream downstream) { + downstream.forward(FnUser.TYPE, message.getUserId(), message); + } +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/kafka/EgressSpecs.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/kafka/EgressSpecs.java new file mode 100644 index 00000000..95f5af92 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/kafka/EgressSpecs.java @@ -0,0 +1,34 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.io.kafka; + +import com.ververica.statefun.docs.models.User; +import com.ververica.statefun.sdk.io.EgressIdentifier; +import com.ververica.statefun.sdk.io.EgressSpec; +import com.ververica.statefun.sdk.kafka.KafkaEgressBuilder; + +public class EgressSpecs { + + public static final EgressIdentifier ID = + new EgressIdentifier<>("ververica", "output-egress", User.class); + + public static final EgressSpec kafkaEgress = + KafkaEgressBuilder.forIdentifier(ID) + .withKafkaAddress("localhost:9092") + .withSerializer(UserSerializer.class) + .build(); +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/kafka/IngressSpecs.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/kafka/IngressSpecs.java new file mode 100644 index 00000000..49a4b7c9 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/kafka/IngressSpecs.java @@ -0,0 +1,37 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.io.kafka; + +import com.ververica.statefun.docs.models.User; +import com.ververica.statefun.sdk.io.IngressIdentifier; +import com.ververica.statefun.sdk.io.IngressSpec; +import com.ververica.statefun.sdk.kafka.KafkaIngressBuilder; +import org.apache.kafka.clients.consumer.ConsumerConfig; + +public class IngressSpecs { + + public static final IngressIdentifier ID = + new IngressIdentifier<>(User.class, "ververica", "input-ingress"); + + public static final IngressSpec kafkaIngress = + KafkaIngressBuilder.forIdentifier(ID) + .withKafkaAddress("localhost:9092") + .withTopic("my-topic") + .withDeserializer(UserDeserializer.class) + .withProperty(ConsumerConfig.GROUP_ID_CONFIG, "greetings") + .build(); +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/kafka/UserDeserializer.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/kafka/UserDeserializer.java new file mode 100644 index 00000000..f935d39c --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/kafka/UserDeserializer.java @@ -0,0 +1,42 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.io.kafka; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.ververica.statefun.docs.models.User; +import com.ververica.statefun.sdk.kafka.KafkaIngressDeserializer; +import java.io.IOException; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class UserDeserializer implements KafkaIngressDeserializer { + + private static Logger LOG = LoggerFactory.getLogger(UserDeserializer.class); + + private final ObjectMapper mapper = new ObjectMapper(); + + @Override + public User deserialize(ConsumerRecord input) { + try { + return mapper.readValue(input.value(), User.class); + } catch (IOException e) { + LOG.debug("Failed to deserialize record", e); + return null; + } + } +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/kafka/UserSerializer.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/kafka/UserSerializer.java new file mode 100644 index 00000000..df224b14 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/io/kafka/UserSerializer.java @@ -0,0 +1,47 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.io.kafka; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.ververica.statefun.docs.models.User; +import com.ververica.statefun.sdk.kafka.KafkaEgressSerializer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class UserSerializer implements KafkaEgressSerializer { + + private static final Logger LOG = LoggerFactory.getLogger(UserSerializer.class); + + private static final String TOPIC = "user-topic"; + + private final ObjectMapper mapper = new ObjectMapper(); + + @Override + public ProducerRecord serialize(User user) { + try { + byte[] key = user.getUserId().getBytes(); + byte[] value = mapper.writeValueAsBytes(user); + + return new ProducerRecord<>(TOPIC, key, value); + } catch (JsonProcessingException e) { + LOG.info("Failed to serializer user", e); + return null; + } + } +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/match/Customer.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/match/Customer.java new file mode 100644 index 00000000..fd63c5f5 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/match/Customer.java @@ -0,0 +1,24 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.match; + +class Customer { + + String getName() { + return ""; + } +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/match/Employee.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/match/Employee.java new file mode 100644 index 00000000..e594a682 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/match/Employee.java @@ -0,0 +1,28 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.match; + +class Employee { + + String getEmployeeId() { + return ""; + } + + boolean isManager() { + return true; + } +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/match/FnMatchGreeter.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/match/FnMatchGreeter.java new file mode 100644 index 00000000..3e1d3c82 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/match/FnMatchGreeter.java @@ -0,0 +1,44 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.match; + +import com.ververica.statefun.sdk.Context; +import com.ververica.statefun.sdk.match.MatchBinder; +import com.ververica.statefun.sdk.match.StatefulMatchFunction; + +public class FnMatchGreeter extends StatefulMatchFunction { + + @Override + public void configure(MatchBinder binder) { + binder + .predicate(Customer.class, this::greetCustomer) + .predicate(Employee.class, Employee::isManager, this::greetManager) + .predicate(Employee.class, this::greetEmployee); + } + + private void greetManager(Context context, Employee message) { + System.out.println("Hello manager " + message.getEmployeeId()); + } + + private void greetEmployee(Context context, Employee message) { + System.out.println("Hello employee " + message.getEmployeeId()); + } + + private void greetCustomer(Context context, Customer message) { + System.out.println("Hello customer " + message.getName()); + } +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/match/FnMatchGreeterWithCatchAll.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/match/FnMatchGreeterWithCatchAll.java new file mode 100644 index 00000000..d6d66f16 --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/match/FnMatchGreeterWithCatchAll.java @@ -0,0 +1,49 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.match; + +import com.ververica.statefun.sdk.Context; +import com.ververica.statefun.sdk.match.MatchBinder; +import com.ververica.statefun.sdk.match.StatefulMatchFunction; + +public class FnMatchGreeterWithCatchAll extends StatefulMatchFunction { + + @Override + public void configure(MatchBinder binder) { + binder + .predicate(Customer.class, this::greetCustomer) + .predicate(Employee.class, Employee::isManager, this::greetManager) + .predicate(Employee.class, this::greetEmployee) + .otherwise(this::catchAll); + } + + private void catchAll(Context context, Object message) { + System.out.println("Hello unexpected message"); + } + + private void greetManager(Context context, Employee message) { + System.out.println("Hello manager"); + } + + private void greetEmployee(Context context, Employee message) { + System.out.println("Hello employee"); + } + + private void greetCustomer(Context context, Customer message) { + System.out.println("Hello customer"); + } +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/match/FnUserGreeter.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/match/FnUserGreeter.java new file mode 100644 index 00000000..2cf2466b --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/match/FnUserGreeter.java @@ -0,0 +1,53 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.match; + +import com.ververica.statefun.sdk.Context; +import com.ververica.statefun.sdk.StatefulFunction; + +public class FnUserGreeter implements StatefulFunction { + + @Override + public void invoke(Context context, Object input) { + if (input instanceof Employee) { + Employee message = (Employee) input; + + if (message.isManager()) { + greetManager(context, message); + } else { + greetEmployee(context, message); + } + } else if (input instanceof Customer) { + Customer message = (Customer) input; + greetCustomer(context, message); + } else { + throw new IllegalStateException("Unknown message type " + input.getClass()); + } + } + + private void greetManager(Context context, Employee message) { + System.out.println("Hello manager " + message.getEmployeeId()); + } + + private void greetEmployee(Context context, Employee message) { + System.out.println("Hello employee " + message.getEmployeeId()); + } + + private void greetCustomer(Context context, Customer message) { + System.out.println("Hello customer " + message.getName()); + } +} diff --git a/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/models/User.java b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/models/User.java new file mode 100644 index 00000000..6e69887e --- /dev/null +++ b/stateful-functions-docs/src/main/java/com/ververica/statefun/docs/models/User.java @@ -0,0 +1,51 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.docs.models; + +/** A dummy message type. */ +public class User { + + private String userId; + + private String name; + + private long birthday; + + public String getUserId() { + return userId; + } + + public void setUserId(String userId) { + this.userId = userId; + } + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public long getBirthday() { + return birthday; + } + + public void setBirthday(long birthday) { + this.birthday = birthday; + } +} diff --git a/stateful-functions-examples/pom.xml b/stateful-functions-examples/pom.xml new file mode 100644 index 00000000..2cfb470f --- /dev/null +++ b/stateful-functions-examples/pom.xml @@ -0,0 +1,42 @@ + + + + + 4.0.0 + + + stateful-functions-parent + com.ververica + 1.0-SNAPSHOT + + + stateful-functions-examples + stateful-functions-examples + 1.0-SNAPSHOT + pom + + + stateful-functions-greeter-example + stateful-functions-ridesharing-example + stateful-functions-flink-harness-example + stateful-functions-shopping-cart-example + stateful-functions-async-example + + + + diff --git a/stateful-functions-examples/stateful-functions-async-example/pom.xml b/stateful-functions-examples/stateful-functions-async-example/pom.xml new file mode 100644 index 00000000..2d782d5a --- /dev/null +++ b/stateful-functions-examples/stateful-functions-async-example/pom.xml @@ -0,0 +1,36 @@ + + + + + stateful-functions-examples + com.ververica + 1.0-SNAPSHOT + .. + + 4.0.0 + + stateful-functions-async-example + + + com.ververica + stateful-functions-flink-harness + ${project.version} + + + \ No newline at end of file diff --git a/stateful-functions-examples/stateful-functions-async-example/src/main/java/com/ververica/statefun/examples/async/Constants.java b/stateful-functions-examples/stateful-functions-async-example/src/main/java/com/ververica/statefun/examples/async/Constants.java new file mode 100644 index 00000000..5691affb --- /dev/null +++ b/stateful-functions-examples/stateful-functions-async-example/src/main/java/com/ververica/statefun/examples/async/Constants.java @@ -0,0 +1,33 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.async; + +import com.ververica.statefun.examples.async.events.TaskCompletionEvent; +import com.ververica.statefun.examples.async.events.TaskStartedEvent; +import com.ververica.statefun.sdk.io.EgressIdentifier; +import com.ververica.statefun.sdk.io.IngressIdentifier; + +final class Constants { + + static final IngressIdentifier REQUEST_INGRESS = + new IngressIdentifier<>( + TaskStartedEvent.class, "com.ververica.statefun.examples.async", "tasks"); + + static final EgressIdentifier RESULT_EGRESS = + new EgressIdentifier<>( + "com.ververica.statefun.examples.async", "out", TaskCompletionEvent.class); +} diff --git a/stateful-functions-examples/stateful-functions-async-example/src/main/java/com/ververica/statefun/examples/async/Main.java b/stateful-functions-examples/stateful-functions-async-example/src/main/java/com/ververica/statefun/examples/async/Main.java new file mode 100644 index 00000000..a71de06f --- /dev/null +++ b/stateful-functions-examples/stateful-functions-async-example/src/main/java/com/ververica/statefun/examples/async/Main.java @@ -0,0 +1,77 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.async; + +import com.ververica.statefun.examples.async.events.TaskCompletionEvent; +import com.ververica.statefun.examples.async.events.TaskStartedEvent; +import com.ververica.statefun.flink.harness.Harness; +import com.ververica.statefun.flink.harness.io.SerializableConsumer; +import com.ververica.statefun.flink.harness.io.SerializableSupplier; +import java.util.concurrent.ThreadLocalRandom; +import javax.annotation.Nonnull; +import org.apache.flink.util.StringUtils; + +public class Main { + + public static void main(String[] args) throws Exception { + Harness harness = + new Harness() + .noCheckpointing() + .withKryoMessageSerializer() + .withSupplyingIngress(Constants.REQUEST_INGRESS, new MessageGenerator()) + .withConsumingEgress(Constants.RESULT_EGRESS, new MessagePrinter()); + + harness.start(); + } + + /** generate a random message, once a second a second. */ + private static final class MessageGenerator implements SerializableSupplier { + + private static final long serialVersionUID = 1; + + @Override + public TaskStartedEvent get() { + try { + Thread.sleep(1_000); + } catch (InterruptedException e) { + throw new RuntimeException("Interrupted", e); + } + return randomMessage(); + } + + @Nonnull + private TaskStartedEvent randomMessage() { + final ThreadLocalRandom random = ThreadLocalRandom.current(); + final String taskId = StringUtils.generateRandomAlphanumericString(random, 2); + return new TaskStartedEvent(taskId, System.currentTimeMillis()); + } + } + + /** prints the messages to stdout. */ + private static final class MessagePrinter implements SerializableConsumer { + + private static final long serialVersionUID = 1; + + @Override + public void accept(TaskCompletionEvent message) { + System.out.println( + String.format( + "Task %s has completed, total duration: %d ms", + message.getTaskId(), message.getEndTime() - message.getStartTime())); + } + } +} diff --git a/stateful-functions-examples/stateful-functions-async-example/src/main/java/com/ververica/statefun/examples/async/Module.java b/stateful-functions-examples/stateful-functions-async-example/src/main/java/com/ververica/statefun/examples/async/Module.java new file mode 100644 index 00000000..2ef6fa2f --- /dev/null +++ b/stateful-functions-examples/stateful-functions-async-example/src/main/java/com/ververica/statefun/examples/async/Module.java @@ -0,0 +1,38 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.async; + +import com.ververica.statefun.examples.async.service.DummyTaskQueryService; +import com.ververica.statefun.examples.async.service.TaskQueryService; +import com.ververica.statefun.sdk.spi.StatefulFunctionModule; +import java.util.Map; + +public class Module implements StatefulFunctionModule { + + @Override + public void configure(Map globalConfiguration, Binder binder) { + TaskQueryService service = new DummyTaskQueryService(); + + binder.bindFunctionProvider( + TaskDurationTrackerFunction.TYPE, unused -> new TaskDurationTrackerFunction(service)); + + binder.bindIngressRouter( + Constants.REQUEST_INGRESS, + (message, downstream) -> + downstream.forward(TaskDurationTrackerFunction.TYPE, message.getTaskId(), message)); + } +} diff --git a/stateful-functions-examples/stateful-functions-async-example/src/main/java/com/ververica/statefun/examples/async/TaskDurationTrackerFunction.java b/stateful-functions-examples/stateful-functions-async-example/src/main/java/com/ververica/statefun/examples/async/TaskDurationTrackerFunction.java new file mode 100644 index 00000000..0eed50dd --- /dev/null +++ b/stateful-functions-examples/stateful-functions-async-example/src/main/java/com/ververica/statefun/examples/async/TaskDurationTrackerFunction.java @@ -0,0 +1,138 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.async; + +import com.ververica.statefun.examples.async.events.TaskCompletionEvent; +import com.ververica.statefun.examples.async.events.TaskStartedEvent; +import com.ververica.statefun.examples.async.service.TaskQueryService; +import com.ververica.statefun.examples.async.service.TaskStatus; +import com.ververica.statefun.sdk.AsyncOperationResult; +import com.ververica.statefun.sdk.Context; +import com.ververica.statefun.sdk.FunctionType; +import com.ververica.statefun.sdk.StatefulFunction; +import java.time.Duration; +import java.util.Objects; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ThreadLocalRandom; + +/** + * TaskDurationTrackerFunction - demonstrates the use of asynchronous operations. + * + *

In this example scenario there are tasks executing in an an external system, the external + * system exposes a {@link TaskQueryService} that can report back the status of each individual + * task, in addition the external system generates a {@link TaskStartedEvent} with the task id and + * creation time. + * + *

This function reacts to {@link TaskStartedEvent} and pulls the external {@link + * TaskQueryService} for the task completion, when finally the task completes, this function + * produces an {@link TaskCompletionEvent} with the task execution duration. + */ +final class TaskDurationTrackerFunction implements StatefulFunction { + + static final FunctionType TYPE = + new FunctionType("com.ververica.statefun.examples.async", "duration-tracker"); + + private final TaskQueryService service; + + TaskDurationTrackerFunction(TaskQueryService service) { + this.service = Objects.requireNonNull(service); + } + + @Override + public void invoke(Context context, Object message) { + if (message instanceof TaskStartedEvent) { + // We received a TaskStartedEvent, in response, we need to check with an external service what + // is the status of that task. Since the external service (represented by the client) is + // asynchronous (returns a CompletableFuture) we register that future as a pending + // asynchronous operation, and we will + // get notified once the async operation completes via the special AsyncOperationResult + // message. + // we also attach the original input message as metadata to the async operation. + TaskStartedEvent e = (TaskStartedEvent) message; + CompletableFuture result = service.getTaskStatusAsync(e.getTaskId()); + context.registerAsyncOperation(message, result); + return; + } + if (message instanceof AsyncOperationResult) { + // This is a result of an async operation we have previously registered. + // The message's metadata would be the original input event that triggered the async operation + // (TaskStartedEvent) + // and possibly the asynchronously computed TaskStatus. + @SuppressWarnings("unchecked") + AsyncOperationResult asyncOp = + (AsyncOperationResult) message; + + onAsyncOperationResultEvent(context, asyncOp); + return; + } + throw new IllegalArgumentException("Unknown event " + message); + } + + /** + * Handle the result of an asynchronous operation. The logic of this example is as follows: 1. If + * the async operation itself failed (i.e. IOException, TimeoutException etc') just blindly retry + * within a second 2. If the + */ + private static void onAsyncOperationResultEvent( + Context context, AsyncOperationResult asyncOp) { + + // We have attached the original TaskStartedEvent as a metadata, when registering the async + // operation, so we can just grab it. + final TaskStartedEvent e = asyncOp.metadata(); + + if (!asyncOp.successful()) { + // Something went wrong while trying to obtain the TaskStatus asynchronously, we can inspect + // the cause by + // calling asyncOp.throwable() or asking if the status is unknown (asyncOp.unknown()) + // in any case we retry in 1 second, by just sending a delayed message to ourselves. + Duration delay = oneSecondPlusJitter(); + context.sendAfter(delay, context.self(), e); + return; + } + // The async op has completed successfully now we can obtain the asynchronously computed value. + final TaskStatus status = asyncOp.value(); + if (!status.isCompleted()) { + // The task status is not yet complete, therefore we need to pull the status again at some + // later point in time, lets retry in 10 seconds + context.sendAfter(Duration.ofSeconds(10), context.self(), e); + return; + } + handleCompletedTask(context, e, status); + } + + /** + * compute a duration that represents slightly more than one second (with a random jitter) to + * avoid thundering herds. + */ + private static Duration oneSecondPlusJitter() { + final long randomJitter = ThreadLocalRandom.current().nextLong(1_000, 1_250); + return Duration.ofMillis(randomJitter); + } + + /** The task was completed, we can compute the task execution duration and emit it downstream. */ + private static void handleCompletedTask( + Context context, TaskStartedEvent taskStartedEvent, TaskStatus finishedTaskStatus) { + + TaskCompletionEvent taskCompletionEvent = + new TaskCompletionEvent( + taskStartedEvent.getTaskId(), + taskStartedEvent.getStartTime(), + finishedTaskStatus.getCompletionTime()); + + context.send(Constants.RESULT_EGRESS, taskCompletionEvent); + } +} diff --git a/stateful-functions-examples/stateful-functions-async-example/src/main/java/com/ververica/statefun/examples/async/events/TaskCompletionEvent.java b/stateful-functions-examples/stateful-functions-async-example/src/main/java/com/ververica/statefun/examples/async/events/TaskCompletionEvent.java new file mode 100644 index 00000000..8183e71e --- /dev/null +++ b/stateful-functions-examples/stateful-functions-async-example/src/main/java/com/ververica/statefun/examples/async/events/TaskCompletionEvent.java @@ -0,0 +1,46 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.async.events; + +/** A message represents an event of a task completion. */ +public final class TaskCompletionEvent { + private final String taskId; + private final Long startTime; + private final Long endTime; + + public TaskCompletionEvent(String taskId, Long startTime, Long endTime) { + this.taskId = taskId; + this.startTime = startTime; + this.endTime = endTime; + } + + public String getTaskId() { + return taskId; + } + + public Long getStartTime() { + return startTime; + } + + public Long getEndTime() { + return endTime; + } + + public long taskCompletionDuration() { + return endTime - startTime; + } +} diff --git a/stateful-functions-examples/stateful-functions-async-example/src/main/java/com/ververica/statefun/examples/async/events/TaskStartedEvent.java b/stateful-functions-examples/stateful-functions-async-example/src/main/java/com/ververica/statefun/examples/async/events/TaskStartedEvent.java new file mode 100644 index 00000000..aba845ad --- /dev/null +++ b/stateful-functions-examples/stateful-functions-async-example/src/main/java/com/ververica/statefun/examples/async/events/TaskStartedEvent.java @@ -0,0 +1,36 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.async.events; + +/** A message represents an event of a new task was created. A task has an id and a startingTime. */ +public final class TaskStartedEvent { + private final String taskId; + private final Long startTime; + + public TaskStartedEvent(String taskId, Long startTime) { + this.taskId = taskId; + this.startTime = startTime; + } + + public String getTaskId() { + return taskId; + } + + public Long getStartTime() { + return startTime; + } +} diff --git a/stateful-functions-examples/stateful-functions-async-example/src/main/java/com/ververica/statefun/examples/async/service/DummyTaskQueryService.java b/stateful-functions-examples/stateful-functions-async-example/src/main/java/com/ververica/statefun/examples/async/service/DummyTaskQueryService.java new file mode 100644 index 00000000..faa177ec --- /dev/null +++ b/stateful-functions-examples/stateful-functions-async-example/src/main/java/com/ververica/statefun/examples/async/service/DummyTaskQueryService.java @@ -0,0 +1,73 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.async.service; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; + +/** + * A dummy implementation that simulates a service that might take a while to compute it's result. + */ +public final class DummyTaskQueryService implements TaskQueryService { + + /** + * used to complete the futures after an artificial random delay, that simulates network latency, + * service busyness etc' + */ + private final ScheduledExecutorService executor = newScheduledExecutorService(); + + public CompletableFuture getTaskStatusAsync(String taskId) { + CompletableFuture result = new CompletableFuture<>(); + + final long randomCompletionDelay = ThreadLocalRandom.current().nextLong(5_000); + executor.schedule( + completeRandomly(taskId, result), randomCompletionDelay, TimeUnit.MILLISECONDS); + + return result; + } + + /** + * returns a {@link Runnable}, that when runs, it completes the supplied future {@code result} + * with a {@link TaskStatus} that might be either done or still running. If done, then the task + * status would have a completion time. + */ + private static Runnable completeRandomly(String taskId, CompletableFuture result) { + return () -> { + boolean taskStatus = ThreadLocalRandom.current().nextBoolean(); + + if (taskStatus) { + final long now = System.currentTimeMillis(); + result.complete(new TaskStatus(taskId, true, now)); + } else { + result.complete(new TaskStatus(taskId, false, null)); + } + }; + } + + /** A scheduled executor service with daemon threads. */ + private static ScheduledExecutorService newScheduledExecutorService() { + return Executors.newSingleThreadScheduledExecutor( + r -> { + Thread t = new Thread(r); + t.setDaemon(true); + return t; + }); + } +} diff --git a/stateful-functions-examples/stateful-functions-async-example/src/main/java/com/ververica/statefun/examples/async/service/TaskQueryService.java b/stateful-functions-examples/stateful-functions-async-example/src/main/java/com/ververica/statefun/examples/async/service/TaskQueryService.java new file mode 100644 index 00000000..300a0c99 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-async-example/src/main/java/com/ververica/statefun/examples/async/service/TaskQueryService.java @@ -0,0 +1,31 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.async.service; + +import java.util.concurrent.CompletableFuture; + +/** A remote service that keeps task status by id. */ +public interface TaskQueryService { + + /** + * Retrieves a {@link TaskStatus} of a task with an @taskId. + * + * @param taskId the task to retrieves it's status + * @return the status of the task. + */ + CompletableFuture getTaskStatusAsync(String taskId); +} diff --git a/stateful-functions-examples/stateful-functions-async-example/src/main/java/com/ververica/statefun/examples/async/service/TaskStatus.java b/stateful-functions-examples/stateful-functions-async-example/src/main/java/com/ververica/statefun/examples/async/service/TaskStatus.java new file mode 100644 index 00000000..2ca181ef --- /dev/null +++ b/stateful-functions-examples/stateful-functions-async-example/src/main/java/com/ververica/statefun/examples/async/service/TaskStatus.java @@ -0,0 +1,47 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.async.service; + +/** + * Represents A task status, as returned from the Dummy task service. + * + *

A Task might be either completed or uncompleted. If a task is completed then it would also + * have a completion time. + */ +public class TaskStatus { + private final String taskId; + private final boolean completed; + private final Long completionTime; + + TaskStatus(String taskId, boolean completed, Long completionTime) { + this.taskId = taskId; + this.completed = completed; + this.completionTime = completionTime; + } + + public String getTaskId() { + return taskId; + } + + public boolean isCompleted() { + return completed; + } + + public Long getCompletionTime() { + return completionTime; + } +} diff --git a/stateful-functions-examples/stateful-functions-async-example/src/main/java/com/ververica/statefun/examples/async/service/package-info.java b/stateful-functions-examples/stateful-functions-async-example/src/main/java/com/ververica/statefun/examples/async/service/package-info.java new file mode 100644 index 00000000..63d481db --- /dev/null +++ b/stateful-functions-examples/stateful-functions-async-example/src/main/java/com/ververica/statefun/examples/async/service/package-info.java @@ -0,0 +1,25 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +/** + * This package represents a fictions service that needs to compute something asynchronously. In + * this example, a query service keeps tack of tasks that are executed elsewhere (perhaps some + * worker pool outside of the application) see: {@link + * com.ververica.statefun.examples.async.service.TaskQueryService}. In reality, functions might want + * to compute something asynchronously like sending an http request, querying a remote database, or + * anything really that needs some time to complete. + */ +package com.ververica.statefun.examples.async.service; diff --git a/stateful-functions-examples/stateful-functions-async-example/src/main/resources/META-INF/services/com.ververica.statefun.sdk.spi.StatefulFunctionModule b/stateful-functions-examples/stateful-functions-async-example/src/main/resources/META-INF/services/com.ververica.statefun.sdk.spi.StatefulFunctionModule new file mode 100644 index 00000000..c1a6e6a1 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-async-example/src/main/resources/META-INF/services/com.ververica.statefun.sdk.spi.StatefulFunctionModule @@ -0,0 +1,16 @@ +# +# Copyright 2019 Ververica GmbH. +# +# 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. +# +com.ververica.statefun.examples.async.Module \ No newline at end of file diff --git a/stateful-functions-examples/stateful-functions-flink-harness-example/README.md b/stateful-functions-examples/stateful-functions-flink-harness-example/README.md new file mode 100644 index 00000000..0f1bb0df --- /dev/null +++ b/stateful-functions-examples/stateful-functions-flink-harness-example/README.md @@ -0,0 +1,11 @@ +# Using the Flink Harness to run Stateful Functions applications in the IDE + +This is a simple example to demonstrate how one would run Stateful Functions application in the IDE using the +provided Flink Harness. + +Take a look at the `com.ververica.statefun.examples.harness.Main` class to see example usages of `Harness`. + +## Running the example + +Simply run the `com.ververica.statefun.examples.harness.Main` class. + diff --git a/stateful-functions-examples/stateful-functions-flink-harness-example/pom.xml b/stateful-functions-examples/stateful-functions-flink-harness-example/pom.xml new file mode 100644 index 00000000..d90293ba --- /dev/null +++ b/stateful-functions-examples/stateful-functions-flink-harness-example/pom.xml @@ -0,0 +1,36 @@ + + + + + stateful-functions-examples + com.ververica + 1.0-SNAPSHOT + .. + + 4.0.0 + + stateful-functions-flink-harness-example + + + com.ververica + stateful-functions-flink-harness + ${project.version} + + + \ No newline at end of file diff --git a/stateful-functions-examples/stateful-functions-flink-harness-example/src/main/java/com/ververica/statefun/examples/harness/Main.java b/stateful-functions-examples/stateful-functions-flink-harness-example/src/main/java/com/ververica/statefun/examples/harness/Main.java new file mode 100644 index 00000000..38ef0d12 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-flink-harness-example/src/main/java/com/ververica/statefun/examples/harness/Main.java @@ -0,0 +1,74 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.harness; + +import com.ververica.statefun.examples.harness.MyMessages.MyInputMessage; +import com.ververica.statefun.examples.harness.MyMessages.MyOutputMessage; +import com.ververica.statefun.flink.harness.Harness; +import com.ververica.statefun.flink.harness.io.SerializableConsumer; +import com.ververica.statefun.flink.harness.io.SerializableSupplier; +import java.util.concurrent.ThreadLocalRandom; +import javax.annotation.Nonnull; +import org.apache.flink.util.StringUtils; + +public class Main { + + public static void main(String[] args) throws Exception { + Harness harness = + new Harness() + .noCheckpointing() + .withKryoMessageSerializer() + .withSupplyingIngress(MyConstants.REQUEST_INGRESS, new MessageGenerator()) + .withConsumingEgress(MyConstants.RESULT_EGRESS, new MessagePrinter()); + + harness.start(); + } + + /** generate a random message, once a second a second. */ + private static final class MessageGenerator implements SerializableSupplier { + + private static final long serialVersionUID = 1; + + @Override + public MyInputMessage get() { + try { + Thread.sleep(1_000); + } catch (InterruptedException e) { + throw new RuntimeException("Interrupted", e); + } + return randomMessage(); + } + + @Nonnull + private MyInputMessage randomMessage() { + final ThreadLocalRandom random = ThreadLocalRandom.current(); + final String userId = StringUtils.generateRandomAlphanumericString(random, 2); + return new MyInputMessage(userId, "hello " + userId); + } + } + + /** prints the messages to stdout. */ + private static final class MessagePrinter implements SerializableConsumer { + + private static final long serialVersionUID = 1; + + @Override + public void accept(MyOutputMessage message) { + System.out.println(message); + } + } +} diff --git a/stateful-functions-examples/stateful-functions-flink-harness-example/src/main/java/com/ververica/statefun/examples/harness/MyConstants.java b/stateful-functions-examples/stateful-functions-flink-harness-example/src/main/java/com/ververica/statefun/examples/harness/MyConstants.java new file mode 100644 index 00000000..1de86371 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-flink-harness-example/src/main/java/com/ververica/statefun/examples/harness/MyConstants.java @@ -0,0 +1,36 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.harness; + +import com.ververica.statefun.examples.harness.MyMessages.MyInputMessage; +import com.ververica.statefun.examples.harness.MyMessages.MyOutputMessage; +import com.ververica.statefun.sdk.FunctionType; +import com.ververica.statefun.sdk.io.EgressIdentifier; +import com.ververica.statefun.sdk.io.IngressIdentifier; + +final class MyConstants { + static final IngressIdentifier REQUEST_INGRESS = + new IngressIdentifier<>( + MyInputMessage.class, "com.ververica.statefun.examples.harness", "in"); + + static final EgressIdentifier RESULT_EGRESS = + new EgressIdentifier<>( + "com.ververica.statefun.examples.harness", "out", MyOutputMessage.class); + + static final FunctionType MY_FUNCTION_TYPE = + new FunctionType("com.ververica.statefun.examples.harness", "my-function"); +} diff --git a/stateful-functions-examples/stateful-functions-flink-harness-example/src/main/java/com/ververica/statefun/examples/harness/MyFunction.java b/stateful-functions-examples/stateful-functions-flink-harness-example/src/main/java/com/ververica/statefun/examples/harness/MyFunction.java new file mode 100644 index 00000000..54e6c96a --- /dev/null +++ b/stateful-functions-examples/stateful-functions-flink-harness-example/src/main/java/com/ververica/statefun/examples/harness/MyFunction.java @@ -0,0 +1,36 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.harness; + +import com.ververica.statefun.examples.harness.MyMessages.MyInputMessage; +import com.ververica.statefun.examples.harness.MyMessages.MyOutputMessage; +import com.ververica.statefun.sdk.Context; +import com.ververica.statefun.sdk.StatefulFunction; + +final class MyFunction implements StatefulFunction { + + @Override + public void invoke(Context context, Object input) { + if (!(input instanceof MyInputMessage)) { + throw new IllegalArgumentException("Unknown message received " + input); + } + MyInputMessage in = (MyInputMessage) input; + MyOutputMessage out = new MyOutputMessage(in.getUserId(), in.getMessage()); + + context.send(MyConstants.RESULT_EGRESS, out); + } +} diff --git a/stateful-functions-examples/stateful-functions-flink-harness-example/src/main/java/com/ververica/statefun/examples/harness/MyMessages.java b/stateful-functions-examples/stateful-functions-flink-harness-example/src/main/java/com/ververica/statefun/examples/harness/MyMessages.java new file mode 100644 index 00000000..a6fde3ca --- /dev/null +++ b/stateful-functions-examples/stateful-functions-flink-harness-example/src/main/java/com/ververica/statefun/examples/harness/MyMessages.java @@ -0,0 +1,61 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.harness; + +final class MyMessages { + + static final class MyInputMessage { + private final String userId; + private final String message; + + MyInputMessage(String userId, String message) { + this.userId = userId; + this.message = message; + } + + String getUserId() { + return userId; + } + + String getMessage() { + return message; + } + } + + static final class MyOutputMessage { + private final String userId; + private final String content; + + MyOutputMessage(String userId, String content) { + this.userId = userId; + this.content = content; + } + + String getUserId() { + return userId; + } + + String getContent() { + return content; + } + + @Override + public String toString() { + return String.format("MyOutputMessage(%s, %s)", getUserId(), getContent()); + } + } +} diff --git a/stateful-functions-examples/stateful-functions-flink-harness-example/src/main/java/com/ververica/statefun/examples/harness/MyModule.java b/stateful-functions-examples/stateful-functions-flink-harness-example/src/main/java/com/ververica/statefun/examples/harness/MyModule.java new file mode 100644 index 00000000..91520008 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-flink-harness-example/src/main/java/com/ververica/statefun/examples/harness/MyModule.java @@ -0,0 +1,29 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.harness; + +import com.ververica.statefun.sdk.spi.StatefulFunctionModule; +import java.util.Map; + +public class MyModule implements StatefulFunctionModule { + + @Override + public void configure(Map globalConfiguration, Binder binder) { + binder.bindIngressRouter(MyConstants.REQUEST_INGRESS, new MyRouter()); + binder.bindFunctionProvider(MyConstants.MY_FUNCTION_TYPE, unused -> new MyFunction()); + } +} diff --git a/stateful-functions-examples/stateful-functions-flink-harness-example/src/main/java/com/ververica/statefun/examples/harness/MyRouter.java b/stateful-functions-examples/stateful-functions-flink-harness-example/src/main/java/com/ververica/statefun/examples/harness/MyRouter.java new file mode 100644 index 00000000..8ac67c73 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-flink-harness-example/src/main/java/com/ververica/statefun/examples/harness/MyRouter.java @@ -0,0 +1,28 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.harness; + +import com.ververica.statefun.examples.harness.MyMessages.MyInputMessage; +import com.ververica.statefun.sdk.io.Router; + +final class MyRouter implements Router { + + @Override + public void route(MyInputMessage message, Downstream downstream) { + downstream.forward(MyConstants.MY_FUNCTION_TYPE, message.getUserId(), message); + } +} diff --git a/stateful-functions-examples/stateful-functions-flink-harness-example/src/main/resources/META-INF/services/com.ververica.statefun.sdk.spi.StatefulFunctionModule b/stateful-functions-examples/stateful-functions-flink-harness-example/src/main/resources/META-INF/services/com.ververica.statefun.sdk.spi.StatefulFunctionModule new file mode 100644 index 00000000..165413ce --- /dev/null +++ b/stateful-functions-examples/stateful-functions-flink-harness-example/src/main/resources/META-INF/services/com.ververica.statefun.sdk.spi.StatefulFunctionModule @@ -0,0 +1,16 @@ +# +# Copyright 2019 Ververica GmbH. +# +# 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. +# +com.ververica.statefun.examples.harness.MyModule \ No newline at end of file diff --git a/stateful-functions-examples/stateful-functions-greeter-example/Dockerfile b/stateful-functions-examples/stateful-functions-greeter-example/Dockerfile new file mode 100644 index 00000000..c1ee0b38 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-greeter-example/Dockerfile @@ -0,0 +1,20 @@ +# +# Copyright 2019 Ververica GmbH. +# +# 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. +# + +FROM stateful-functions + +RUN mkdir -p /opt/stateful-functions/modules/stateful-functions-greeter-example +COPY target/stateful-functions-greeter-example*jar /opt/stateful-functions/modules/stateful-functions-greeter-example/ diff --git a/stateful-functions-examples/stateful-functions-greeter-example/README.md b/stateful-functions-examples/stateful-functions-greeter-example/README.md new file mode 100644 index 00000000..75f303f1 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-greeter-example/README.md @@ -0,0 +1,32 @@ +# The Greeter Example + +This is a simple example that runs a simple stateful function that accepts requests from a Kafka ingress, +and then responds by sending greeting responses to a Kafka egress. It demonstrates the primitive building blocks +of a Stateful Functions applications, such as ingresses, routing messages to functions, handling state in functions, +and sending messages to egresses. + +## Running the example + +To run the example: + +``` +docker-compose build +docker-compose up +``` + +Then, to see the example in actions, send some messages to the topic `names`, and see what comes out +out of the topic `greetings`: + +``` +KAFKA=$(docker ps -f "name=stateful-functions-greeter-example_kafka-broker_1" --format "{{.ID}}") ; \ +docker exec -it $KAFKA kafka-console-producer.sh \ + --broker-list localhost:9092 \ + --topic names +``` + +``` +KAFKA=$(docker ps -f "name=stateful-functions-greeter-example_kafka-broker_1" --format "{{.ID}}") ; \ +docker exec -it $KAFKA kafka-console-consumer.sh \ + --bootstrap-server localhost:9092 \ + --topic greetings +``` diff --git a/stateful-functions-examples/stateful-functions-greeter-example/docker-compose.yml b/stateful-functions-examples/stateful-functions-greeter-example/docker-compose.yml new file mode 100644 index 00000000..4df1c6cf --- /dev/null +++ b/stateful-functions-examples/stateful-functions-greeter-example/docker-compose.yml @@ -0,0 +1,66 @@ +# +# Copyright 2019 Ververica GmbH. +# +# 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. +# + +version: "2.1" +services: + zookeeper: + image: wurstmeister/zookeeper + ports: + - "2181:2181" + kafka-broker: + image: wurstmeister/kafka:2.12-2.0.1 + ports: + - "9092:9092" + environment: + HOSTNAME_COMMAND: "route -n | awk '/UG[ \t]/{print $$2}'" + KAFKA_CREATE_TOPICS: "names:1:1,greetings:1:1" + KAFKA_ZOOKEEPER_CONNECT: zookeeper:2181 + depends_on: + - zookeeper + volumes: + - /var/run/docker.sock:/var/run/docker.sock + master: + build: + context: . + # uncomment to start from a savepoint + #command: -s /checkpoint-dir/savepoint/savepoint-bf101a-4e99820085fe + expose: + - "6123" + ports: + - "8081:8081" + environment: + - ROLE=master + - MASTER_HOST=master + volumes: + - ./checkpoint-dir:/checkpoint-dir + worker: + build: + context: . + expose: + - "6121" + - "6122" + depends_on: + - master + - kafka-broker + links: + - "master:master" + - "kafka-broker:kafka-broker" + environment: + - ROLE=worker + - MASTER_HOST=master + volumes: + - ./checkpoint-dir:/checkpoint-dir + diff --git a/stateful-functions-examples/stateful-functions-greeter-example/pom.xml b/stateful-functions-examples/stateful-functions-greeter-example/pom.xml new file mode 100644 index 00000000..8c462b36 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-greeter-example/pom.xml @@ -0,0 +1,48 @@ + + + + + stateful-functions-examples + com.ververica + 1.0-SNAPSHOT + .. + + 4.0.0 + + stateful-functions-greeter-example + + + + com.ververica + stateful-functions-sdk + ${project.version} + + + com.ververica + stateful-functions-kafka-io + ${project.version} + + + com.google.protobuf + protobuf-java + 3.8.0 + + + + diff --git a/stateful-functions-examples/stateful-functions-greeter-example/src/main/java/com/ververica/statefun/examples/greeter/GreetRouter.java b/stateful-functions-examples/stateful-functions-greeter-example/src/main/java/com/ververica/statefun/examples/greeter/GreetRouter.java new file mode 100644 index 00000000..c7d1d46e --- /dev/null +++ b/stateful-functions-examples/stateful-functions-greeter-example/src/main/java/com/ververica/statefun/examples/greeter/GreetRouter.java @@ -0,0 +1,32 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.greeter; + +import com.ververica.statefun.examples.greeter.generated.GreetRequest; +import com.ververica.statefun.sdk.io.Router; + +/** + * The greet router takes each message from an ingress and routes it to a greeter function based on + * the users id. + */ +final class GreetRouter implements Router { + + @Override + public void route(GreetRequest message, Downstream downstream) { + downstream.forward(GreetStatefulFunction.TYPE, message.getWho(), message); + } +} diff --git a/stateful-functions-examples/stateful-functions-greeter-example/src/main/java/com/ververica/statefun/examples/greeter/GreetStatefulFunction.java b/stateful-functions-examples/stateful-functions-greeter-example/src/main/java/com/ververica/statefun/examples/greeter/GreetStatefulFunction.java new file mode 100644 index 00000000..fa194719 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-greeter-example/src/main/java/com/ververica/statefun/examples/greeter/GreetStatefulFunction.java @@ -0,0 +1,82 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.greeter; + +import com.ververica.statefun.examples.greeter.generated.GreetRequest; +import com.ververica.statefun.examples.greeter.generated.GreetResponse; +import com.ververica.statefun.sdk.Context; +import com.ververica.statefun.sdk.FunctionType; +import com.ververica.statefun.sdk.StatefulFunction; +import com.ververica.statefun.sdk.annotations.Persisted; +import com.ververica.statefun.sdk.state.PersistedValue; + +/** + * A stateful function that generates a unique greeting for each user based on how many times that + * user has been seen by the system. + */ +final class GreetStatefulFunction implements StatefulFunction { + + /** + * The function type is the unique identifier that identifies this type of function. The type, in + * conjunction with an identifier, is how routers and other functions can use to reference a + * particular instance of a greeter function. + * + *

If this was a multi-module application, the function type could be in different package so + * functions in other modules could message the greeter without a direct dependency on this class. + */ + static final FunctionType TYPE = new FunctionType("ververica", "greeter"); + + /** + * The persisted value for maintaining state about a particular user. The value returned by this + * field is always scoped to the current user. seenCount is the number of times the user has been + * greeted. + */ + @Persisted + private final PersistedValue seenCount = PersistedValue.of("seen-count", Integer.class); + + @Override + public void invoke(Context context, Object input) { + GreetRequest greetMessage = (GreetRequest) input; + GreetResponse response = computePersonalizedGreeting(greetMessage); + context.send(GreetingIO.GREETING_EGRESS_ID, response); + } + + private GreetResponse computePersonalizedGreeting(GreetRequest greetMessage) { + final String name = greetMessage.getWho(); + final int seen = seenCount.getOrDefault(0); + seenCount.set(seen + 1); + + String greeting = greetText(name, seen); + + return GreetResponse.newBuilder().setWho(name).setGreeting(greeting).build(); + } + + private static String greetText(String name, int seen) { + switch (seen) { + case 0: + return String.format("Hello %s ! \uD83D\uDE0E", name); + case 1: + return String.format("Hello again %s ! \uD83E\uDD17", name); + case 2: + return String.format("Third time is a charm! %s! \uD83E\uDD73", name); + case 3: + return String.format("Happy to see you once again %s ! \uD83D\uDE32", name); + default: + return String.format("Hello at the %d-th time %s \uD83D\uDE4C", seen + 1, name); + } + } +} diff --git a/stateful-functions-examples/stateful-functions-greeter-example/src/main/java/com/ververica/statefun/examples/greeter/GreetingIO.java b/stateful-functions-examples/stateful-functions-greeter-example/src/main/java/com/ververica/statefun/examples/greeter/GreetingIO.java new file mode 100644 index 00000000..7e3d961d --- /dev/null +++ b/stateful-functions-examples/stateful-functions-greeter-example/src/main/java/com/ververica/statefun/examples/greeter/GreetingIO.java @@ -0,0 +1,98 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.greeter; + +import com.ververica.statefun.examples.greeter.generated.GreetRequest; +import com.ververica.statefun.examples.greeter.generated.GreetResponse; +import com.ververica.statefun.sdk.io.EgressIdentifier; +import com.ververica.statefun.sdk.io.EgressSpec; +import com.ververica.statefun.sdk.io.IngressIdentifier; +import com.ververica.statefun.sdk.io.IngressSpec; +import com.ververica.statefun.sdk.kafka.KafkaEgressBuilder; +import com.ververica.statefun.sdk.kafka.KafkaEgressSerializer; +import com.ververica.statefun.sdk.kafka.KafkaIngressBuilder; +import com.ververica.statefun.sdk.kafka.KafkaIngressDeserializer; +import java.nio.charset.StandardCharsets; +import java.util.Objects; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.producer.ProducerRecord; + +/** + * A collection of all the components necessary to consume from and write to an external system, in + * this case Apache Kafka. + * + *

The ingress and egress identifiers provide named references without exposing the underlying + * system. This way, in a multi-module deployment, functions can interact with IO modules through + * identifiers without depending on specific implementations. + */ +final class GreetingIO { + + static final IngressIdentifier GREETING_INGRESS_ID = + new IngressIdentifier<>(GreetRequest.class, "ververica", "greet-ingress"); + + static final EgressIdentifier GREETING_EGRESS_ID = + new EgressIdentifier<>("ververica", "kafka-greeting-output", GreetResponse.class); + + private final String kafkaAddress; + + GreetingIO(String kafkaAddress) { + this.kafkaAddress = Objects.requireNonNull(kafkaAddress); + } + + IngressSpec getIngressSpec() { + return KafkaIngressBuilder.forIdentifier(GREETING_INGRESS_ID) + .withKafkaAddress(kafkaAddress) + .withTopic("names") + .withDeserializer(GreetKafkaDeserializer.class) + .withProperty(ConsumerConfig.GROUP_ID_CONFIG, "greetings") + .build(); + } + + EgressSpec getEgressSpec() { + return KafkaEgressBuilder.forIdentifier(GREETING_EGRESS_ID) + .withKafkaAddress(kafkaAddress) + .withSerializer(GreetKafkaSerializer.class) + .build(); + } + + private static final class GreetKafkaDeserializer + implements KafkaIngressDeserializer { + + private static final long serialVersionUID = 1L; + + @Override + public GreetRequest deserialize(ConsumerRecord input) { + String who = new String(input.value(), StandardCharsets.UTF_8); + + return GreetRequest.newBuilder().setWho(who).build(); + } + } + + private static final class GreetKafkaSerializer implements KafkaEgressSerializer { + + private static final long serialVersionUID = 1L; + + @Override + public ProducerRecord serialize(GreetResponse response) { + byte[] key = response.getWho().getBytes(StandardCharsets.UTF_8); + byte[] value = response.getGreeting().getBytes(StandardCharsets.UTF_8); + + return new ProducerRecord<>("greetings", key, value); + } + } +} diff --git a/stateful-functions-examples/stateful-functions-greeter-example/src/main/java/com/ververica/statefun/examples/greeter/GreetingModule.java b/stateful-functions-examples/stateful-functions-greeter-example/src/main/java/com/ververica/statefun/examples/greeter/GreetingModule.java new file mode 100644 index 00000000..8da87c71 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-greeter-example/src/main/java/com/ververica/statefun/examples/greeter/GreetingModule.java @@ -0,0 +1,51 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.greeter; + +import com.ververica.statefun.sdk.spi.StatefulFunctionModule; +import java.util.Map; + +/** + * The top level entry point for this application. + * + *

On deployment, the address of the Kafka brokers can be configured by passing the flag + * `--kafka-address

`. If no flag is passed, then the default address will be used. + */ +public final class GreetingModule implements StatefulFunctionModule { + + private static final String KAFKA_KEY = "kafka-address"; + + private static final String DEFAULT_KAFKA_ADDRESS = "kafka-broker:9092"; + + @Override + public void configure(Map globalConfiguration, Binder binder) { + + // pull the configured kafka broker address, or default if none was passed. + String kafkaAddress = globalConfiguration.getOrDefault(KAFKA_KEY, DEFAULT_KAFKA_ADDRESS); + GreetingIO ioModule = new GreetingIO(kafkaAddress); + + // bind an ingress to the system along with the router + binder.bindIngress(ioModule.getIngressSpec()); + binder.bindIngressRouter(GreetingIO.GREETING_INGRESS_ID, new GreetRouter()); + + // bind an egress to the system + binder.bindEgress(ioModule.getEgressSpec()); + + // bind a function provider to a function type + binder.bindFunctionProvider(GreetStatefulFunction.TYPE, unused -> new GreetStatefulFunction()); + } +} diff --git a/stateful-functions-examples/stateful-functions-greeter-example/src/main/java/com/ververica/statefun/examples/greeter/generated/GreetRequest.java b/stateful-functions-examples/stateful-functions-greeter-example/src/main/java/com/ververica/statefun/examples/greeter/generated/GreetRequest.java new file mode 100644 index 00000000..8dc21ac3 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-greeter-example/src/main/java/com/ververica/statefun/examples/greeter/generated/GreetRequest.java @@ -0,0 +1,533 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/greeter.proto + +package com.ververica.statefun.examples.greeter.generated; + +/** Protobuf type {@code com.ververica.statefun.examples.kafka.GreetRequest} */ +public final class GreetRequest extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.kafka.GreetRequest) + GreetRequestOrBuilder { + private static final long serialVersionUID = 0L; + // Use GreetRequest.newBuilder() to construct. + private GreetRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private GreetRequest() { + who_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private GreetRequest( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + java.lang.String s = input.readStringRequireUtf8(); + + who_ = s; + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.greeter.generated.Greeter + .internal_static_com_ververica_statefun_examples_kafka_GreetRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.greeter.generated.Greeter + .internal_static_com_ververica_statefun_examples_kafka_GreetRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.greeter.generated.GreetRequest.class, + com.ververica.statefun.examples.greeter.generated.GreetRequest.Builder.class); + } + + public static final int WHO_FIELD_NUMBER = 1; + private volatile java.lang.Object who_; + /** string who = 1; */ + public java.lang.String getWho() { + java.lang.Object ref = who_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + who_ = s; + return s; + } + } + /** string who = 1; */ + public com.google.protobuf.ByteString getWhoBytes() { + java.lang.Object ref = who_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + who_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!getWhoBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, who_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!getWhoBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, who_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.ververica.statefun.examples.greeter.generated.GreetRequest)) { + return super.equals(obj); + } + com.ververica.statefun.examples.greeter.generated.GreetRequest other = + (com.ververica.statefun.examples.greeter.generated.GreetRequest) obj; + + if (!getWho().equals(other.getWho())) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + WHO_FIELD_NUMBER; + hash = (53 * hash) + getWho().hashCode(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.greeter.generated.GreetRequest parseFrom( + java.nio.ByteBuffer data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.greeter.generated.GreetRequest parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.greeter.generated.GreetRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.greeter.generated.GreetRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.greeter.generated.GreetRequest parseFrom( + byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.greeter.generated.GreetRequest parseFrom( + byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.greeter.generated.GreetRequest parseFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.greeter.generated.GreetRequest parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.greeter.generated.GreetRequest parseDelimitedFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.greeter.generated.GreetRequest parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.greeter.generated.GreetRequest parseFrom( + com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.greeter.generated.GreetRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.greeter.generated.GreetRequest prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code com.ververica.statefun.examples.kafka.GreetRequest} */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.kafka.GreetRequest) + com.ververica.statefun.examples.greeter.generated.GreetRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.greeter.generated.Greeter + .internal_static_com_ververica_statefun_examples_kafka_GreetRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.greeter.generated.Greeter + .internal_static_com_ververica_statefun_examples_kafka_GreetRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.greeter.generated.GreetRequest.class, + com.ververica.statefun.examples.greeter.generated.GreetRequest.Builder.class); + } + + // Construct using com.ververica.statefun.examples.greeter.generated.GreetRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + who_ = ""; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.greeter.generated.Greeter + .internal_static_com_ververica_statefun_examples_kafka_GreetRequest_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.greeter.generated.GreetRequest + getDefaultInstanceForType() { + return com.ververica.statefun.examples.greeter.generated.GreetRequest.getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.greeter.generated.GreetRequest build() { + com.ververica.statefun.examples.greeter.generated.GreetRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.greeter.generated.GreetRequest buildPartial() { + com.ververica.statefun.examples.greeter.generated.GreetRequest result = + new com.ververica.statefun.examples.greeter.generated.GreetRequest(this); + result.who_ = who_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.ververica.statefun.examples.greeter.generated.GreetRequest) { + return mergeFrom((com.ververica.statefun.examples.greeter.generated.GreetRequest) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.ververica.statefun.examples.greeter.generated.GreetRequest other) { + if (other + == com.ververica.statefun.examples.greeter.generated.GreetRequest.getDefaultInstance()) + return this; + if (!other.getWho().isEmpty()) { + who_ = other.who_; + onChanged(); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.greeter.generated.GreetRequest parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.greeter.generated.GreetRequest) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private java.lang.Object who_ = ""; + /** string who = 1; */ + public java.lang.String getWho() { + java.lang.Object ref = who_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + who_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string who = 1; */ + public com.google.protobuf.ByteString getWhoBytes() { + java.lang.Object ref = who_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + who_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string who = 1; */ + public Builder setWho(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + who_ = value; + onChanged(); + return this; + } + /** string who = 1; */ + public Builder clearWho() { + + who_ = getDefaultInstance().getWho(); + onChanged(); + return this; + } + /** string who = 1; */ + public Builder setWhoBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + who_ = value; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.kafka.GreetRequest) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.kafka.GreetRequest) + private static final com.ververica.statefun.examples.greeter.generated.GreetRequest + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = new com.ververica.statefun.examples.greeter.generated.GreetRequest(); + } + + public static com.ververica.statefun.examples.greeter.generated.GreetRequest + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public GreetRequest parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new GreetRequest(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.greeter.generated.GreetRequest + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } +} diff --git a/stateful-functions-examples/stateful-functions-greeter-example/src/main/java/com/ververica/statefun/examples/greeter/generated/GreetRequestOrBuilder.java b/stateful-functions-examples/stateful-functions-greeter-example/src/main/java/com/ververica/statefun/examples/greeter/generated/GreetRequestOrBuilder.java new file mode 100644 index 00000000..b2dcf787 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-greeter-example/src/main/java/com/ververica/statefun/examples/greeter/generated/GreetRequestOrBuilder.java @@ -0,0 +1,15 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/greeter.proto + +package com.ververica.statefun.examples.greeter.generated; + +public interface GreetRequestOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.kafka.GreetRequest) + com.google.protobuf.MessageOrBuilder { + + /** string who = 1; */ + java.lang.String getWho(); + /** string who = 1; */ + com.google.protobuf.ByteString getWhoBytes(); +} diff --git a/stateful-functions-examples/stateful-functions-greeter-example/src/main/java/com/ververica/statefun/examples/greeter/generated/GreetResponse.java b/stateful-functions-examples/stateful-functions-greeter-example/src/main/java/com/ververica/statefun/examples/greeter/generated/GreetResponse.java new file mode 100644 index 00000000..c9bea49d --- /dev/null +++ b/stateful-functions-examples/stateful-functions-greeter-example/src/main/java/com/ververica/statefun/examples/greeter/generated/GreetResponse.java @@ -0,0 +1,639 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/greeter.proto + +package com.ververica.statefun.examples.greeter.generated; + +/** Protobuf type {@code com.ververica.statefun.examples.kafka.GreetResponse} */ +public final class GreetResponse extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.kafka.GreetResponse) + GreetResponseOrBuilder { + private static final long serialVersionUID = 0L; + // Use GreetResponse.newBuilder() to construct. + private GreetResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private GreetResponse() { + who_ = ""; + greeting_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private GreetResponse( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + java.lang.String s = input.readStringRequireUtf8(); + + who_ = s; + break; + } + case 18: + { + java.lang.String s = input.readStringRequireUtf8(); + + greeting_ = s; + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.greeter.generated.Greeter + .internal_static_com_ververica_statefun_examples_kafka_GreetResponse_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.greeter.generated.Greeter + .internal_static_com_ververica_statefun_examples_kafka_GreetResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.greeter.generated.GreetResponse.class, + com.ververica.statefun.examples.greeter.generated.GreetResponse.Builder.class); + } + + public static final int WHO_FIELD_NUMBER = 1; + private volatile java.lang.Object who_; + /** string who = 1; */ + public java.lang.String getWho() { + java.lang.Object ref = who_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + who_ = s; + return s; + } + } + /** string who = 1; */ + public com.google.protobuf.ByteString getWhoBytes() { + java.lang.Object ref = who_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + who_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int GREETING_FIELD_NUMBER = 2; + private volatile java.lang.Object greeting_; + /** string greeting = 2; */ + public java.lang.String getGreeting() { + java.lang.Object ref = greeting_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + greeting_ = s; + return s; + } + } + /** string greeting = 2; */ + public com.google.protobuf.ByteString getGreetingBytes() { + java.lang.Object ref = greeting_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + greeting_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!getWhoBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, who_); + } + if (!getGreetingBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 2, greeting_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!getWhoBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, who_); + } + if (!getGreetingBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, greeting_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.ververica.statefun.examples.greeter.generated.GreetResponse)) { + return super.equals(obj); + } + com.ververica.statefun.examples.greeter.generated.GreetResponse other = + (com.ververica.statefun.examples.greeter.generated.GreetResponse) obj; + + if (!getWho().equals(other.getWho())) return false; + if (!getGreeting().equals(other.getGreeting())) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + WHO_FIELD_NUMBER; + hash = (53 * hash) + getWho().hashCode(); + hash = (37 * hash) + GREETING_FIELD_NUMBER; + hash = (53 * hash) + getGreeting().hashCode(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.greeter.generated.GreetResponse parseFrom( + java.nio.ByteBuffer data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.greeter.generated.GreetResponse parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.greeter.generated.GreetResponse parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.greeter.generated.GreetResponse parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.greeter.generated.GreetResponse parseFrom( + byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.greeter.generated.GreetResponse parseFrom( + byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.greeter.generated.GreetResponse parseFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.greeter.generated.GreetResponse parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.greeter.generated.GreetResponse parseDelimitedFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.greeter.generated.GreetResponse parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.greeter.generated.GreetResponse parseFrom( + com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.greeter.generated.GreetResponse parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.greeter.generated.GreetResponse prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code com.ververica.statefun.examples.kafka.GreetResponse} */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.kafka.GreetResponse) + com.ververica.statefun.examples.greeter.generated.GreetResponseOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.greeter.generated.Greeter + .internal_static_com_ververica_statefun_examples_kafka_GreetResponse_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.greeter.generated.Greeter + .internal_static_com_ververica_statefun_examples_kafka_GreetResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.greeter.generated.GreetResponse.class, + com.ververica.statefun.examples.greeter.generated.GreetResponse.Builder.class); + } + + // Construct using com.ververica.statefun.examples.greeter.generated.GreetResponse.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + who_ = ""; + + greeting_ = ""; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.greeter.generated.Greeter + .internal_static_com_ververica_statefun_examples_kafka_GreetResponse_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.greeter.generated.GreetResponse + getDefaultInstanceForType() { + return com.ververica.statefun.examples.greeter.generated.GreetResponse.getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.greeter.generated.GreetResponse build() { + com.ververica.statefun.examples.greeter.generated.GreetResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.greeter.generated.GreetResponse buildPartial() { + com.ververica.statefun.examples.greeter.generated.GreetResponse result = + new com.ververica.statefun.examples.greeter.generated.GreetResponse(this); + result.who_ = who_; + result.greeting_ = greeting_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.ververica.statefun.examples.greeter.generated.GreetResponse) { + return mergeFrom((com.ververica.statefun.examples.greeter.generated.GreetResponse) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.greeter.generated.GreetResponse other) { + if (other + == com.ververica.statefun.examples.greeter.generated.GreetResponse.getDefaultInstance()) + return this; + if (!other.getWho().isEmpty()) { + who_ = other.who_; + onChanged(); + } + if (!other.getGreeting().isEmpty()) { + greeting_ = other.greeting_; + onChanged(); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.greeter.generated.GreetResponse parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.greeter.generated.GreetResponse) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private java.lang.Object who_ = ""; + /** string who = 1; */ + public java.lang.String getWho() { + java.lang.Object ref = who_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + who_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string who = 1; */ + public com.google.protobuf.ByteString getWhoBytes() { + java.lang.Object ref = who_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + who_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string who = 1; */ + public Builder setWho(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + who_ = value; + onChanged(); + return this; + } + /** string who = 1; */ + public Builder clearWho() { + + who_ = getDefaultInstance().getWho(); + onChanged(); + return this; + } + /** string who = 1; */ + public Builder setWhoBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + who_ = value; + onChanged(); + return this; + } + + private java.lang.Object greeting_ = ""; + /** string greeting = 2; */ + public java.lang.String getGreeting() { + java.lang.Object ref = greeting_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + greeting_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string greeting = 2; */ + public com.google.protobuf.ByteString getGreetingBytes() { + java.lang.Object ref = greeting_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + greeting_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string greeting = 2; */ + public Builder setGreeting(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + greeting_ = value; + onChanged(); + return this; + } + /** string greeting = 2; */ + public Builder clearGreeting() { + + greeting_ = getDefaultInstance().getGreeting(); + onChanged(); + return this; + } + /** string greeting = 2; */ + public Builder setGreetingBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + greeting_ = value; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.kafka.GreetResponse) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.kafka.GreetResponse) + private static final com.ververica.statefun.examples.greeter.generated.GreetResponse + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = new com.ververica.statefun.examples.greeter.generated.GreetResponse(); + } + + public static com.ververica.statefun.examples.greeter.generated.GreetResponse + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public GreetResponse parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new GreetResponse(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.greeter.generated.GreetResponse + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } +} diff --git a/stateful-functions-examples/stateful-functions-greeter-example/src/main/java/com/ververica/statefun/examples/greeter/generated/GreetResponseOrBuilder.java b/stateful-functions-examples/stateful-functions-greeter-example/src/main/java/com/ververica/statefun/examples/greeter/generated/GreetResponseOrBuilder.java new file mode 100644 index 00000000..b57a8945 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-greeter-example/src/main/java/com/ververica/statefun/examples/greeter/generated/GreetResponseOrBuilder.java @@ -0,0 +1,20 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/greeter.proto + +package com.ververica.statefun.examples.greeter.generated; + +public interface GreetResponseOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.kafka.GreetResponse) + com.google.protobuf.MessageOrBuilder { + + /** string who = 1; */ + java.lang.String getWho(); + /** string who = 1; */ + com.google.protobuf.ByteString getWhoBytes(); + + /** string greeting = 2; */ + java.lang.String getGreeting(); + /** string greeting = 2; */ + com.google.protobuf.ByteString getGreetingBytes(); +} diff --git a/stateful-functions-examples/stateful-functions-greeter-example/src/main/java/com/ververica/statefun/examples/greeter/generated/Greeter.java b/stateful-functions-examples/stateful-functions-greeter-example/src/main/java/com/ververica/statefun/examples/greeter/generated/Greeter.java new file mode 100644 index 00000000..ae2c4839 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-greeter-example/src/main/java/com/ververica/statefun/examples/greeter/generated/Greeter.java @@ -0,0 +1,68 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/greeter.proto + +package com.ververica.statefun.examples.greeter.generated; + +public final class Greeter { + private Greeter() {} + + public static void registerAllExtensions(com.google.protobuf.ExtensionRegistryLite registry) {} + + public static void registerAllExtensions(com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions((com.google.protobuf.ExtensionRegistryLite) registry); + } + + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_kafka_GreetRequest_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_kafka_GreetRequest_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_kafka_GreetResponse_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_kafka_GreetResponse_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { + return descriptor; + } + + private static com.google.protobuf.Descriptors.FileDescriptor descriptor; + + static { + java.lang.String[] descriptorData = { + "\n\037src/main/protobuf/greeter.proto\022%com.v" + + "erverica.statefun.examples.kafka\"\033\n\014Gree" + + "tRequest\022\013\n\003who\030\001 \001(\t\".\n\rGreetResponse\022\013" + + "\n\003who\030\001 \001(\t\022\020\n\010greeting\030\002 \001(\tB5\n1com.ver" + + "verica.statefun.examples.greeter.generat" + + "edP\001b\006proto3" + }; + com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = + new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { + public com.google.protobuf.ExtensionRegistry assignDescriptors( + com.google.protobuf.Descriptors.FileDescriptor root) { + descriptor = root; + return null; + } + }; + com.google.protobuf.Descriptors.FileDescriptor.internalBuildGeneratedFileFrom( + descriptorData, new com.google.protobuf.Descriptors.FileDescriptor[] {}, assigner); + internal_static_com_ververica_statefun_examples_kafka_GreetRequest_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_com_ververica_statefun_examples_kafka_GreetRequest_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_kafka_GreetRequest_descriptor, + new java.lang.String[] { + "Who", + }); + internal_static_com_ververica_statefun_examples_kafka_GreetResponse_descriptor = + getDescriptor().getMessageTypes().get(1); + internal_static_com_ververica_statefun_examples_kafka_GreetResponse_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_kafka_GreetResponse_descriptor, + new java.lang.String[] { + "Who", "Greeting", + }); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/stateful-functions-examples/stateful-functions-greeter-example/src/main/protobuf/greeter.proto b/stateful-functions-examples/stateful-functions-greeter-example/src/main/protobuf/greeter.proto new file mode 100644 index 00000000..01b0a604 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-greeter-example/src/main/protobuf/greeter.proto @@ -0,0 +1,32 @@ +// +// Copyright 2019 Ververica GmbH. +// +// 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"; + +package com.ververica.statefun.examples.kafka; +option java_package = "com.ververica.statefun.examples.greeter.generated"; +option java_multiple_files = true; + +message GreetRequest { + string who = 1; +} + +message GreetResponse { + string who = 1; + string greeting = 2; +} + + diff --git a/stateful-functions-examples/stateful-functions-greeter-example/src/main/resources/META-INF/services/com.ververica.statefun.sdk.spi.StatefulFunctionModule b/stateful-functions-examples/stateful-functions-greeter-example/src/main/resources/META-INF/services/com.ververica.statefun.sdk.spi.StatefulFunctionModule new file mode 100644 index 00000000..c22c3e67 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-greeter-example/src/main/resources/META-INF/services/com.ververica.statefun.sdk.spi.StatefulFunctionModule @@ -0,0 +1,19 @@ +# +# Copyright 2019 Ververica GmbH. +# +# 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. +# + +# The service loader entry so this module will be discovered +# by the runtime. +com.ververica.statefun.examples.greeter.GreetingModule \ No newline at end of file diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/Dockerfile.functions b/stateful-functions-examples/stateful-functions-ridesharing-example/Dockerfile.functions new file mode 100644 index 00000000..8bd1f85b --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/Dockerfile.functions @@ -0,0 +1,19 @@ +# Copyright 2019 Ververica GmbH. +# +# 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. + +FROM stateful-functions + +RUN mkdir -p /opt/stateful-functions/modules/stateful-functions-ridesharing-example +COPY stateful-functions-ridesharing-example-functions/target/stateful-functions-ridesharing-example*jar /opt/stateful-functions/modules/stateful-functions-ridesharing-example/ +COPY stateful-functions-ridesharing-protocol/target/stateful-functions-ridesharing-protocol*jar /opt/stateful-functions/modules/stateful-functions-ridesharing-example/ diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/README.md b/stateful-functions-examples/stateful-functions-ridesharing-example/README.md new file mode 100644 index 00000000..7655d5f9 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/README.md @@ -0,0 +1,28 @@ +# The Ridesharing Example + +The Ridesharing example is a more complicated Stateful Functions example consisting of 4 different functions, each +corresponding to a real-world entity in a ridesharing scenario: `FnDriver`, `FnGeoCell`, `FnPassenger`, and `FnRide`. + +The whole example also includes a simulator program, which simulates real-world drivers and passengers sending +events to the Stateful Functions application. Driver simulations will be sending their location updates to the +application, while passenger simulations will be sending ride requests. + +## Running the example + +To run the example: + +``` +docker-compose build +docker-compose up +``` + +This starts both the simulator program and Stateful Functions ridesharing example application. + +After all the components have fully started, you can take a look at the web UI of the Flink Jobmanager to see the +application running, at `localhost:8081`. + +Then, you need to issue a request to the simulator program to start the simulation: + +``` +curl -X POST localhost:5656/api/start +``` diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/docker-compose.yml b/stateful-functions-examples/stateful-functions-ridesharing-example/docker-compose.yml new file mode 100644 index 00000000..11c567fd --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/docker-compose.yml @@ -0,0 +1,81 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +################################################################################ +version: "2.1" +services: + zookeeper: + image: wurstmeister/zookeeper + ports: + - "2181:2181" + kafka-broker: + image: wurstmeister/kafka:2.12-2.0.1 + ports: + - "9092:9092" + environment: + KAFKA_BROKER_ID: 1 + HOSTNAME_COMMAND: "route -n | awk '/UG[ \t]/{print $$2}'" + KAFKA_CREATE_TOPICS: "to-driver:1:1,to-passenger:1:1,from-driver:1:1,from-passenger:1:1" + KAFKA_ZOOKEEPER_CONNECT: zookeeper:2181 + depends_on: + - zookeeper + volumes: + - /var/run/docker.sock:/var/run/docker.sock + - ./kafka:/kafka + master: + build: + dockerfile: Dockerfile.functions + context: . + # uncomment to start from a savepoint + #command: -s /checkpoint-dir/savepoint/savepoint-bf101a-4e99820085fe + expose: + - "6123" + ports: + - "8081:8081" + environment: + - ROLE=master + - MASTER_HOST=master + volumes: + - ./checkpoint-dir:/checkpoint-dir + worker: + build: + dockerfile: Dockerfile.functions + context: . + expose: + - "6121" + - "6122" + depends_on: + - master + - kafka-broker + links: + - "master:master" + - "kafka-broker:kafka-broker" + environment: + - ROLE=worker + - MASTER_HOST=master + volumes: + - ./checkpoint-dir:/checkpoint-dir + simulator: + build: + context: ./stateful-functions-ridesharing-example-simulator + expose: + - "5656" + ports: + - "5656:5656" + depends_on: + - kafka-broker + - worker + diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/pom.xml b/stateful-functions-examples/stateful-functions-ridesharing-example/pom.xml new file mode 100644 index 00000000..02e504bf --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/pom.xml @@ -0,0 +1,38 @@ + + + + + stateful-functions-examples + com.ververica + 1.0-SNAPSHOT + .. + + 4.0.0 + pom + + stateful-functions-ridesharing-example + + + stateful-functions-ridesharing-protocol + stateful-functions-ridesharing-example-functions + stateful-functions-ridesharing-example-simulator + + + + \ No newline at end of file diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/pom.xml b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/pom.xml new file mode 100644 index 00000000..2bdef6bf --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/pom.xml @@ -0,0 +1,59 @@ + + + + + stateful-functions-ridesharing-example + com.ververica + 1.0-SNAPSHOT + + 4.0.0 + + stateful-functions-ridesharing-example-functions + + + + + com.ververica + stateful-functions-ridesharing-protocol + 1.0-SNAPSHOT + + + + + com.ververica + stateful-functions-sdk + ${project.version} + + + com.ververica + stateful-functions-kafka-io + ${project.version} + + + + + junit + junit + 4.12 + test + + + + + \ No newline at end of file diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/java/com/ververica/statefun/examples/ridesharing/FnDriver.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/java/com/ververica/statefun/examples/ridesharing/FnDriver.java new file mode 100644 index 00000000..3fa305a3 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/java/com/ververica/statefun/examples/ridesharing/FnDriver.java @@ -0,0 +1,140 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing; + +import com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide; +import com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup; +import com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage; +import com.ververica.statefun.examples.ridesharing.generated.JoinCell; +import com.ververica.statefun.examples.ridesharing.generated.LeaveCell; +import com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage; +import com.ververica.statefun.examples.ridesharing.generated.PickupPassenger; +import com.ververica.statefun.examples.ridesharing.generated.RideEnded; +import com.ververica.statefun.examples.ridesharing.generated.RideStarted; +import com.ververica.statefun.sdk.Context; +import com.ververica.statefun.sdk.FunctionType; +import com.ververica.statefun.sdk.annotations.Persisted; +import com.ververica.statefun.sdk.match.MatchBinder; +import com.ververica.statefun.sdk.match.StatefulMatchFunction; +import com.ververica.statefun.sdk.state.PersistedValue; + +public class FnDriver extends StatefulMatchFunction { + + static final FunctionType TYPE = new FunctionType(Identifiers.NAMESPACE, "driver"); + + @Persisted + private final PersistedValue currentRideId = PersistedValue.of("ride", String.class); + + @Persisted + private final PersistedValue location = PersistedValue.of("location", Integer.class); + + @Override + public void configure(MatchBinder binder) { + binder + .predicate(PickupPassenger.class, this::whenPickupIsNeeded) + .predicate( + InboundDriverMessage.class, + InboundDriverMessage::hasRideStarted, + this::whenRideHasStarted) + .predicate( + InboundDriverMessage.class, InboundDriverMessage::hasRideEnded, this::whenRideHasEnded) + .predicate( + InboundDriverMessage.class, + InboundDriverMessage::hasLocationUpdate, + this::whenLocationIsUpdated); + } + + private void whenPickupIsNeeded(Context context, PickupPassenger pickupPassenger) { + if (isTaken()) { + // this driver is currently in a ride, and therefore can't take any more + // passengers. + context.reply( + DriverRejectsPickup.newBuilder() + .setDriverId(context.self().id()) + .setRideId(context.caller().id()) + .build()); + return; + } + // We are called by the ride function, so we remember it's id for future communication. + currentRideId.set(context.caller().id()); + + // We also need to unregister ourselves from the current geo cell we belong to. + final int currentLocation = + location.getOrDefault(0); // drivers should have a location at this point. + context.send(FnGeoCell.TYPE, String.valueOf(currentLocation), LeaveCell.getDefaultInstance()); + + // reply to the ride, saying we are taking this passenger + context.reply( + DriverJoinsRide.newBuilder() + .setDriverId(context.self().id()) + .setDriverLocation(currentLocation) + .build()); + + // also send a command to the physical driver to pickup the passenger + context.send( + Identifiers.TO_OUTBOUND_DRIVER, + OutboundDriverMessage.newBuilder() + .setDriverId(context.self().id()) + .setPickupPassenger( + OutboundDriverMessage.PickupPassenger.newBuilder() + .setRideId(pickupPassenger.getPassengerId()) + .setStartGeoLocation(pickupPassenger.getPassengerStartCell()) + .setEndGeoLocation(pickupPassenger.getPassengerEndCell()) + .build()) + .build()); + } + + private void whenRideHasStarted(Context context, InboundDriverMessage ignored) { + context.send( + FnRide.TYPE, + currentRideId.get(), + RideStarted.newBuilder() + .setDriverId(context.self().id()) + .setDriverGeoCell(location.get()) + .build()); + } + + private void whenRideHasEnded(Context context, InboundDriverMessage ignored) { + context.send(FnRide.TYPE, currentRideId.get(), RideEnded.getDefaultInstance()); + currentRideId.clear(); + + // register at the current location as free driver. + Integer currentLocation = location.get(); + context.send(FnGeoCell.TYPE, String.valueOf(currentLocation), JoinCell.getDefaultInstance()); + } + + private void whenLocationIsUpdated(Context context, InboundDriverMessage locationUpdate) { + final int updated = locationUpdate.getLocationUpdate().getCurrentGeoCell(); + final int last = location.getOrDefault(-1); + if (last == -1) { + // this is the first time this driver gets a location update. + // so we notify the relevant geo cell function. + location.set(updated); + context.send(FnGeoCell.TYPE, String.valueOf(updated), JoinCell.getDefaultInstance()); + return; + } + if (last == updated) { + return; + } + location.set(updated); + } + + private boolean isTaken() { + String rideId = currentRideId.get(); + return rideId != null; + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/java/com/ververica/statefun/examples/ridesharing/FnGeoCell.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/java/com/ververica/statefun/examples/ridesharing/FnGeoCell.java new file mode 100644 index 00000000..e7846760 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/java/com/ververica/statefun/examples/ridesharing/FnGeoCell.java @@ -0,0 +1,88 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing; + +import com.ververica.statefun.examples.ridesharing.generated.*; +import com.ververica.statefun.sdk.Address; +import com.ververica.statefun.sdk.Context; +import com.ververica.statefun.sdk.FunctionType; +import com.ververica.statefun.sdk.StatefulFunction; +import com.ververica.statefun.sdk.annotations.Persisted; +import com.ververica.statefun.sdk.state.PersistedValue; + +public class FnGeoCell implements StatefulFunction { + static final FunctionType TYPE = new FunctionType(Identifiers.NAMESPACE, "geo-cell"); + + @Persisted + private final PersistedValue drivers = + PersistedValue.of("drivers", GeoCellState.class); + + @Override + public void invoke(Context context, Object input) { + Address caller = context.caller(); + if (input instanceof JoinCell) { + addDriver(caller); + } else if (input instanceof LeaveCell) { + removeDriver(caller); + } else if (input instanceof GetDriver) { + getDriver(context); + } else { + throw new IllegalStateException("Unknown message type " + input); + } + } + + private void getDriver(Context context) { + final GeoCellState state = drivers.get(); + + if (hasDriver(state)) { + String nextDriverId = state.getDriverIdList().get(0); + context.reply(DriverInCell.newBuilder().setDriverId(nextDriverId).build()); + } else { + context.reply(DriverInCell.newBuilder().build()); + } + } + + private void addDriver(Address driver) { + GeoCellState state = drivers.get(); + if (state == null) { + state = GeoCellState.newBuilder().addDriverId(driver.id()).build(); + } else { + state = state.toBuilder().addDriverId(driver.id()).build(); + } + drivers.set(state); + } + + private void removeDriver(Address driver) { + GeoCellState state = drivers.get(); + if (state == null) { + return; + } + GeoCellState.Builder nextState = state.toBuilder(); + nextState.clearDriverId(); + + for (String otherDriverID : state.getDriverIdList()) { + if (!otherDriverID.equals(driver.id())) { + nextState.addDriverId(otherDriverID); + } + } + drivers.set(nextState.build()); + } + + private boolean hasDriver(GeoCellState registeredDrivers) { + return registeredDrivers != null && !registeredDrivers.getDriverIdList().isEmpty(); + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/java/com/ververica/statefun/examples/ridesharing/FnPassenger.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/java/com/ververica/statefun/examples/ridesharing/FnPassenger.java new file mode 100644 index 00000000..c6e2b642 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/java/com/ververica/statefun/examples/ridesharing/FnPassenger.java @@ -0,0 +1,114 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing; + +import com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide; +import com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage; +import com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage.RequestRide; +import com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage; +import com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide; +import com.ververica.statefun.examples.ridesharing.generated.RideEnded; +import com.ververica.statefun.examples.ridesharing.generated.RideFailed; +import com.ververica.statefun.examples.ridesharing.generated.RideStarted; +import com.ververica.statefun.sdk.Context; +import com.ververica.statefun.sdk.FunctionType; +import com.ververica.statefun.sdk.match.MatchBinder; +import com.ververica.statefun.sdk.match.StatefulMatchFunction; +import java.util.concurrent.ThreadLocalRandom; + +public class FnPassenger extends StatefulMatchFunction { + + static final FunctionType TYPE = new FunctionType(Identifiers.NAMESPACE, "passenger"); + + @Override + public void configure(MatchBinder binder) { + binder + .predicate( + InboundPassengerMessage.class, + InboundPassengerMessage::hasRequestRide, + this::whenRideIsRequested) + .predicate(DriverJoinsRide.class, this::whenDriverJoins) + .predicate(RideFailed.class, this::whenRideFails) + .predicate(RideStarted.class, this::whenRideHasStarted) + .predicate(RideEnded.class, this::whenRideHasEnded); + } + + private void whenRideIsRequested(Context context, InboundPassengerMessage request) { + String passengerID = context.self().id(); + String rideId = "ride-" + ThreadLocalRandom.current().nextLong(); + + RequestRide rideRequest = request.getRequestRide(); + PassengerJoinsRide joinRide = + PassengerJoinsRide.newBuilder() + .setPassengerId(passengerID) + .setStartGeoCell(rideRequest.getStartGeoCell()) + .setEndGeoCell(rideRequest.getEndGeoCell()) + .build(); + + context.send(FnRide.TYPE, rideId, joinRide); + } + + private void whenRideHasEnded(Context context, RideEnded ignored) { + final OutboundPassengerMessage out = + OutboundPassengerMessage.newBuilder() + .setPassengerId(context.self().id()) + .setRideEnded(OutboundPassengerMessage.RideEnded.newBuilder().build()) + .build(); + + context.send(Identifiers.TO_PASSENGER_EGRESS, out); + } + + private void whenRideHasStarted(Context context, RideStarted rideStarted) { + final OutboundPassengerMessage out = + OutboundPassengerMessage.newBuilder() + .setPassengerId(context.self().id()) + .setRideStarted( + OutboundPassengerMessage.RideStarted.newBuilder() + .setDriverId(rideStarted.getDriverId()) + .build()) + .build(); + + context.send(Identifiers.TO_PASSENGER_EGRESS, out); + } + + private void whenDriverJoins(Context context, DriverJoinsRide message) { + final OutboundPassengerMessage out = + OutboundPassengerMessage.newBuilder() + .setPassengerId(context.self().id()) + .setDriverFound( + OutboundPassengerMessage.DriverHasBeenFound.newBuilder() + .setDriverId(message.getDriverId()) + .setDriverGeoCell(message.getDriverLocation()) + .build()) + .build(); + + context.send(Identifiers.TO_PASSENGER_EGRESS, out); + } + + private void whenRideFails(Context context, RideFailed rideFailed) { + final OutboundPassengerMessage out = + OutboundPassengerMessage.newBuilder() + .setPassengerId(context.self().id()) + .setRideFailed( + OutboundPassengerMessage.RideFailed.newBuilder() + .setRideId(rideFailed.getRideId()) + .build()) + .build(); + + context.send(Identifiers.TO_PASSENGER_EGRESS, out); + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/java/com/ververica/statefun/examples/ridesharing/FnRide.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/java/com/ververica/statefun/examples/ridesharing/FnRide.java new file mode 100644 index 00000000..7164ca56 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/java/com/ververica/statefun/examples/ridesharing/FnRide.java @@ -0,0 +1,126 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing; + +import com.ververica.statefun.examples.ridesharing.generated.DriverInCell; +import com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide; +import com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup; +import com.ververica.statefun.examples.ridesharing.generated.GetDriver; +import com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide; +import com.ververica.statefun.examples.ridesharing.generated.PickupPassenger; +import com.ververica.statefun.examples.ridesharing.generated.RideEnded; +import com.ververica.statefun.examples.ridesharing.generated.RideFailed; +import com.ververica.statefun.examples.ridesharing.generated.RideStarted; +import com.ververica.statefun.sdk.Context; +import com.ververica.statefun.sdk.FunctionType; +import com.ververica.statefun.sdk.annotations.Persisted; +import com.ververica.statefun.sdk.match.MatchBinder; +import com.ververica.statefun.sdk.match.StatefulMatchFunction; +import com.ververica.statefun.sdk.state.PersistedValue; + +final class FnRide extends StatefulMatchFunction { + + static final FunctionType TYPE = new FunctionType(Identifiers.NAMESPACE, "ride"); + + @Persisted + private final PersistedValue passenger = + PersistedValue.of("passenger", PassengerJoinsRide.class); + + @Persisted + private final PersistedValue driver = PersistedValue.of("driver", String.class); + + public void configure(MatchBinder binder) { + binder + .predicate(PassengerJoinsRide.class, this::whenPassengerJoins) + .predicate(DriverInCell.class, this::whenGeoCellResponds) + .predicate(DriverRejectsPickup.class, this::whenDriverRejectsPickup) + .predicate(DriverJoinsRide.class, this::whenDriverJoins) + .predicate(RideStarted.class, this::whenRideHasStarted) + .predicate(RideEnded.class, this::whenRideHasEnded); + } + + /** + * When a user joins a ride, we have to: 1. remember that user id 2. remember the starting + * location of that ride 3. contact the geo cell of the starting location, and ask for a free + * driver + */ + private void whenPassengerJoins(Context context, PassengerJoinsRide in) { + final String cellKey = String.valueOf(in.getStartGeoCell()); + passenger.set(in); + + context.send(FnGeoCell.TYPE, cellKey, GetDriver.getDefaultInstance()); + } + + /** + * Geo cell responds, it might respond with: - there is no driver, in that case we fail the ride - + * there is a driver, let's ask them to pickup the passenger. + */ + private void whenGeoCellResponds(Context context, DriverInCell in) { + final String driverId = in.getDriverId(); + final PassengerJoinsRide rideRequest = passenger.get(); + if (driverId != null && !driverId.isEmpty()) { + context.send( + FnDriver.TYPE, + driverId, + PickupPassenger.newBuilder() + .setPassengerId(rideRequest.getPassengerId()) + .setPassengerStartCell(rideRequest.getStartGeoCell()) + .setPassengerEndCell(rideRequest.getEndGeoCell()) + .build()); + return; + } + // no free drivers in this geo cell, at this example we just fail the ride + // but we can imagine that this is where we will expand our search to near geo cells + context.send(FnPassenger.TYPE, rideRequest.getPassengerId(), RideFailed.getDefaultInstance()); + + // by clearing our state, we essentially delete this instance of the ride actor + passenger.clear(); + } + + /** + * A driver might not be free, or for some other reason they can't take this ride, so we try + * another driver in that cell. + */ + @SuppressWarnings("unused") + private void whenDriverRejectsPickup(Context context, DriverRejectsPickup ignored) { + // try another driver, realistically we need to pass in a list of 'banned' drivers, + // so that the GeoCell will not offer us these drivers again, but in this example + // if a driver rejects a ride, it means that he is currently busy (and it would soon delete + // itself from the geo cell) + final int startGeoCell = passenger.get().getStartGeoCell(); + String cellKey = String.valueOf(startGeoCell); + context.send(FnGeoCell.TYPE, cellKey, GetDriver.getDefaultInstance()); + } + + /** A driver has taken this ride. */ + private void whenDriverJoins(Context context, DriverJoinsRide driverJoinRide) { + driver.set(context.caller().id()); + context.send(FnPassenger.TYPE, passenger.get().getPassengerId(), driverJoinRide); + } + + /** A driver has successfully picked up the passenger */ + private void whenRideHasStarted(Context context, RideStarted rideStarted) { + context.send(FnPassenger.TYPE, passenger.get().getPassengerId(), rideStarted); + } + + /** The driver has successfully reached the destination. */ + private void whenRideHasEnded(Context context, RideEnded rideEnded) { + context.send(FnPassenger.TYPE, passenger.get().getPassengerId(), rideEnded); + passenger.clear(); + driver.clear(); + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/java/com/ververica/statefun/examples/ridesharing/FunctionProvider.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/java/com/ververica/statefun/examples/ridesharing/FunctionProvider.java new file mode 100644 index 00000000..8f8f0776 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/java/com/ververica/statefun/examples/ridesharing/FunctionProvider.java @@ -0,0 +1,39 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing; + +import com.ververica.statefun.sdk.FunctionType; +import com.ververica.statefun.sdk.StatefulFunction; +import com.ververica.statefun.sdk.StatefulFunctionProvider; + +public class FunctionProvider implements StatefulFunctionProvider { + + @Override + public StatefulFunction functionOfType(FunctionType type) { + if (type.equals(FnPassenger.TYPE)) { + return new FnPassenger(); + } else if (type.equals(FnDriver.TYPE)) { + return new FnDriver(); + } else if (type.equals(FnRide.TYPE)) { + return new FnRide(); + } else if (type.equals(FnGeoCell.TYPE)) { + return new FnGeoCell(); + } else { + throw new IllegalArgumentException("Unknown type " + type); + } + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/java/com/ververica/statefun/examples/ridesharing/Identifiers.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/java/com/ververica/statefun/examples/ridesharing/Identifiers.java new file mode 100644 index 00000000..7c933af8 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/java/com/ververica/statefun/examples/ridesharing/Identifiers.java @@ -0,0 +1,41 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing; + +import com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage; +import com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage; +import com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage; +import com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage; +import com.ververica.statefun.sdk.io.EgressIdentifier; +import com.ververica.statefun.sdk.io.IngressIdentifier; + +final class Identifiers { + + static final String NAMESPACE = "com.ververica.statefun.examples.ridesharing"; + + static final IngressIdentifier FROM_PASSENGERS = + new IngressIdentifier<>(InboundPassengerMessage.class, NAMESPACE, "from-passenger"); + + static final IngressIdentifier FROM_DRIVER = + new IngressIdentifier<>(InboundDriverMessage.class, NAMESPACE, "from-driver"); + + static EgressIdentifier TO_PASSENGER_EGRESS = + new EgressIdentifier<>(NAMESPACE, "to-passenger", OutboundPassengerMessage.class); + + static EgressIdentifier TO_OUTBOUND_DRIVER = + new EgressIdentifier<>(NAMESPACE, "to-driver", OutboundDriverMessage.class); +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/java/com/ververica/statefun/examples/ridesharing/InboundDriverRouter.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/java/com/ververica/statefun/examples/ridesharing/InboundDriverRouter.java new file mode 100644 index 00000000..8b79663b --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/java/com/ververica/statefun/examples/ridesharing/InboundDriverRouter.java @@ -0,0 +1,28 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing; + +import com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage; +import com.ververica.statefun.sdk.io.Router; + +public class InboundDriverRouter implements Router { + + @Override + public void route(InboundDriverMessage message, Downstream downstream) { + downstream.forward(FnDriver.TYPE, message.getDriverId(), message); + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/java/com/ververica/statefun/examples/ridesharing/InboundPassengerRouter.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/java/com/ververica/statefun/examples/ridesharing/InboundPassengerRouter.java new file mode 100644 index 00000000..dc6b0380 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/java/com/ververica/statefun/examples/ridesharing/InboundPassengerRouter.java @@ -0,0 +1,29 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing; + +import com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage; +import com.ververica.statefun.sdk.io.Router; + +public class InboundPassengerRouter implements Router { + + @Override + public void route( + InboundPassengerMessage message, Downstream downstream) { + downstream.forward(FnPassenger.TYPE, message.getPassengerId(), message); + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/java/com/ververica/statefun/examples/ridesharing/KafkaSpecs.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/java/com/ververica/statefun/examples/ridesharing/KafkaSpecs.java new file mode 100644 index 00000000..833b7352 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/java/com/ververica/statefun/examples/ridesharing/KafkaSpecs.java @@ -0,0 +1,126 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing; + +import com.google.protobuf.InvalidProtocolBufferException; +import com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage; +import com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage; +import com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage; +import com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage; +import com.ververica.statefun.sdk.io.EgressSpec; +import com.ververica.statefun.sdk.io.IngressSpec; +import com.ververica.statefun.sdk.kafka.KafkaEgressBuilder; +import com.ververica.statefun.sdk.kafka.KafkaEgressSerializer; +import com.ververica.statefun.sdk.kafka.KafkaIngressBuilder; +import com.ververica.statefun.sdk.kafka.KafkaIngressDeserializer; +import java.nio.charset.StandardCharsets; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.producer.ProducerRecord; + +final class KafkaSpecs { + + private static final String KAFKA_SERVER = "kafka-broker:9092"; + private static final String TO_PASSENGER_KAFKA_TOPIC_NAME = "to-passenger"; + private static final String TO_DRIVER_TOPIC_NAME = "to-driver"; + private static final String FROM_DRIVER_TOPIC_NAME = "from-driver"; + private static final String FROM_PASSENGER_TOPIC_NAME = "from-passenger"; + + static IngressSpec FROM_DRIVER_SPEC = + KafkaIngressBuilder.forIdentifier(Identifiers.FROM_DRIVER) + .withKafkaAddress(KAFKA_SERVER) + .withTopic(FROM_DRIVER_TOPIC_NAME) + .withProperty(ConsumerConfig.GROUP_ID_CONFIG, "statefun-from-driver-group") + .withDeserializer(FromDriverDeserializer.class) + .build(); + + static IngressSpec FROM_PASSENGER_SPEC = + KafkaIngressBuilder.forIdentifier(Identifiers.FROM_PASSENGERS) + .withKafkaAddress(KAFKA_SERVER) + .withTopic(FROM_PASSENGER_TOPIC_NAME) + .withProperty(ConsumerConfig.GROUP_ID_CONFIG, "statefun-from-passenger-group") + .withDeserializer(FromPassengersDeserializer.class) + .build(); + + static EgressSpec TO_PASSENGER_SPEC = + KafkaEgressBuilder.forIdentifier(Identifiers.TO_PASSENGER_EGRESS) + .withKafkaAddress(KAFKA_SERVER) + .withSerializer(ToPassengersSerializer.class) + .build(); + + static EgressSpec TO_DRIVER_SPEC = + KafkaEgressBuilder.forIdentifier(Identifiers.TO_OUTBOUND_DRIVER) + .withKafkaAddress(KAFKA_SERVER) + .withSerializer(ToDriverSerializer.class) + .build(); + + private static final class FromDriverDeserializer + implements KafkaIngressDeserializer { + + private static final long serialVersionUID = 1; + + @Override + public InboundDriverMessage deserialize(ConsumerRecord input) { + try { + return InboundDriverMessage.parseFrom(input.value()); + } catch (InvalidProtocolBufferException ex) { + throw new RuntimeException(ex); + } + } + } + + private static final class FromPassengersDeserializer + implements KafkaIngressDeserializer { + + private static final long serialVersionUID = 1; + + @Override + public InboundPassengerMessage deserialize(ConsumerRecord input) { + try { + return InboundPassengerMessage.parseFrom(input.value()); + } catch (InvalidProtocolBufferException ex) { + throw new RuntimeException(ex); + } + } + } + + private static final class ToPassengersSerializer + implements KafkaEgressSerializer { + + private static final long serialVersionUID = 1; + + @Override + public ProducerRecord serialize(OutboundPassengerMessage message) { + byte[] key = message.getPassengerId().getBytes(StandardCharsets.UTF_8); + byte[] value = message.toByteArray(); + return new ProducerRecord<>(TO_PASSENGER_KAFKA_TOPIC_NAME, key, value); + } + } + + private static final class ToDriverSerializer + implements KafkaEgressSerializer { + + private static final long serialVersionUID = 1; + + @Override + public ProducerRecord serialize(OutboundDriverMessage message) { + byte[] key = message.getDriverId().getBytes(StandardCharsets.UTF_8); + byte[] value = message.toByteArray(); + return new ProducerRecord<>(TO_DRIVER_TOPIC_NAME, key, value); + } + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/java/com/ververica/statefun/examples/ridesharing/Module.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/java/com/ververica/statefun/examples/ridesharing/Module.java new file mode 100644 index 00000000..b383dc7b --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/java/com/ververica/statefun/examples/ridesharing/Module.java @@ -0,0 +1,40 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing; + +import com.ververica.statefun.sdk.spi.StatefulFunctionModule; +import java.util.Map; + +public class Module implements StatefulFunctionModule { + + @Override + public void configure(Map globalConfiguration, Binder binder) { + FunctionProvider provider = new FunctionProvider(); + binder.bindFunctionProvider(FnPassenger.TYPE, provider); + binder.bindFunctionProvider(FnDriver.TYPE, provider); + binder.bindFunctionProvider(FnRide.TYPE, provider); + binder.bindFunctionProvider(FnGeoCell.TYPE, provider); + + binder.bindIngress(KafkaSpecs.FROM_DRIVER_SPEC); + binder.bindIngressRouter(Identifiers.FROM_DRIVER, new InboundDriverRouter()); + binder.bindEgress(KafkaSpecs.TO_DRIVER_SPEC); + + binder.bindIngress(KafkaSpecs.FROM_PASSENGER_SPEC); + binder.bindIngressRouter(Identifiers.FROM_PASSENGERS, new InboundPassengerRouter()); + binder.bindEgress(KafkaSpecs.TO_PASSENGER_SPEC); + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/resources/META-INF/services/com.ververica.statefun.sdk.spi.StatefulFunctionModule b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/resources/META-INF/services/com.ververica.statefun.sdk.spi.StatefulFunctionModule new file mode 100644 index 00000000..3c2a4af7 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-functions/src/main/resources/META-INF/services/com.ververica.statefun.sdk.spi.StatefulFunctionModule @@ -0,0 +1,15 @@ +# Copyright 2019 Ververica GmbH. +# +# 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. + +com.ververica.statefun.examples.ridesharing.Module \ No newline at end of file diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/Dockerfile b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/Dockerfile new file mode 100644 index 00000000..7d803bcb --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/Dockerfile @@ -0,0 +1,14 @@ +# Start with a base image containing Java runtime +FROM openjdk:8-jdk-alpine + +# Add a volume pointing to /tmp +VOLUME /tmp + +# Make port 5656 available to the world outside this container +EXPOSE 5656 + +ADD target/*-simulator-*.jar simulator.jar +ADD application.yaml application.yaml + +# Run the jar file +ENTRYPOINT ["java","-Djava.security.egd=file:/dev/./urandom","-Dspring.profiles.active=dev","-Dspring.config.location=file:/application.yaml","-jar","/simulator.jar"] diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/application.yaml b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/application.yaml new file mode 100644 index 00000000..4762011e --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/application.yaml @@ -0,0 +1,43 @@ +# Copyright 2019 Ververica GmbH. +# +# 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. + +server: + port: 5656 + +security.auth.enabled: false +security.basic.enable: false +security.ignored: /** + +simulation: + grid: 50 + drivers: 2500 + passengers: 10 + +kafka: + topic: + from-driver: from-driver + to-driver: to-driver + from-passenger: from-passenger + to-passenger: to-passenger + listeners: + transactions.id: simulator-java-listener + bootstrap-servers: kafka-broker:9092 + +web-socket: + topic: + passenger: /topic/passenger + driver: /topic/driver + +management.endpoints.web.exposure.include: mappings, loggers + \ No newline at end of file diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/pom.xml b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/pom.xml new file mode 100644 index 00000000..a62de0a6 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/pom.xml @@ -0,0 +1,179 @@ + + + + + + + org.springframework.boot + spring-boot-starter-parent + 2.1.6.RELEASE + + + + 4.0.0 + + stateful-functions-ridesharing-example-simulator + + + UTF-8 + UTF-8 + 1.8 + 1.2.3 + 2.9.2 + + + + + + com.ververica + stateful-functions-ridesharing-protocol + 1.0-SNAPSHOT + + + + + org.springframework.boot + spring-boot-starter-web + + + org.springframework.boot + spring-boot-starter-websocket + + + + org.springframework.kafka + spring-kafka + + + org.springframework.boot + spring-boot-devtools + true + + + + + org.webjars + sockjs-client + 1.0.2 + + + org.webjars + stomp-websocket + 2.3.3 + + + + + + com.google.protobuf + protobuf-java + 3.8.0 + + + com.google.guava + guava + 28.0-jre + + + + org.projectlombok + lombok + 1.16.16 + provided + + + ch.qos.logback + logback-classic + ${logback.version} + + + + + org.springframework.boot + spring-boot-starter-test + test + + + + + + + maven-resources-plugin + + + copy-resources + process-classes + + copy-resources + + + ${basedir}/target/classes/static + + + build + + + + + + + + + org.springframework.boot + spring-boot-maven-plugin + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.8.1 + + 1.8 + 1.8 + + + + + + com.diffplug.spotless + spotless-maven-plugin + 1.20.0 + + + + 1.7 + + + + + + + + spotless-check + verify + + check + + + + + + + + \ No newline at end of file diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/Main.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/Main.java new file mode 100644 index 00000000..a658aba7 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/Main.java @@ -0,0 +1,31 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing.simulator; + +import org.springframework.boot.SpringApplication; +import org.springframework.boot.autoconfigure.EnableAutoConfiguration; +import org.springframework.boot.autoconfigure.SpringBootApplication; +import org.springframework.context.annotation.ComponentScan; + +@SpringBootApplication +@EnableAutoConfiguration +@ComponentScan("com.ververica") +public class Main { + public static void main(String[] args) { + SpringApplication.run(Main.class, args); + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/configurations/KafkaConsumerConfig.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/configurations/KafkaConsumerConfig.java new file mode 100644 index 00000000..364e093c --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/configurations/KafkaConsumerConfig.java @@ -0,0 +1,62 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing.simulator.configurations; + +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.springframework.beans.factory.annotation.Value; +import org.springframework.context.annotation.Bean; +import org.springframework.context.annotation.Configuration; +import org.springframework.kafka.annotation.EnableKafka; +import org.springframework.kafka.config.ConcurrentKafkaListenerContainerFactory; +import org.springframework.kafka.config.KafkaListenerContainerFactory; +import org.springframework.kafka.core.ConsumerFactory; +import org.springframework.kafka.core.DefaultKafkaConsumerFactory; +import org.springframework.kafka.listener.ConcurrentMessageListenerContainer; + +@EnableKafka +@Configuration +public class KafkaConsumerConfig { + private final String bootstrapServer; + + public KafkaConsumerConfig(@Value("${kafka.bootstrap-servers}") String bootstrapServer) { + this.bootstrapServer = Objects.requireNonNull(bootstrapServer); + } + + @Bean + public ConsumerFactory consumerFactory() { + Map props = new HashMap<>(); + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class); + props.put(ConsumerConfig.GROUP_ID_CONFIG, "simulator-java.group"); + props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest"); + return new DefaultKafkaConsumerFactory<>(props); + } + + @Bean + public KafkaListenerContainerFactory> + kafkaListenerContainerFactory(ConsumerFactory consumerFactory) { + ConcurrentKafkaListenerContainerFactory factory = + new ConcurrentKafkaListenerContainerFactory<>(); + factory.setConsumerFactory(consumerFactory); + return factory; + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/configurations/KafkaProducerConfig.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/configurations/KafkaProducerConfig.java new file mode 100644 index 00000000..ecac02cc --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/configurations/KafkaProducerConfig.java @@ -0,0 +1,57 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing.simulator.configurations; + +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.springframework.beans.factory.annotation.Value; +import org.springframework.context.annotation.Bean; +import org.springframework.context.annotation.Configuration; +import org.springframework.kafka.core.DefaultKafkaProducerFactory; +import org.springframework.kafka.core.KafkaTemplate; +import org.springframework.kafka.core.ProducerFactory; + +@Configuration +public class KafkaProducerConfig { + private final String bootstrapServers; + + public KafkaProducerConfig(@Value("${kafka.bootstrap-servers}") String bootstrapServers) { + this.bootstrapServers = Objects.requireNonNull(bootstrapServers); + } + + @Bean + public ProducerFactory producerFactory() { + Map props = new HashMap<>(); + props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); + props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + props.put("batch.size", 16384); + props.put("linger.ms", 1); + props.put("buffer.memory", 33554432); + + return new DefaultKafkaProducerFactory<>(props); + } + + @Bean + public KafkaTemplate kafkaTemplate( + ProducerFactory producerFactory) { + return new KafkaTemplate<>(producerFactory); + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/configurations/WebSocketConfig.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/configurations/WebSocketConfig.java new file mode 100644 index 00000000..43caaad4 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/configurations/WebSocketConfig.java @@ -0,0 +1,52 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing.simulator.configurations; + +import org.springframework.context.annotation.Bean; +import org.springframework.context.annotation.Configuration; +import org.springframework.messaging.simp.config.MessageBrokerRegistry; +import org.springframework.web.servlet.config.annotation.CorsRegistry; +import org.springframework.web.servlet.config.annotation.WebMvcConfigurer; +import org.springframework.web.servlet.config.annotation.WebMvcConfigurerAdapter; +import org.springframework.web.socket.config.annotation.EnableWebSocketMessageBroker; +import org.springframework.web.socket.config.annotation.StompEndpointRegistry; +import org.springframework.web.socket.config.annotation.WebSocketMessageBrokerConfigurer; + +@Configuration +@EnableWebSocketMessageBroker +public class WebSocketConfig implements WebSocketMessageBrokerConfigurer { + + @Bean + public WebMvcConfigurer corsConfigurer() { + return new WebMvcConfigurerAdapter() { + @Override + public void addCorsMappings(CorsRegistry registry) { + registry.addMapping("/**").allowedMethods("HEAD", "GET", "PUT", "POST", "DELETE", "PATCH"); + } + }; + } + + @Override + public void registerStompEndpoints(StompEndpointRegistry registry) { + registry.addEndpoint("/ws/simulation").setAllowedOrigins("*").withSockJS(); + } + + @Override + public void configureMessageBroker(MessageBrokerRegistry registry) { + registry.enableSimpleBroker("/topic"); + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/controllers/ApiController.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/controllers/ApiController.java new file mode 100644 index 00000000..455585a8 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/controllers/ApiController.java @@ -0,0 +1,46 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing.simulator.controllers; + +import com.ververica.statefun.examples.ridesharing.simulator.model.SimulationStartedEvent; +import com.ververica.statefun.examples.ridesharing.simulator.simulation.Simulation; +import java.util.Objects; +import lombok.extern.slf4j.Slf4j; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.http.MediaType; +import org.springframework.web.bind.annotation.PostMapping; +import org.springframework.web.bind.annotation.RequestMapping; +import org.springframework.web.bind.annotation.RestController; + +@RequestMapping(produces = MediaType.APPLICATION_JSON_UTF8_VALUE, path = "/api") +@RestController +@Slf4j +public class ApiController { + + private final Simulation simulation; + + @Autowired + public ApiController(Simulation simulation) { + this.simulation = Objects.requireNonNull(simulation); + } + + @PostMapping("/start") + public SimulationStartedEvent startSimulation() throws Exception { + simulation.start(); + return new SimulationStartedEvent(true); + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/model/SimulationStartedEvent.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/model/SimulationStartedEvent.java new file mode 100644 index 00000000..0cd86458 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/model/SimulationStartedEvent.java @@ -0,0 +1,31 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing.simulator.model; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@Builder +@NoArgsConstructor +@AllArgsConstructor +public class SimulationStartedEvent { + + boolean started; +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/model/WebsocketDriverEvent.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/model/WebsocketDriverEvent.java new file mode 100644 index 00000000..515a95c3 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/model/WebsocketDriverEvent.java @@ -0,0 +1,50 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing.simulator.model; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@Builder +@NoArgsConstructor +@AllArgsConstructor +public class WebsocketDriverEvent { + String driverId; + int currentLocation; + DriverStatus driverStatus; + RideInformation ride; + + public enum DriverStatus { + IDLE, + PICKUP, + ENROUTE, + RIDE_COMPLETED + } + + @Data + @Builder + @NoArgsConstructor + @AllArgsConstructor + public static class RideInformation { + String passengerId; + int pickupLocation; + int dropoffLocation; + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/model/WebsocketPassengerEvent.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/model/WebsocketPassengerEvent.java new file mode 100644 index 00000000..6c0bd774 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/model/WebsocketPassengerEvent.java @@ -0,0 +1,44 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing.simulator.model; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@Builder +@NoArgsConstructor +@AllArgsConstructor +public class WebsocketPassengerEvent { + String passengerId; + PassengerStatus status; + String rideId; + String driverId; + int startLocation; + int endLocation; + + public enum PassengerStatus { + IDLE, + REQUESTING, + WAITING_FOR_RIDE_TO_START, + FAIL, + RIDING, + DONE + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/services/KafkaConsumerService.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/services/KafkaConsumerService.java new file mode 100644 index 00000000..064f3ea9 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/services/KafkaConsumerService.java @@ -0,0 +1,51 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing.simulator.services; + +import com.google.protobuf.InvalidProtocolBufferException; +import com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage; +import com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage; +import com.ververica.statefun.examples.ridesharing.simulator.simulation.messaging.Communication; +import java.util.Objects; +import lombok.extern.slf4j.Slf4j; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.kafka.annotation.KafkaListener; +import org.springframework.messaging.handler.annotation.Payload; +import org.springframework.stereotype.Service; + +@Service +@Slf4j +public class KafkaConsumerService { + private final Communication simulation; + + @Autowired + public KafkaConsumerService(Communication simulation) { + this.simulation = Objects.requireNonNull(simulation); + } + + @KafkaListener(topics = "${kafka.topic.to-passenger}", groupId = "passengers") + public void toPassenger(@Payload byte[] message) throws InvalidProtocolBufferException { + OutboundPassengerMessage passengerMessage = OutboundPassengerMessage.parseFrom(message); + simulation.incomingPassengerEvent(passengerMessage); + } + + @KafkaListener(topics = "${kafka.topic.to-driver}", groupId = "drivers") + public void toDriver(@Payload byte[] message) throws InvalidProtocolBufferException { + OutboundDriverMessage driverMessage = OutboundDriverMessage.parseFrom(message); + simulation.incomingDriverEvent(driverMessage); + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/services/KafkaDriverPublisher.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/services/KafkaDriverPublisher.java new file mode 100644 index 00000000..816563f0 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/services/KafkaDriverPublisher.java @@ -0,0 +1,62 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing.simulator.services; + +import com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage; +import java.nio.charset.StandardCharsets; +import java.util.function.Consumer; +import lombok.extern.slf4j.Slf4j; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.beans.factory.annotation.Value; +import org.springframework.kafka.core.KafkaTemplate; +import org.springframework.kafka.support.SendResult; +import org.springframework.stereotype.Service; +import org.springframework.util.concurrent.ListenableFuture; +import org.springframework.util.concurrent.ListenableFutureCallback; + +@Service +@Slf4j +public class KafkaDriverPublisher implements Consumer { + private final String topic; + private KafkaTemplate kafkaTemplate; + + @Autowired + public KafkaDriverPublisher( + KafkaTemplate kafkaTemplateForJson, + @Value("${kafka.topic.from-driver}") String topic) { + this.kafkaTemplate = kafkaTemplateForJson; + this.topic = topic; + } + + @Override + public void accept(InboundDriverMessage driver) { + byte[] keyBytes = driver.getDriverId().getBytes(StandardCharsets.UTF_8); + ListenableFuture> future = + kafkaTemplate.send(topic, keyBytes, driver.toByteArray()); + + future.addCallback( + new ListenableFutureCallback>() { + @Override + public void onFailure(Throwable throwable) { + log.warn("Failed sending an event to kafka", throwable); + } + + @Override + public void onSuccess(SendResult objectObjectSendResult) {} + }); + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/services/KafkaPassengerPublisher.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/services/KafkaPassengerPublisher.java new file mode 100644 index 00000000..4fadeb85 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/services/KafkaPassengerPublisher.java @@ -0,0 +1,64 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing.simulator.services; + +import com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage; +import java.nio.charset.StandardCharsets; +import java.util.Objects; +import java.util.function.Consumer; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.beans.factory.annotation.Value; +import org.springframework.kafka.core.KafkaTemplate; +import org.springframework.kafka.support.SendResult; +import org.springframework.stereotype.Service; +import org.springframework.util.concurrent.ListenableFutureCallback; + +@Service +@Slf4j +public class KafkaPassengerPublisher implements Consumer { + private final String topic; + private final KafkaTemplate kafkaTemplate; + + @Autowired + public KafkaPassengerPublisher( + KafkaTemplate kafkaTemplate, + @Value("${kafka.topic.from-passenger}") String topic) { + this.kafkaTemplate = Objects.requireNonNull(kafkaTemplate); + this.topic = Objects.requireNonNull(topic); + } + + @Override + public void accept(InboundPassengerMessage passenger) { + byte[] bytes = passenger.getPassengerId().getBytes(StandardCharsets.UTF_8); + kafkaTemplate + .send(topic, bytes, passenger.toByteArray()) + .addCallback( + new ListenableFutureCallback>() { + @Override + public void onFailure(@NonNull Throwable throwable) { + log.warn("couldn't send passenger data.", throwable); + } + + @Override + public void onSuccess(SendResult objectObjectSendResult) { + log.info("Sent passenger data"); + } + }); + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/Driver.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/Driver.java new file mode 100644 index 00000000..d4d800a4 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/Driver.java @@ -0,0 +1,296 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing.simulator.simulation; + +import com.google.common.base.Preconditions; +import com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage; +import com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage; +import com.ververica.statefun.examples.ridesharing.simulator.model.WebsocketDriverEvent; +import com.ververica.statefun.examples.ridesharing.simulator.simulation.engine.LifecycleMessages.Initialization; +import com.ververica.statefun.examples.ridesharing.simulator.simulation.engine.LifecycleMessages.TimeTick; +import com.ververica.statefun.examples.ridesharing.simulator.simulation.engine.Simulatee; +import java.util.concurrent.ThreadLocalRandom; +import javax.annotation.Nullable; + +public class Driver implements Simulatee { + private final String driverId; + private final DriverMessaging messaging; + private final StateMachine stateMachine; + private final int gridDimension; + + private int currentLocation; + + @Nullable private WebsocketDriverEvent.RideInformation rideInformation; + + Driver(String driverId, DriverMessaging messaging, int gridDimension, int startLocation) { + this.driverId = driverId; + this.messaging = messaging; + this.gridDimension = gridDimension; + this.currentLocation = startLocation; + + stateMachine = new StateMachine<>(WebsocketDriverEvent.DriverStatus.IDLE); + + // we don't have anything to do on initialization + stateMachine.withState( + WebsocketDriverEvent.DriverStatus.IDLE, + Initialization.class, + unused -> WebsocketDriverEvent.DriverStatus.IDLE); + + // idle heartbeat + stateMachine.withState(WebsocketDriverEvent.DriverStatus.IDLE, TimeTick.class, this::heartbeat); + + // pickup request + stateMachine.withState( + WebsocketDriverEvent.DriverStatus.IDLE, + OutboundDriverMessage.class, + OutboundDriverMessage::hasPickupPassenger, + this::pickupPassenger); + + stateMachine.withState( + WebsocketDriverEvent.DriverStatus.PICKUP, TimeTick.class, this::preformPickup); + stateMachine.withState( + WebsocketDriverEvent.DriverStatus.ENROUTE, TimeTick.class, this::preformRoute); + } + + /** send periodic heart beats when idle */ + private WebsocketDriverEvent.DriverStatus heartbeat( + @SuppressWarnings("unused") TimeTick ignored) { + + // notify application + messaging.outgoingDriverEvent( + InboundDriverMessage.newBuilder() + .setDriverId(driverId) + .setLocationUpdate( + InboundDriverMessage.LocationUpdate.newBuilder() + .setCurrentGeoCell(currentLocation) + .build()) + .build()); + + // notify the websocket + messaging.broadcastDriverSimulationEvent( + WebsocketDriverEvent.builder() + .currentLocation(currentLocation) + .driverId(driverId) + .driverStatus(WebsocketDriverEvent.DriverStatus.IDLE) + .ride(null) + .build()); + + return WebsocketDriverEvent.DriverStatus.IDLE; + } + + /** receive a pickup command and start riding to the passenger */ + private WebsocketDriverEvent.DriverStatus pickupPassenger(OutboundDriverMessage message) { + OutboundDriverMessage.PickupPassenger pickup = message.getPickupPassenger(); + + // capture ride info from the pickup message + this.rideInformation = + WebsocketDriverEvent.RideInformation.builder() + .passengerId(pickup.getRideId()) // TODO: fix this at the application side. + .pickupLocation(pickup.getStartGeoLocation()) + .dropoffLocation(pickup.getEndGeoLocation()) + .build(); + + // notify the websocket + messaging.broadcastDriverSimulationEvent( + WebsocketDriverEvent.builder() + .ride(rideInformation) + .driverStatus(WebsocketDriverEvent.DriverStatus.PICKUP) + .driverId(driverId) + .currentLocation(currentLocation) + .build()); + + return WebsocketDriverEvent.DriverStatus.PICKUP; + } + + private WebsocketDriverEvent.DriverStatus preformPickup( + @SuppressWarnings("unused") TimeTick ignored) { + Preconditions.checkState(rideInformation != null, "should have ride information."); + if (currentLocation == rideInformation.getPickupLocation()) { + // we have reached to the passenger, lets pick him up! + + messaging.broadcastDriverSimulationEvent( + WebsocketDriverEvent.builder() + .currentLocation(currentLocation) + .driverId(driverId) + .driverStatus(WebsocketDriverEvent.DriverStatus.ENROUTE) + .ride(rideInformation) + .build()); + + messaging.outgoingDriverEvent( + InboundDriverMessage.newBuilder() + .setDriverId(driverId) + .setRideStarted(InboundDriverMessage.RideStarted.getDefaultInstance()) + .build()); + + return WebsocketDriverEvent.DriverStatus.ENROUTE; + } + + // we need to advance toward the passenger + int selfX = currentLocation / gridDimension; + int selfY = currentLocation % gridDimension; + final int passX = rideInformation.getPickupLocation() / gridDimension; + final int passY = rideInformation.getPickupLocation() % gridDimension; + + if (ThreadLocalRandom.current().nextBoolean()) { + // + // advance in X + // + if (selfX > passX) { + selfX--; + } else if (selfX < passX) { + selfX++; + } + } else { + // + // advance in Y + // + if (selfY > passY) { + selfY--; + } else if (selfY < passY) { + selfY++; + } + } + + currentLocation = selfX * gridDimension + selfY; + + // send a heartbeat with our new location + // notify application of our new location + messaging.outgoingDriverEvent( + InboundDriverMessage.newBuilder() + .setDriverId(driverId) + .setLocationUpdate( + InboundDriverMessage.LocationUpdate.newBuilder() + .setCurrentGeoCell(currentLocation) + .build()) + .build()); + + // notify the websocket + messaging.broadcastDriverSimulationEvent( + WebsocketDriverEvent.builder() + .currentLocation(currentLocation) + .driverId(driverId) + .driverStatus(WebsocketDriverEvent.DriverStatus.PICKUP) + .ride(rideInformation) + .build()); + + return WebsocketDriverEvent.DriverStatus.PICKUP; + } + + private WebsocketDriverEvent.DriverStatus preformRoute( + @SuppressWarnings("unused") TimeTick ignored) { + Preconditions.checkState(rideInformation != null, "should have ride information."); + if (currentLocation == rideInformation.getDropoffLocation()) { + // done! + + // notify websocket + messaging.broadcastDriverSimulationEvent( + WebsocketDriverEvent.builder() + .currentLocation(currentLocation) + .driverId(driverId) + .driverStatus( + WebsocketDriverEvent.DriverStatus + .RIDE_COMPLETED) // TODO: should we send ride done? + .ride(rideInformation) + .build()); + + // notify application + messaging.outgoingDriverEvent( + InboundDriverMessage.newBuilder() + .setDriverId(driverId) + .setRideEnded( + InboundDriverMessage.RideEnded.newBuilder() + .setRideId(rideInformation.getPassengerId()) + .build()) + .build()); + + rideInformation = null; + + // we switch back to idle + return WebsocketDriverEvent.DriverStatus.IDLE; + } + + // we need to advance toward the passenger's dropoff location + int selfX = currentLocation / gridDimension; + int selfY = currentLocation % gridDimension; + final int passX = rideInformation.getDropoffLocation() / gridDimension; + final int passY = rideInformation.getDropoffLocation() % gridDimension; + + if (ThreadLocalRandom.current().nextBoolean()) { + // + // advance in X + // + if (selfX > passX) { + selfX--; + } else if (selfX < passX) { + selfX++; + } + } else { + // + // advance in Y + // + if (selfY > passY) { + selfY--; + } else if (selfY < passY) { + selfY++; + } + } + + currentLocation = selfX * gridDimension + selfY; + + // send a heartbeat with our new location + // notify application of our new location + messaging.outgoingDriverEvent( + InboundDriverMessage.newBuilder() + .setDriverId(driverId) + .setLocationUpdate( + InboundDriverMessage.LocationUpdate.newBuilder() + .setCurrentGeoCell(currentLocation) + .build()) + .build()); + + // notify the websocket + messaging.broadcastDriverSimulationEvent( + WebsocketDriverEvent.builder() + .currentLocation(currentLocation) + .driverId(driverId) + .driverStatus(WebsocketDriverEvent.DriverStatus.ENROUTE) + .ride(rideInformation) + .build()); + + return WebsocketDriverEvent.DriverStatus.ENROUTE; + } + + @Override + public String id() { + return driverId; + } + + @Override + public boolean isDone() { + return stateMachine.isAtTerminalState(); + } + + @Override + public void apply(Object event) { + stateMachine.apply(event); + } + + @Override + public boolean needReschedule() { + return true; // the driver never rests. + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/DriverMessaging.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/DriverMessaging.java new file mode 100644 index 00000000..4f9c77f4 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/DriverMessaging.java @@ -0,0 +1,33 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing.simulator.simulation; + +import com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage; +import com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage; +import com.ververica.statefun.examples.ridesharing.simulator.model.WebsocketDriverEvent; + +public interface DriverMessaging { + + /** handle an event that was sent from application to the simulator */ + void incomingDriverEvent(OutboundDriverMessage message); + + /** send an event to the application */ + void outgoingDriverEvent(InboundDriverMessage message); + + /** notify to whoever is listening that there is a driver state change */ + void broadcastDriverSimulationEvent(WebsocketDriverEvent driverEvent); +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/Passenger.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/Passenger.java new file mode 100644 index 00000000..813b1d32 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/Passenger.java @@ -0,0 +1,208 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing.simulator.simulation; + +import com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage; +import com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage; +import com.ververica.statefun.examples.ridesharing.simulator.model.WebsocketPassengerEvent; +import com.ververica.statefun.examples.ridesharing.simulator.model.WebsocketPassengerEvent.PassengerStatus; +import com.ververica.statefun.examples.ridesharing.simulator.simulation.engine.LifecycleMessages; +import com.ververica.statefun.examples.ridesharing.simulator.simulation.engine.Simulatee; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class Passenger implements Simulatee { + private final PassengerMessaging simulation; + private final StateMachine stateMachine; + private final String id; + private final int startCell; + private final int endCell; + + private String driverId; + + Passenger(PassengerMessaging simulation, String id, int startCell, int endCell) { + this.simulation = simulation; + this.id = id; + this.startCell = startCell; + this.endCell = endCell; + this.stateMachine = passengerStateMachine(); + } + + /** + * when this state machine initializes we send a ride request, and move to the next state ({@link + * PassengerStatus#REQUESTING} + */ + @SuppressWarnings("unused") + private PassengerStatus init(LifecycleMessages.Initialization ignored) { + + final InboundPassengerMessage rideRequest = + InboundPassengerMessage.newBuilder() + .setPassengerId(id) + .setRequestRide( + InboundPassengerMessage.RequestRide.newBuilder() + .setStartGeoCell(startCell) + .setEndGeoCell(endCell) + .build()) + .build(); + + // send to application + simulation.outgoingPassengerEvent(rideRequest); + + // send to the web socket + simulation.broadcastPassengerSimulationEvent( + WebsocketPassengerEvent.builder() + .passengerId(id) + .startLocation(startCell) + .endLocation(endCell) + .status(PassengerStatus.REQUESTING) + .rideId("") // we don't have it yet. + .build()); + + // next state would be requesting a ride + return PassengerStatus.REQUESTING; + } + + /** The ride failed, couldn't find a driver nearby. */ + private PassengerStatus rideFailed(OutboundPassengerMessage message) { + final String failedRideId = message.getRideFailed().getRideId(); + // + // notify the websocket + // + simulation.broadcastPassengerSimulationEvent( + WebsocketPassengerEvent.builder() + .passengerId(id) + .startLocation(startCell) + .endLocation(endCell) + .status(PassengerStatus.FAIL) + .rideId(failedRideId) + .build()); + + return PassengerStatus.FAIL; + } + + /** A driver was found, now waiting for the pickup to happen. */ + private PassengerStatus driverFound(OutboundPassengerMessage message) { + OutboundPassengerMessage.DriverHasBeenFound driverFound = message.getDriverFound(); + + simulation.broadcastPassengerSimulationEvent( + WebsocketPassengerEvent.builder() + .rideId("") // TODO: ? + .passengerId(id) + .startLocation(startCell) + .endLocation(endCell) + .driverId(driverFound.getDriverId()) + .status(PassengerStatus.WAITING_FOR_RIDE_TO_START) + .build()); + + // capture the driver id + driverId = driverFound.getDriverId(); + + return PassengerStatus.WAITING_FOR_RIDE_TO_START; + } + + private PassengerStatus rideStarted(OutboundPassengerMessage message) { + OutboundPassengerMessage.RideStarted rideStarted = message.getRideStarted(); + + simulation.broadcastPassengerSimulationEvent( + WebsocketPassengerEvent.builder() + .rideId("") // TODO: ? + .passengerId(id) + .startLocation(startCell) + .endLocation(endCell) + .driverId(rideStarted.getDriverId()) + .status(PassengerStatus.RIDING) + .build()); + + return PassengerStatus.RIDING; + } + + @SuppressWarnings("unused") + private PassengerStatus rideEnded(OutboundPassengerMessage ignored) { + simulation.broadcastPassengerSimulationEvent( + WebsocketPassengerEvent.builder() + .rideId("") // TODO: ? + .passengerId(id) + .startLocation(startCell) + .endLocation(endCell) + .driverId(driverId) + .status(PassengerStatus.DONE) + .build()); + + return PassengerStatus.DONE; + } + + private StateMachine passengerStateMachine() { + StateMachine stateMachine = new StateMachine<>(PassengerStatus.IDLE); + + stateMachine.withTerminalState(PassengerStatus.FAIL); + stateMachine.withTerminalState(PassengerStatus.DONE); + + // send the req + stateMachine.withState( + PassengerStatus.IDLE, LifecycleMessages.Initialization.class, this::init); + + // req success + stateMachine.withState( + PassengerStatus.REQUESTING, + OutboundPassengerMessage.class, + OutboundPassengerMessage::hasDriverFound, + this::driverFound); + + // req failure (terminal) + stateMachine.withState( + PassengerStatus.REQUESTING, + OutboundPassengerMessage.class, + OutboundPassengerMessage::hasRideFailed, + this::rideFailed); + + // ride started + stateMachine.withState( + PassengerStatus.WAITING_FOR_RIDE_TO_START, + OutboundPassengerMessage.class, + OutboundPassengerMessage::hasRideStarted, + this::rideStarted); + + // ride finished (terminal) + stateMachine.withState( + PassengerStatus.RIDING, + OutboundPassengerMessage.class, + OutboundPassengerMessage::hasRideEnded, + this::rideEnded); + + return stateMachine; + } + + @Override + public String id() { + return id; + } + + @Override + public boolean isDone() { + return stateMachine.isAtTerminalState(); + } + + @Override + public void apply(Object event) { + stateMachine.apply(event); + } + + @Override + public boolean needReschedule() { + return false; + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/PassengerMessaging.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/PassengerMessaging.java new file mode 100644 index 00000000..5e7a67c0 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/PassengerMessaging.java @@ -0,0 +1,33 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing.simulator.simulation; + +import com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage; +import com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage; +import com.ververica.statefun.examples.ridesharing.simulator.model.WebsocketPassengerEvent; + +public interface PassengerMessaging { + + /** handle an event that was sent from application to the simulator */ + void incomingPassengerEvent(OutboundPassengerMessage message); + + /** send an event to the application */ + void outgoingPassengerEvent(InboundPassengerMessage message); + + /** notify whoever is listening that there is a passenger state change */ + void broadcastPassengerSimulationEvent(WebsocketPassengerEvent passengerEvent); +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/Simulation.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/Simulation.java new file mode 100644 index 00000000..1eec9760 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/Simulation.java @@ -0,0 +1,140 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing.simulator.simulation; + +import com.google.common.util.concurrent.RateLimiter; +import com.ververica.statefun.examples.ridesharing.simulator.simulation.engine.Scheduler; +import com.ververica.statefun.examples.ridesharing.simulator.simulation.engine.Simulatee; +import com.ververica.statefun.examples.ridesharing.simulator.simulation.messaging.Communication; +import java.util.Objects; +import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; +import lombok.extern.slf4j.Slf4j; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.beans.factory.annotation.Value; +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class Simulation { + private final Scheduler scheduler; + private final Communication communication; + private final int gridDimension; + private final int driverCount; + private final int passengerCount; + + @Autowired + public Simulation( + Scheduler scheduler, + Communication communication, + @Value("${simulation.grid}") int gridDimension, + @Value("${simulation.drivers}") int driverCount, + @Value("${simulation.passengers}") int passengerCount) { + this.scheduler = Objects.requireNonNull(scheduler); + this.communication = Objects.requireNonNull(communication); + this.gridDimension = gridDimension; + this.driverCount = driverCount; + this.passengerCount = passengerCount; + } + + public void start() { + if (!scheduler.start()) { + log.info("Already started"); + return; + } + // + // create the drivers + // + createDrivers(); + createPassengers(); + } + + private void createDrivers() { + final ThreadLocalRandom random = ThreadLocalRandom.current(); + for (int i = 0; i < driverCount; i++) { + final String id = "driver-" + UUID.randomUUID(); + final int startLocation = random.nextInt(gridDimension * gridDimension); + + Driver driver = new Driver(id, communication, gridDimension, startLocation); + scheduler.add(driver); + } + } + + private void createPassengers() { + Thread t = new Thread(new PassengerLoop()); + t.setDaemon(true); + t.start(); + } + + private final class PassengerLoop implements Runnable { + + @SuppressWarnings("InfiniteLoopStatement") + @Override + public void run() { + final ThreadLocalRandom random = ThreadLocalRandom.current(); + while (true) { + try { + createBatchOfPassengers(random); + log.info("produced passengers, sleeping..."); + } catch (Throwable t) { + log.info("Error in passenger loop, recovering ... ", t); + } + } + } + + private void createBatchOfPassengers(ThreadLocalRandom random) { + @SuppressWarnings("UnstableApiUsage") + RateLimiter rate = RateLimiter.create(2); + for (int j = 0; j < passengerCount; j++) { + rate.acquire(); + int startCell, endCell; + do { + startCell = random.nextInt(gridDimension * gridDimension); + int dx = random.nextInt(-10, 10); + int dy = random.nextInt(-10, 10); + endCell = moveSlightly(startCell, dx, dy); + } while (startCell == endCell); + + String id = "passenger-" + UUID.randomUUID(); + Simulatee passenger = new Passenger(communication, id, startCell, endCell); + scheduler.add(passenger); + } + } + + private int moveSlightly(int startCell, int dx, int dy) { + int x = startCell / gridDimension; + int y = startCell % gridDimension; + + x += dx; + y += dy; + + if (x < 0) { + x = 0; + } + if (x >= gridDimension) { + x = gridDimension - 1; + } + if (y < 0) { + y = 0; + } + if (y >= gridDimension) { + y = gridDimension - 1; + } + return x * gridDimension + y; + } + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/StateMachine.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/StateMachine.java new file mode 100644 index 00000000..97ba4961 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/StateMachine.java @@ -0,0 +1,120 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing.simulator.simulation; + +import static com.google.common.base.Preconditions.checkState; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.function.Predicate; + +final class StateMachine> { + + public interface Behaviour, E> { + S apply(E event); + } + + private final class Case { + private final Class eventType; + private final Predicate predicate; + private final Behaviour action; + + Case(Class eventType, Predicate predicate, Behaviour action) { + this.eventType = eventType; + this.predicate = predicate; + this.action = action; + } + + Optional tryApply(Object event) { + if (!eventType.isInstance(event)) { + return Optional.empty(); + } + final E e = eventType.cast(event); + if (!predicate.test(e)) { + return Optional.empty(); + } + final S nextState = action.apply(e); + return Optional.ofNullable(nextState); + } + } + + private final Map>> cases = new HashMap<>(); + + private S current; + private final Set terminalStates = new HashSet<>(); + + StateMachine(S initialState) { + this.current = initialState; + } + + @SuppressWarnings("UnusedReturnValue") + StateMachine withState(S state, Class eventType, Behaviour action) { + return withState(state, eventType, unused -> true, action); + } + + @SuppressWarnings("UnusedReturnValue") + StateMachine withState( + S state, Class eventType, Predicate guard, Behaviour action) { + List> stateCases = cases.computeIfAbsent(state, unused -> new ArrayList<>()); + stateCases.add(new Case<>(eventType, guard, action)); + return this; + } + + void withTerminalState(S terminalState) { + this.terminalStates.add(terminalState); + } + + void apply(Object event) { + checkState(!terminalStates.contains(current), "Already at a terminal state " + current); + Optional next = tryApply(event); + if (!next.isPresent()) { + throw new IllegalArgumentException( + "Don't know how to handle the event " + + safeGetClass(event) + + " in state " + + current + + " event:" + + event); + } + current = next.get(); + } + + boolean isAtTerminalState() { + return terminalStates.contains(current); + } + + private Optional tryApply(Object event) { + final List> stateCases = cases.getOrDefault(current, Collections.emptyList()); + for (Case c : stateCases) { + final Optional nextState = c.tryApply(event); + if (nextState.isPresent()) { + return nextState; + } + } + return Optional.empty(); + } + + private static String safeGetClass(Object event) { + return event == null ? "" : event.getClass().toString(); + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/engine/DaemonThreadFactory.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/engine/DaemonThreadFactory.java new file mode 100644 index 00000000..a70e7de4 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/engine/DaemonThreadFactory.java @@ -0,0 +1,31 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing.simulator.simulation.engine; + +import java.util.concurrent.ThreadFactory; +import javax.annotation.Nonnull; + +public enum DaemonThreadFactory implements ThreadFactory { + INSTANCE; + + @Override + public Thread newThread(@Nonnull Runnable r) { + Thread t = new Thread(r); + t.setDaemon(true); + return t; + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/engine/EventLoop.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/engine/EventLoop.java new file mode 100644 index 00000000..a6a7adca --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/engine/EventLoop.java @@ -0,0 +1,81 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing.simulator.simulation.engine; + +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +final class EventLoop implements Runnable { + private final ReadySet readySet = new ReadySet(); + private final ConcurrentHashMap globalTasks; + private final ScheduledExecutorService timerExecutor = + Executors.newScheduledThreadPool(1, DaemonThreadFactory.INSTANCE); + + EventLoop(ConcurrentHashMap globalTasks) { + this.globalTasks = globalTasks; + } + + void addToReadySet(Task task) { + readySet.add(task); + } + + @SuppressWarnings("InfiniteLoopStatement") + @Override + public void run() { + while (true) { + try { + processTask(); + } catch (Throwable t) { + log.warn("Exception caught in the main event loop, recovering ...", t); + } + } + } + + private void processTask() throws InterruptedException { + final Task task = readySet.take(); + try { + task.processEnqueued(); + if (task.isDone()) { + // this entity is done and can be removed from the system + log.info("Completed entity {}", task.id()); + globalTasks.remove(task.id()); + } else if (task.needReschedule()) { + scheduleLater(task); + } + } catch (Throwable ex) { + log.warn("Error processing an event for {}", task.id(), ex); + } + } + + private void scheduleLater(Task e) { + final int jitter = ThreadLocalRandom.current().nextInt(500, 1100); + Objects.requireNonNull(e); + timerExecutor.schedule( + () -> { + e.enqueue(LifecycleMessages.timeTick()); + readySet.add(e); + }, + jitter, + TimeUnit.MILLISECONDS); + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/engine/LifecycleMessages.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/engine/LifecycleMessages.java new file mode 100644 index 00000000..cbabb300 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/engine/LifecycleMessages.java @@ -0,0 +1,37 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing.simulator.simulation.engine; + +public final class LifecycleMessages { + private LifecycleMessages() {} + + static Initialization initialization() { + return Initialization.INSTANCE; + } + + static TimeTick timeTick() { + return TimeTick.INSTANCE; + } + + public enum Initialization { + INSTANCE + } + + public enum TimeTick { + INSTANCE + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/engine/ReadySet.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/engine/ReadySet.java new file mode 100644 index 00000000..84b6b4e2 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/engine/ReadySet.java @@ -0,0 +1,58 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing.simulator.simulation.engine; + +import java.util.ArrayDeque; +import java.util.HashSet; +import java.util.Objects; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.ReentrantLock; + +final class ReadySet { + private final ReentrantLock lock = new ReentrantLock(true); + private final Condition hasWork = lock.newCondition(); + private final HashSet enqueuedIds = new HashSet<>(); + private final ArrayDeque ready = new ArrayDeque<>(4096); + + void add(Task e) { + Objects.requireNonNull(e); + lock.lock(); + try { + if (!enqueuedIds.add(e.id())) { + return; + } + ready.addLast(e); + hasWork.signalAll(); + } finally { + lock.unlock(); + } + } + + Task take() throws InterruptedException { + lock.lock(); + try { + while (ready.isEmpty()) { + hasWork.await(); + } + final Task e = ready.poll(); + enqueuedIds.remove(e.id()); + return e; + } finally { + lock.unlock(); + } + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/engine/Scheduler.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/engine/Scheduler.java new file mode 100644 index 00000000..34538a68 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/engine/Scheduler.java @@ -0,0 +1,87 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing.simulator.simulation.engine; + +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicBoolean; +import javax.annotation.Nullable; +import lombok.extern.slf4j.Slf4j; +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class Scheduler { + + private static final int THREAD_COUNT = 4; + + private final AtomicBoolean started = new AtomicBoolean(false); + + private final ConcurrentHashMap tasks = new ConcurrentHashMap<>(); + + private final Executor eventLoopExecutor = + Executors.newFixedThreadPool(THREAD_COUNT, DaemonThreadFactory.INSTANCE); + + private final EventLoop[] eventLoops = new EventLoop[THREAD_COUNT]; + + public boolean start() { + if (!started.compareAndSet(false, true)) { + return false; + } + for (int i = 0; i < THREAD_COUNT; i++) { + EventLoop eventLoop = new EventLoop(tasks); + eventLoops[i] = eventLoop; + eventLoopExecutor.execute(eventLoop); + } + return true; + } + + public void add(Simulatee simulatee) { + Objects.requireNonNull(simulatee); + + Task task = new Task(simulatee); + task.enqueue(LifecycleMessages.initialization()); + + tasks.put(task.id(), task); + + eventLoopFor(task).addToReadySet(task); + } + + public void enqueueTaskMessage(String simulateeId, Object message) { + Objects.requireNonNull(simulateeId); + Objects.requireNonNull(message); + + final @Nullable Task task = tasks.get(simulateeId); + if (task == null) { + log.info("An old message for a completed or a non existing entity: {}", message); + return; + } + task.enqueue(message); + eventLoopFor(task).addToReadySet(task); + } + + private EventLoop eventLoopFor(Task task) { + return eventLoops[partition(task)]; + } + + private static int partition(Task task) { + Objects.requireNonNull(task); + return Math.abs(task.id().hashCode()) % THREAD_COUNT; + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/engine/Simulatee.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/engine/Simulatee.java new file mode 100644 index 00000000..fe389d39 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/engine/Simulatee.java @@ -0,0 +1,28 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing.simulator.simulation.engine; + +public interface Simulatee { + + String id(); + + void apply(Object event); + + boolean isDone(); + + boolean needReschedule(); +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/engine/Task.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/engine/Task.java new file mode 100644 index 00000000..5e0e17aa --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/engine/Task.java @@ -0,0 +1,60 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing.simulator.simulation.engine; + +import java.util.Objects; +import java.util.concurrent.ConcurrentLinkedDeque; +import javax.annotation.Nullable; + +final class Task { + + private final ConcurrentLinkedDeque events = new ConcurrentLinkedDeque<>(); + + private final Simulatee simulatee; + + Task(Simulatee simulatee) { + this.simulatee = Objects.requireNonNull(simulatee); + } + + String id() { + return simulatee.id(); + } + + final void enqueue(Object event) { + Objects.requireNonNull(event); + events.add(event); + } + + final void processEnqueued() { + final Simulatee simulatee = this.simulatee; + while (!simulatee.isDone()) { + final @Nullable Object event = events.poll(); + if (event == null) { + return; + } + simulatee.apply(event); + } + } + + boolean isDone() { + return simulatee.isDone(); + } + + boolean needReschedule() { + return simulatee.needReschedule(); + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/messaging/Communication.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/messaging/Communication.java new file mode 100644 index 00000000..f30ee96f --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-example-simulator/src/main/java/com/ververica/statefun/examples/ridesharing/simulator/simulation/messaging/Communication.java @@ -0,0 +1,104 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.ridesharing.simulator.simulation.messaging; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage; +import com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage; +import com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage; +import com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage; +import com.ververica.statefun.examples.ridesharing.simulator.model.WebsocketDriverEvent; +import com.ververica.statefun.examples.ridesharing.simulator.model.WebsocketPassengerEvent; +import com.ververica.statefun.examples.ridesharing.simulator.services.KafkaDriverPublisher; +import com.ververica.statefun.examples.ridesharing.simulator.services.KafkaPassengerPublisher; +import com.ververica.statefun.examples.ridesharing.simulator.simulation.DriverMessaging; +import com.ververica.statefun.examples.ridesharing.simulator.simulation.PassengerMessaging; +import com.ververica.statefun.examples.ridesharing.simulator.simulation.engine.Scheduler; +import java.util.Objects; +import lombok.extern.slf4j.Slf4j; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.beans.factory.annotation.Value; +import org.springframework.messaging.simp.SimpMessagingTemplate; +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class Communication implements PassengerMessaging, DriverMessaging { + private final KafkaPassengerPublisher passengerPublisher; + private final KafkaDriverPublisher driverPublisher; + private final SimpMessagingTemplate simpSender; + private final String passengerWebSocketTopic; + private final String driverWebSocketTopic; + private final ObjectMapper objectMapper = new ObjectMapper(); + + private final Scheduler scheduler; + + @Autowired + public Communication( + KafkaPassengerPublisher passengerPublisher, + KafkaDriverPublisher driverPublisher, + SimpMessagingTemplate simpSender, + @Value("${web-socket.topic.passenger}") String passengerWebSocketTopic, + @Value("${web-socket.topic.driver}") String driverWebSocketTopic, + Scheduler scheduler) { + this.passengerPublisher = Objects.requireNonNull(passengerPublisher); + this.driverPublisher = Objects.requireNonNull(driverPublisher); + this.simpSender = Objects.requireNonNull(simpSender); + this.passengerWebSocketTopic = Objects.requireNonNull(passengerWebSocketTopic); + this.driverWebSocketTopic = Objects.requireNonNull(driverWebSocketTopic); + this.scheduler = Objects.requireNonNull(scheduler); + } + + public void incomingPassengerEvent(OutboundPassengerMessage message) { + scheduler.enqueueTaskMessage(message.getPassengerId(), message); + } + + public void incomingDriverEvent(OutboundDriverMessage message) { + scheduler.enqueueTaskMessage(message.getDriverId(), message); + } + + public void outgoingPassengerEvent(InboundPassengerMessage message) { + passengerPublisher.accept(message); + } + + public void outgoingDriverEvent(InboundDriverMessage message) { + driverPublisher.accept(message); + } + + public void broadcastPassengerSimulationEvent(WebsocketPassengerEvent passengerEvent) { + String json = toJsonString(passengerEvent); + simpSender.convertAndSend(passengerWebSocketTopic, json); + } + + public void broadcastDriverSimulationEvent(WebsocketDriverEvent driverEvent) { + if (driverEvent.getDriverStatus() == WebsocketDriverEvent.DriverStatus.IDLE) { + // don't broadcast idle drivers, this is slightly too much. + return; + } + String json = toJsonString(driverEvent); + simpSender.convertAndSend(driverWebSocketTopic, json); + } + + private String toJsonString(Object what) { + try { + return objectMapper.writeValueAsString(what); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/pom.xml b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/pom.xml new file mode 100644 index 00000000..8ec5d15d --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/pom.xml @@ -0,0 +1,37 @@ + + + + + stateful-functions-ridesharing-example + com.ververica + 1.0-SNAPSHOT + + 4.0.0 + + stateful-functions-ridesharing-protocol + + + + com.google.protobuf + protobuf-java + 3.8.0 + + + + \ No newline at end of file diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/DriverInCell.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/DriverInCell.java new file mode 100644 index 00000000..01c84c6f --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/DriverInCell.java @@ -0,0 +1,538 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +/** Protobuf type {@code com.ververica.statefun.examples.ridesharing.DriverInCell} */ +public final class DriverInCell extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.ridesharing.DriverInCell) + DriverInCellOrBuilder { + private static final long serialVersionUID = 0L; + // Use DriverInCell.newBuilder() to construct. + private DriverInCell(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private DriverInCell() { + driverId_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private DriverInCell( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + java.lang.String s = input.readStringRequireUtf8(); + + driverId_ = s; + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_DriverInCell_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_DriverInCell_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.DriverInCell.class, + com.ververica.statefun.examples.ridesharing.generated.DriverInCell.Builder.class); + } + + public static final int DRIVER_ID_FIELD_NUMBER = 1; + private volatile java.lang.Object driverId_; + /** string driver_id = 1; */ + public java.lang.String getDriverId() { + java.lang.Object ref = driverId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + driverId_ = s; + return s; + } + } + /** string driver_id = 1; */ + public com.google.protobuf.ByteString getDriverIdBytes() { + java.lang.Object ref = driverId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + driverId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!getDriverIdBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, driverId_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!getDriverIdBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, driverId_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.ververica.statefun.examples.ridesharing.generated.DriverInCell)) { + return super.equals(obj); + } + com.ververica.statefun.examples.ridesharing.generated.DriverInCell other = + (com.ververica.statefun.examples.ridesharing.generated.DriverInCell) obj; + + if (!getDriverId().equals(other.getDriverId())) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + DRIVER_ID_FIELD_NUMBER; + hash = (53 * hash) + getDriverId().hashCode(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverInCell parseFrom( + java.nio.ByteBuffer data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverInCell parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverInCell parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverInCell parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverInCell parseFrom( + byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverInCell parseFrom( + byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverInCell parseFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverInCell parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverInCell + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverInCell + parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverInCell parseFrom( + com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverInCell parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.ridesharing.generated.DriverInCell prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code com.ververica.statefun.examples.ridesharing.DriverInCell} */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.ridesharing.DriverInCell) + com.ververica.statefun.examples.ridesharing.generated.DriverInCellOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_DriverInCell_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_DriverInCell_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.DriverInCell.class, + com.ververica.statefun.examples.ridesharing.generated.DriverInCell.Builder.class); + } + + // Construct using + // com.ververica.statefun.examples.ridesharing.generated.DriverInCell.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + driverId_ = ""; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_DriverInCell_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.DriverInCell + getDefaultInstanceForType() { + return com.ververica.statefun.examples.ridesharing.generated.DriverInCell + .getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.DriverInCell build() { + com.ververica.statefun.examples.ridesharing.generated.DriverInCell result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.DriverInCell buildPartial() { + com.ververica.statefun.examples.ridesharing.generated.DriverInCell result = + new com.ververica.statefun.examples.ridesharing.generated.DriverInCell(this); + result.driverId_ = driverId_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.ververica.statefun.examples.ridesharing.generated.DriverInCell) { + return mergeFrom( + (com.ververica.statefun.examples.ridesharing.generated.DriverInCell) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.ridesharing.generated.DriverInCell other) { + if (other + == com.ververica.statefun.examples.ridesharing.generated.DriverInCell + .getDefaultInstance()) return this; + if (!other.getDriverId().isEmpty()) { + driverId_ = other.driverId_; + onChanged(); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.ridesharing.generated.DriverInCell parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.ridesharing.generated.DriverInCell) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private java.lang.Object driverId_ = ""; + /** string driver_id = 1; */ + public java.lang.String getDriverId() { + java.lang.Object ref = driverId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + driverId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string driver_id = 1; */ + public com.google.protobuf.ByteString getDriverIdBytes() { + java.lang.Object ref = driverId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + driverId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string driver_id = 1; */ + public Builder setDriverId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + driverId_ = value; + onChanged(); + return this; + } + /** string driver_id = 1; */ + public Builder clearDriverId() { + + driverId_ = getDefaultInstance().getDriverId(); + onChanged(); + return this; + } + /** string driver_id = 1; */ + public Builder setDriverIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + driverId_ = value; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.ridesharing.DriverInCell) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.ridesharing.DriverInCell) + private static final com.ververica.statefun.examples.ridesharing.generated.DriverInCell + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = new com.ververica.statefun.examples.ridesharing.generated.DriverInCell(); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverInCell + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public DriverInCell parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new DriverInCell(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.DriverInCell + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/DriverInCellOrBuilder.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/DriverInCellOrBuilder.java new file mode 100644 index 00000000..b7272ab7 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/DriverInCellOrBuilder.java @@ -0,0 +1,15 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +public interface DriverInCellOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.ridesharing.DriverInCell) + com.google.protobuf.MessageOrBuilder { + + /** string driver_id = 1; */ + java.lang.String getDriverId(); + /** string driver_id = 1; */ + com.google.protobuf.ByteString getDriverIdBytes(); +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/DriverJoinsRide.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/DriverJoinsRide.java new file mode 100644 index 00000000..90ad8d6e --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/DriverJoinsRide.java @@ -0,0 +1,690 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +/** Protobuf type {@code com.ververica.statefun.examples.ridesharing.DriverJoinsRide} */ +public final class DriverJoinsRide extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.ridesharing.DriverJoinsRide) + DriverJoinsRideOrBuilder { + private static final long serialVersionUID = 0L; + // Use DriverJoinsRide.newBuilder() to construct. + private DriverJoinsRide(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private DriverJoinsRide() { + driverId_ = ""; + passengerId_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private DriverJoinsRide( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + java.lang.String s = input.readStringRequireUtf8(); + + driverId_ = s; + break; + } + case 18: + { + java.lang.String s = input.readStringRequireUtf8(); + + passengerId_ = s; + break; + } + case 24: + { + driverLocation_ = input.readInt32(); + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_DriverJoinsRide_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_DriverJoinsRide_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide.class, + com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide.Builder.class); + } + + public static final int DRIVER_ID_FIELD_NUMBER = 1; + private volatile java.lang.Object driverId_; + /** string driver_id = 1; */ + public java.lang.String getDriverId() { + java.lang.Object ref = driverId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + driverId_ = s; + return s; + } + } + /** string driver_id = 1; */ + public com.google.protobuf.ByteString getDriverIdBytes() { + java.lang.Object ref = driverId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + driverId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int PASSENGER_ID_FIELD_NUMBER = 2; + private volatile java.lang.Object passengerId_; + /** string passenger_id = 2; */ + public java.lang.String getPassengerId() { + java.lang.Object ref = passengerId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + passengerId_ = s; + return s; + } + } + /** string passenger_id = 2; */ + public com.google.protobuf.ByteString getPassengerIdBytes() { + java.lang.Object ref = passengerId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + passengerId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int DRIVER_LOCATION_FIELD_NUMBER = 3; + private int driverLocation_; + /** int32 driver_location = 3; */ + public int getDriverLocation() { + return driverLocation_; + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!getDriverIdBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, driverId_); + } + if (!getPassengerIdBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 2, passengerId_); + } + if (driverLocation_ != 0) { + output.writeInt32(3, driverLocation_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!getDriverIdBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, driverId_); + } + if (!getPassengerIdBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, passengerId_); + } + if (driverLocation_ != 0) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(3, driverLocation_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide)) { + return super.equals(obj); + } + com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide other = + (com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide) obj; + + if (!getDriverId().equals(other.getDriverId())) return false; + if (!getPassengerId().equals(other.getPassengerId())) return false; + if (getDriverLocation() != other.getDriverLocation()) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + DRIVER_ID_FIELD_NUMBER; + hash = (53 * hash) + getDriverId().hashCode(); + hash = (37 * hash) + PASSENGER_ID_FIELD_NUMBER; + hash = (53 * hash) + getPassengerId().hashCode(); + hash = (37 * hash) + DRIVER_LOCATION_FIELD_NUMBER; + hash = (53 * hash) + getDriverLocation(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide parseFrom( + java.nio.ByteBuffer data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide parseFrom( + byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide parseFrom( + byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide parseFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide + parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide parseFrom( + com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code com.ververica.statefun.examples.ridesharing.DriverJoinsRide} */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.ridesharing.DriverJoinsRide) + com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRideOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_DriverJoinsRide_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_DriverJoinsRide_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide.class, + com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide.Builder.class); + } + + // Construct using + // com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + driverId_ = ""; + + passengerId_ = ""; + + driverLocation_ = 0; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_DriverJoinsRide_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide + getDefaultInstanceForType() { + return com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide + .getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide build() { + com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide buildPartial() { + com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide result = + new com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide(this); + result.driverId_ = driverId_; + result.passengerId_ = passengerId_; + result.driverLocation_ = driverLocation_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide) { + return mergeFrom( + (com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide other) { + if (other + == com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide + .getDefaultInstance()) return this; + if (!other.getDriverId().isEmpty()) { + driverId_ = other.driverId_; + onChanged(); + } + if (!other.getPassengerId().isEmpty()) { + passengerId_ = other.passengerId_; + onChanged(); + } + if (other.getDriverLocation() != 0) { + setDriverLocation(other.getDriverLocation()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private java.lang.Object driverId_ = ""; + /** string driver_id = 1; */ + public java.lang.String getDriverId() { + java.lang.Object ref = driverId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + driverId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string driver_id = 1; */ + public com.google.protobuf.ByteString getDriverIdBytes() { + java.lang.Object ref = driverId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + driverId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string driver_id = 1; */ + public Builder setDriverId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + driverId_ = value; + onChanged(); + return this; + } + /** string driver_id = 1; */ + public Builder clearDriverId() { + + driverId_ = getDefaultInstance().getDriverId(); + onChanged(); + return this; + } + /** string driver_id = 1; */ + public Builder setDriverIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + driverId_ = value; + onChanged(); + return this; + } + + private java.lang.Object passengerId_ = ""; + /** string passenger_id = 2; */ + public java.lang.String getPassengerId() { + java.lang.Object ref = passengerId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + passengerId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string passenger_id = 2; */ + public com.google.protobuf.ByteString getPassengerIdBytes() { + java.lang.Object ref = passengerId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + passengerId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string passenger_id = 2; */ + public Builder setPassengerId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + passengerId_ = value; + onChanged(); + return this; + } + /** string passenger_id = 2; */ + public Builder clearPassengerId() { + + passengerId_ = getDefaultInstance().getPassengerId(); + onChanged(); + return this; + } + /** string passenger_id = 2; */ + public Builder setPassengerIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + passengerId_ = value; + onChanged(); + return this; + } + + private int driverLocation_; + /** int32 driver_location = 3; */ + public int getDriverLocation() { + return driverLocation_; + } + /** int32 driver_location = 3; */ + public Builder setDriverLocation(int value) { + + driverLocation_ = value; + onChanged(); + return this; + } + /** int32 driver_location = 3; */ + public Builder clearDriverLocation() { + + driverLocation_ = 0; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.ridesharing.DriverJoinsRide) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.ridesharing.DriverJoinsRide) + private static final com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = new com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide(); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public DriverJoinsRide parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new DriverJoinsRide(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.DriverJoinsRide + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/DriverJoinsRideOrBuilder.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/DriverJoinsRideOrBuilder.java new file mode 100644 index 00000000..512183df --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/DriverJoinsRideOrBuilder.java @@ -0,0 +1,23 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +public interface DriverJoinsRideOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.ridesharing.DriverJoinsRide) + com.google.protobuf.MessageOrBuilder { + + /** string driver_id = 1; */ + java.lang.String getDriverId(); + /** string driver_id = 1; */ + com.google.protobuf.ByteString getDriverIdBytes(); + + /** string passenger_id = 2; */ + java.lang.String getPassengerId(); + /** string passenger_id = 2; */ + com.google.protobuf.ByteString getPassengerIdBytes(); + + /** int32 driver_location = 3; */ + int getDriverLocation(); +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/DriverRejectsPickup.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/DriverRejectsPickup.java new file mode 100644 index 00000000..03035473 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/DriverRejectsPickup.java @@ -0,0 +1,651 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +/** Protobuf type {@code com.ververica.statefun.examples.ridesharing.DriverRejectsPickup} */ +public final class DriverRejectsPickup extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.ridesharing.DriverRejectsPickup) + DriverRejectsPickupOrBuilder { + private static final long serialVersionUID = 0L; + // Use DriverRejectsPickup.newBuilder() to construct. + private DriverRejectsPickup(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private DriverRejectsPickup() { + driverId_ = ""; + rideId_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private DriverRejectsPickup( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + java.lang.String s = input.readStringRequireUtf8(); + + driverId_ = s; + break; + } + case 18: + { + java.lang.String s = input.readStringRequireUtf8(); + + rideId_ = s; + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_DriverRejectsPickup_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_DriverRejectsPickup_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup.class, + com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup.Builder + .class); + } + + public static final int DRIVER_ID_FIELD_NUMBER = 1; + private volatile java.lang.Object driverId_; + /** string driver_id = 1; */ + public java.lang.String getDriverId() { + java.lang.Object ref = driverId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + driverId_ = s; + return s; + } + } + /** string driver_id = 1; */ + public com.google.protobuf.ByteString getDriverIdBytes() { + java.lang.Object ref = driverId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + driverId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int RIDE_ID_FIELD_NUMBER = 2; + private volatile java.lang.Object rideId_; + /** string ride_id = 2; */ + public java.lang.String getRideId() { + java.lang.Object ref = rideId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + rideId_ = s; + return s; + } + } + /** string ride_id = 2; */ + public com.google.protobuf.ByteString getRideIdBytes() { + java.lang.Object ref = rideId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + rideId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!getDriverIdBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, driverId_); + } + if (!getRideIdBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 2, rideId_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!getDriverIdBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, driverId_); + } + if (!getRideIdBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, rideId_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup)) { + return super.equals(obj); + } + com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup other = + (com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup) obj; + + if (!getDriverId().equals(other.getDriverId())) return false; + if (!getRideId().equals(other.getRideId())) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + DRIVER_ID_FIELD_NUMBER; + hash = (53 * hash) + getDriverId().hashCode(); + hash = (37 * hash) + RIDE_ID_FIELD_NUMBER; + hash = (53 * hash) + getRideId().hashCode(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup parseFrom( + java.nio.ByteBuffer data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup parseFrom( + byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup parseFrom( + byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup parseFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup + parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup parseFrom( + com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code com.ververica.statefun.examples.ridesharing.DriverRejectsPickup} */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.ridesharing.DriverRejectsPickup) + com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickupOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_DriverRejectsPickup_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_DriverRejectsPickup_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup.class, + com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup.Builder + .class); + } + + // Construct using + // com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + driverId_ = ""; + + rideId_ = ""; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_DriverRejectsPickup_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup + getDefaultInstanceForType() { + return com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup + .getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup build() { + com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup result = + buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup + buildPartial() { + com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup result = + new com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup(this); + result.driverId_ = driverId_; + result.rideId_ = rideId_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other + instanceof com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup) { + return mergeFrom( + (com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup other) { + if (other + == com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup + .getDefaultInstance()) return this; + if (!other.getDriverId().isEmpty()) { + driverId_ = other.driverId_; + onChanged(); + } + if (!other.getRideId().isEmpty()) { + rideId_ = other.rideId_; + onChanged(); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup parsedMessage = + null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private java.lang.Object driverId_ = ""; + /** string driver_id = 1; */ + public java.lang.String getDriverId() { + java.lang.Object ref = driverId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + driverId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string driver_id = 1; */ + public com.google.protobuf.ByteString getDriverIdBytes() { + java.lang.Object ref = driverId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + driverId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string driver_id = 1; */ + public Builder setDriverId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + driverId_ = value; + onChanged(); + return this; + } + /** string driver_id = 1; */ + public Builder clearDriverId() { + + driverId_ = getDefaultInstance().getDriverId(); + onChanged(); + return this; + } + /** string driver_id = 1; */ + public Builder setDriverIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + driverId_ = value; + onChanged(); + return this; + } + + private java.lang.Object rideId_ = ""; + /** string ride_id = 2; */ + public java.lang.String getRideId() { + java.lang.Object ref = rideId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + rideId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string ride_id = 2; */ + public com.google.protobuf.ByteString getRideIdBytes() { + java.lang.Object ref = rideId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + rideId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string ride_id = 2; */ + public Builder setRideId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + rideId_ = value; + onChanged(); + return this; + } + /** string ride_id = 2; */ + public Builder clearRideId() { + + rideId_ = getDefaultInstance().getRideId(); + onChanged(); + return this; + } + /** string ride_id = 2; */ + public Builder setRideIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + rideId_ = value; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.ridesharing.DriverRejectsPickup) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.ridesharing.DriverRejectsPickup) + private static final com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup(); + } + + public static com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public DriverRejectsPickup parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new DriverRejectsPickup(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.DriverRejectsPickup + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/DriverRejectsPickupOrBuilder.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/DriverRejectsPickupOrBuilder.java new file mode 100644 index 00000000..157f526f --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/DriverRejectsPickupOrBuilder.java @@ -0,0 +1,20 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +public interface DriverRejectsPickupOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.ridesharing.DriverRejectsPickup) + com.google.protobuf.MessageOrBuilder { + + /** string driver_id = 1; */ + java.lang.String getDriverId(); + /** string driver_id = 1; */ + com.google.protobuf.ByteString getDriverIdBytes(); + + /** string ride_id = 2; */ + java.lang.String getRideId(); + /** string ride_id = 2; */ + com.google.protobuf.ByteString getRideIdBytes(); +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/GeoCellState.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/GeoCellState.java new file mode 100644 index 00000000..a6009a0b --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/GeoCellState.java @@ -0,0 +1,573 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +/** Protobuf type {@code com.ververica.statefun.examples.ridesharing.GeoCellState} */ +public final class GeoCellState extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.ridesharing.GeoCellState) + GeoCellStateOrBuilder { + private static final long serialVersionUID = 0L; + // Use GeoCellState.newBuilder() to construct. + private GeoCellState(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private GeoCellState() { + driverId_ = com.google.protobuf.LazyStringArrayList.EMPTY; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private GeoCellState( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + java.lang.String s = input.readStringRequireUtf8(); + if (!((mutable_bitField0_ & 0x00000001) != 0)) { + driverId_ = new com.google.protobuf.LazyStringArrayList(); + mutable_bitField0_ |= 0x00000001; + } + driverId_.add(s); + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + if (((mutable_bitField0_ & 0x00000001) != 0)) { + driverId_ = driverId_.getUnmodifiableView(); + } + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_GeoCellState_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_GeoCellState_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.GeoCellState.class, + com.ververica.statefun.examples.ridesharing.generated.GeoCellState.Builder.class); + } + + public static final int DRIVER_ID_FIELD_NUMBER = 1; + private com.google.protobuf.LazyStringList driverId_; + /** repeated string driver_id = 1; */ + public com.google.protobuf.ProtocolStringList getDriverIdList() { + return driverId_; + } + /** repeated string driver_id = 1; */ + public int getDriverIdCount() { + return driverId_.size(); + } + /** repeated string driver_id = 1; */ + public java.lang.String getDriverId(int index) { + return driverId_.get(index); + } + /** repeated string driver_id = 1; */ + public com.google.protobuf.ByteString getDriverIdBytes(int index) { + return driverId_.getByteString(index); + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + for (int i = 0; i < driverId_.size(); i++) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, driverId_.getRaw(i)); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + { + int dataSize = 0; + for (int i = 0; i < driverId_.size(); i++) { + dataSize += computeStringSizeNoTag(driverId_.getRaw(i)); + } + size += dataSize; + size += 1 * getDriverIdList().size(); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.ververica.statefun.examples.ridesharing.generated.GeoCellState)) { + return super.equals(obj); + } + com.ververica.statefun.examples.ridesharing.generated.GeoCellState other = + (com.ververica.statefun.examples.ridesharing.generated.GeoCellState) obj; + + if (!getDriverIdList().equals(other.getDriverIdList())) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (getDriverIdCount() > 0) { + hash = (37 * hash) + DRIVER_ID_FIELD_NUMBER; + hash = (53 * hash) + getDriverIdList().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.ridesharing.generated.GeoCellState parseFrom( + java.nio.ByteBuffer data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.GeoCellState parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.GeoCellState parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.GeoCellState parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.GeoCellState parseFrom( + byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.GeoCellState parseFrom( + byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.GeoCellState parseFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.GeoCellState parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.GeoCellState + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.GeoCellState + parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.GeoCellState parseFrom( + com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.GeoCellState parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.ridesharing.generated.GeoCellState prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code com.ververica.statefun.examples.ridesharing.GeoCellState} */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.ridesharing.GeoCellState) + com.ververica.statefun.examples.ridesharing.generated.GeoCellStateOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_GeoCellState_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_GeoCellState_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.GeoCellState.class, + com.ververica.statefun.examples.ridesharing.generated.GeoCellState.Builder.class); + } + + // Construct using + // com.ververica.statefun.examples.ridesharing.generated.GeoCellState.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + driverId_ = com.google.protobuf.LazyStringArrayList.EMPTY; + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_GeoCellState_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.GeoCellState + getDefaultInstanceForType() { + return com.ververica.statefun.examples.ridesharing.generated.GeoCellState + .getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.GeoCellState build() { + com.ververica.statefun.examples.ridesharing.generated.GeoCellState result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.GeoCellState buildPartial() { + com.ververica.statefun.examples.ridesharing.generated.GeoCellState result = + new com.ververica.statefun.examples.ridesharing.generated.GeoCellState(this); + int from_bitField0_ = bitField0_; + if (((bitField0_ & 0x00000001) != 0)) { + driverId_ = driverId_.getUnmodifiableView(); + bitField0_ = (bitField0_ & ~0x00000001); + } + result.driverId_ = driverId_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.ververica.statefun.examples.ridesharing.generated.GeoCellState) { + return mergeFrom( + (com.ververica.statefun.examples.ridesharing.generated.GeoCellState) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.ridesharing.generated.GeoCellState other) { + if (other + == com.ververica.statefun.examples.ridesharing.generated.GeoCellState + .getDefaultInstance()) return this; + if (!other.driverId_.isEmpty()) { + if (driverId_.isEmpty()) { + driverId_ = other.driverId_; + bitField0_ = (bitField0_ & ~0x00000001); + } else { + ensureDriverIdIsMutable(); + driverId_.addAll(other.driverId_); + } + onChanged(); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.ridesharing.generated.GeoCellState parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.ridesharing.generated.GeoCellState) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int bitField0_; + + private com.google.protobuf.LazyStringList driverId_ = + com.google.protobuf.LazyStringArrayList.EMPTY; + + private void ensureDriverIdIsMutable() { + if (!((bitField0_ & 0x00000001) != 0)) { + driverId_ = new com.google.protobuf.LazyStringArrayList(driverId_); + bitField0_ |= 0x00000001; + } + } + /** repeated string driver_id = 1; */ + public com.google.protobuf.ProtocolStringList getDriverIdList() { + return driverId_.getUnmodifiableView(); + } + /** repeated string driver_id = 1; */ + public int getDriverIdCount() { + return driverId_.size(); + } + /** repeated string driver_id = 1; */ + public java.lang.String getDriverId(int index) { + return driverId_.get(index); + } + /** repeated string driver_id = 1; */ + public com.google.protobuf.ByteString getDriverIdBytes(int index) { + return driverId_.getByteString(index); + } + /** repeated string driver_id = 1; */ + public Builder setDriverId(int index, java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + ensureDriverIdIsMutable(); + driverId_.set(index, value); + onChanged(); + return this; + } + /** repeated string driver_id = 1; */ + public Builder addDriverId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + ensureDriverIdIsMutable(); + driverId_.add(value); + onChanged(); + return this; + } + /** repeated string driver_id = 1; */ + public Builder addAllDriverId(java.lang.Iterable values) { + ensureDriverIdIsMutable(); + com.google.protobuf.AbstractMessageLite.Builder.addAll(values, driverId_); + onChanged(); + return this; + } + /** repeated string driver_id = 1; */ + public Builder clearDriverId() { + driverId_ = com.google.protobuf.LazyStringArrayList.EMPTY; + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + return this; + } + /** repeated string driver_id = 1; */ + public Builder addDriverIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + ensureDriverIdIsMutable(); + driverId_.add(value); + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.ridesharing.GeoCellState) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.ridesharing.GeoCellState) + private static final com.ververica.statefun.examples.ridesharing.generated.GeoCellState + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = new com.ververica.statefun.examples.ridesharing.generated.GeoCellState(); + } + + public static com.ververica.statefun.examples.ridesharing.generated.GeoCellState + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public GeoCellState parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new GeoCellState(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.GeoCellState + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/GeoCellStateOrBuilder.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/GeoCellStateOrBuilder.java new file mode 100644 index 00000000..6b3675c6 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/GeoCellStateOrBuilder.java @@ -0,0 +1,19 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +public interface GeoCellStateOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.ridesharing.GeoCellState) + com.google.protobuf.MessageOrBuilder { + + /** repeated string driver_id = 1; */ + java.util.List getDriverIdList(); + /** repeated string driver_id = 1; */ + int getDriverIdCount(); + /** repeated string driver_id = 1; */ + java.lang.String getDriverId(int index); + /** repeated string driver_id = 1; */ + com.google.protobuf.ByteString getDriverIdBytes(int index); +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/GetDriver.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/GetDriver.java new file mode 100644 index 00000000..9112a242 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/GetDriver.java @@ -0,0 +1,427 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +/** Protobuf type {@code com.ververica.statefun.examples.ridesharing.GetDriver} */ +public final class GetDriver extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.ridesharing.GetDriver) + GetDriverOrBuilder { + private static final long serialVersionUID = 0L; + // Use GetDriver.newBuilder() to construct. + private GetDriver(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private GetDriver() {} + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private GetDriver( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_GetDriver_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_GetDriver_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.GetDriver.class, + com.ververica.statefun.examples.ridesharing.generated.GetDriver.Builder.class); + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.ververica.statefun.examples.ridesharing.generated.GetDriver)) { + return super.equals(obj); + } + com.ververica.statefun.examples.ridesharing.generated.GetDriver other = + (com.ververica.statefun.examples.ridesharing.generated.GetDriver) obj; + + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.ridesharing.generated.GetDriver parseFrom( + java.nio.ByteBuffer data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.GetDriver parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.GetDriver parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.GetDriver parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.GetDriver parseFrom( + byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.GetDriver parseFrom( + byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.GetDriver parseFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.GetDriver parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.GetDriver parseDelimitedFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.GetDriver parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.GetDriver parseFrom( + com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.GetDriver parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.ridesharing.generated.GetDriver prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code com.ververica.statefun.examples.ridesharing.GetDriver} */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.ridesharing.GetDriver) + com.ververica.statefun.examples.ridesharing.generated.GetDriverOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_GetDriver_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_GetDriver_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.GetDriver.class, + com.ververica.statefun.examples.ridesharing.generated.GetDriver.Builder.class); + } + + // Construct using com.ververica.statefun.examples.ridesharing.generated.GetDriver.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_GetDriver_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.GetDriver + getDefaultInstanceForType() { + return com.ververica.statefun.examples.ridesharing.generated.GetDriver.getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.GetDriver build() { + com.ververica.statefun.examples.ridesharing.generated.GetDriver result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.GetDriver buildPartial() { + com.ververica.statefun.examples.ridesharing.generated.GetDriver result = + new com.ververica.statefun.examples.ridesharing.generated.GetDriver(this); + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.ververica.statefun.examples.ridesharing.generated.GetDriver) { + return mergeFrom((com.ververica.statefun.examples.ridesharing.generated.GetDriver) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.ridesharing.generated.GetDriver other) { + if (other + == com.ververica.statefun.examples.ridesharing.generated.GetDriver.getDefaultInstance()) + return this; + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.ridesharing.generated.GetDriver parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.ridesharing.generated.GetDriver) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.ridesharing.GetDriver) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.ridesharing.GetDriver) + private static final com.ververica.statefun.examples.ridesharing.generated.GetDriver + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = new com.ververica.statefun.examples.ridesharing.generated.GetDriver(); + } + + public static com.ververica.statefun.examples.ridesharing.generated.GetDriver + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public GetDriver parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new GetDriver(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.GetDriver + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/GetDriverOrBuilder.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/GetDriverOrBuilder.java new file mode 100644 index 00000000..09f03f4c --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/GetDriverOrBuilder.java @@ -0,0 +1,9 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +public interface GetDriverOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.ridesharing.GetDriver) + com.google.protobuf.MessageOrBuilder {} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/InboundDriverMessage.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/InboundDriverMessage.java new file mode 100644 index 00000000..11f07c1d --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/InboundDriverMessage.java @@ -0,0 +1,3203 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +/** + * + * + *
+ * the messages that corresponds to the FROM_DRIVER_INGRESS
+ * 
+ * + * Protobuf type {@code com.ververica.statefun.examples.ridesharing.InboundDriverMessage} + */ +public final class InboundDriverMessage extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.ridesharing.InboundDriverMessage) + InboundDriverMessageOrBuilder { + private static final long serialVersionUID = 0L; + // Use InboundDriverMessage.newBuilder() to construct. + private InboundDriverMessage(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private InboundDriverMessage() { + driverId_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private InboundDriverMessage( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + java.lang.String s = input.readStringRequireUtf8(); + + driverId_ = s; + break; + } + case 18: + { + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted + .Builder + subBuilder = null; + if (messageCase_ == 2) { + subBuilder = + ((com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted) + message_) + .toBuilder(); + } + message_ = + input.readMessage( + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted.parser(), + extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom( + (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted) + message_); + message_ = subBuilder.buildPartial(); + } + messageCase_ = 2; + break; + } + case 26: + { + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + .Builder + subBuilder = null; + if (messageCase_ == 3) { + subBuilder = + ((com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEnded) + message_) + .toBuilder(); + } + message_ = + input.readMessage( + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEnded.parser(), + extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom( + (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEnded) + message_); + message_ = subBuilder.buildPartial(); + } + messageCase_ = 3; + break; + } + case 34: + { + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate.Builder + subBuilder = null; + if (messageCase_ == 4) { + subBuilder = + ((com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate) + message_) + .toBuilder(); + } + message_ = + input.readMessage( + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate.parser(), + extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom( + (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate) + message_); + message_ = subBuilder.buildPartial(); + } + messageCase_ = 4; + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.class, + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.Builder + .class); + } + + public interface RideStartedOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideStarted) + com.google.protobuf.MessageOrBuilder {} + /** + * + * + *
+   * A message the driver sends once they pickup the passenger
+   * 
+ * + * Protobuf type {@code + * com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideStarted} + */ + public static final class RideStarted extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideStarted) + RideStartedOrBuilder { + private static final long serialVersionUID = 0L; + // Use RideStarted.newBuilder() to construct. + private RideStarted(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private RideStarted() {} + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private RideStarted( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_RideStarted_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_RideStarted_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted + .class, + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted + .Builder.class); + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted)) { + return super.equals(obj); + } + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted other = + (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted) + obj; + + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted + parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted + parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted + parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted + parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted + parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted + parseFrom(byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted + parseFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted + parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted + parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted + parseFrom(com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted + parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted + prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * + * + *
+     * A message the driver sends once they pickup the passenger
+     * 
+ * + * Protobuf type {@code + * com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideStarted} + */ + public static final class Builder + extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideStarted) + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStartedOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_RideStarted_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_RideStarted_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted.class, + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted.Builder.class); + } + + // Construct using + // com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_RideStarted_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted + getDefaultInstanceForType() { + return com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted.getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted + build() { + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted + result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted + buildPartial() { + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted + result = + new com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted(this); + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other + instanceof + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted) { + return mergeFrom( + (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted) + other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted + other) { + if (other + == com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted.getDefaultInstance()) return this; + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted + parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideStarted) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideStarted) + private static final com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted(); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public RideStarted parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new RideStarted(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + public interface RideEndedOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideEnded) + com.google.protobuf.MessageOrBuilder { + + /** string ride_id = 2; */ + java.lang.String getRideId(); + /** string ride_id = 2; */ + com.google.protobuf.ByteString getRideIdBytes(); + } + /** + * + * + *
+   * A message the driver sends when they drop off the passenger.
+   * 
+ * + * Protobuf type {@code + * com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideEnded} + */ + public static final class RideEnded extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideEnded) + RideEndedOrBuilder { + private static final long serialVersionUID = 0L; + // Use RideEnded.newBuilder() to construct. + private RideEnded(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private RideEnded() { + rideId_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private RideEnded( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 18: + { + java.lang.String s = input.readStringRequireUtf8(); + + rideId_ = s; + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_RideEnded_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_RideEnded_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + .class, + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + .Builder.class); + } + + public static final int RIDE_ID_FIELD_NUMBER = 2; + private volatile java.lang.Object rideId_; + /** string ride_id = 2; */ + public java.lang.String getRideId() { + java.lang.Object ref = rideId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + rideId_ = s; + return s; + } + } + /** string ride_id = 2; */ + public com.google.protobuf.ByteString getRideIdBytes() { + java.lang.Object ref = rideId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + rideId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!getRideIdBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 2, rideId_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!getRideIdBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, rideId_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded)) { + return super.equals(obj); + } + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded other = + (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded) + obj; + + if (!getRideId().equals(other.getRideId())) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + RIDE_ID_FIELD_NUMBER; + hash = (53 * hash) + getRideId().hashCode(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEnded + parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEnded + parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEnded + parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEnded + parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEnded + parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEnded + parseFrom(byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEnded + parseFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEnded + parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEnded + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEnded + parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEnded + parseFrom(com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEnded + parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * + * + *
+     * A message the driver sends when they drop off the passenger.
+     * 
+ * + * Protobuf type {@code + * com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideEnded} + */ + public static final class Builder + extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideEnded) + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEndedOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_RideEnded_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_RideEnded_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + .class, + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + .Builder.class); + } + + // Construct using + // com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + rideId_ = ""; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_RideEnded_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + getDefaultInstanceForType() { + return com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + .getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + build() { + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + buildPartial() { + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + result = + new com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEnded(this); + result.rideId_ = rideId_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other + instanceof + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded) { + return mergeFrom( + (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded) + other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + other) { + if (other + == com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + .getDefaultInstance()) return this; + if (!other.getRideId().isEmpty()) { + rideId_ = other.rideId_; + onChanged(); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private java.lang.Object rideId_ = ""; + /** string ride_id = 2; */ + public java.lang.String getRideId() { + java.lang.Object ref = rideId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + rideId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string ride_id = 2; */ + public com.google.protobuf.ByteString getRideIdBytes() { + java.lang.Object ref = rideId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + rideId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string ride_id = 2; */ + public Builder setRideId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + rideId_ = value; + onChanged(); + return this; + } + /** string ride_id = 2; */ + public Builder clearRideId() { + + rideId_ = getDefaultInstance().getRideId(); + onChanged(); + return this; + } + /** string ride_id = 2; */ + public Builder setRideIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + rideId_ = value; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideEnded) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideEnded) + private static final com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEnded + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEnded(); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEnded + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public RideEnded parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new RideEnded(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + public interface LocationUpdateOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.ridesharing.InboundDriverMessage.LocationUpdate) + com.google.protobuf.MessageOrBuilder { + + /** int32 current_geo_cell = 2; */ + int getCurrentGeoCell(); + } + /** + * + * + *
+   * A periodic location update message
+   * 
+ * + * Protobuf type {@code + * com.ververica.statefun.examples.ridesharing.InboundDriverMessage.LocationUpdate} + */ + public static final class LocationUpdate extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.ridesharing.InboundDriverMessage.LocationUpdate) + LocationUpdateOrBuilder { + private static final long serialVersionUID = 0L; + // Use LocationUpdate.newBuilder() to construct. + private LocationUpdate(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private LocationUpdate() {} + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private LocationUpdate( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 16: + { + currentGeoCell_ = input.readInt32(); + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_LocationUpdate_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_LocationUpdate_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate.class, + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate.Builder.class); + } + + public static final int CURRENT_GEO_CELL_FIELD_NUMBER = 2; + private int currentGeoCell_; + /** int32 current_geo_cell = 2; */ + public int getCurrentGeoCell() { + return currentGeoCell_; + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (currentGeoCell_ != 0) { + output.writeInt32(2, currentGeoCell_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (currentGeoCell_ != 0) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(2, currentGeoCell_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate)) { + return super.equals(obj); + } + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.LocationUpdate + other = + (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate) + obj; + + if (getCurrentGeoCell() != other.getCurrentGeoCell()) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + CURRENT_GEO_CELL_FIELD_NUMBER; + hash = (53 * hash) + getCurrentGeoCell(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate + parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate + parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate + parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate + parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate + parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate + parseFrom(byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate + parseFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate + parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate + parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate + parseFrom(com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate + parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.LocationUpdate + prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * + * + *
+     * A periodic location update message
+     * 
+ * + * Protobuf type {@code + * com.ververica.statefun.examples.ridesharing.InboundDriverMessage.LocationUpdate} + */ + public static final class Builder + extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.ridesharing.InboundDriverMessage.LocationUpdate) + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdateOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_LocationUpdate_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_LocationUpdate_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate.class, + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate.Builder.class); + } + + // Construct using + // com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.LocationUpdate.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + currentGeoCell_ = 0; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_LocationUpdate_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate + getDefaultInstanceForType() { + return com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate.getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate + build() { + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.LocationUpdate + result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate + buildPartial() { + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.LocationUpdate + result = + new com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate(this); + result.currentGeoCell_ = currentGeoCell_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other + instanceof + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate) { + return mergeFrom( + (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate) + other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.LocationUpdate + other) { + if (other + == com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate.getDefaultInstance()) return this; + if (other.getCurrentGeoCell() != 0) { + setCurrentGeoCell(other.getCurrentGeoCell()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.LocationUpdate + parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int currentGeoCell_; + /** int32 current_geo_cell = 2; */ + public int getCurrentGeoCell() { + return currentGeoCell_; + } + /** int32 current_geo_cell = 2; */ + public Builder setCurrentGeoCell(int value) { + + currentGeoCell_ = value; + onChanged(); + return this; + } + /** int32 current_geo_cell = 2; */ + public Builder clearCurrentGeoCell() { + + currentGeoCell_ = 0; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.ridesharing.InboundDriverMessage.LocationUpdate) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.ridesharing.InboundDriverMessage.LocationUpdate) + private static final com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate(); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public LocationUpdate parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new LocationUpdate(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.LocationUpdate + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + private int messageCase_ = 0; + private java.lang.Object message_; + + public enum MessageCase implements com.google.protobuf.Internal.EnumLite { + RIDE_STARTED(2), + RIDE_ENDED(3), + LOCATION_UPDATE(4), + MESSAGE_NOT_SET(0); + private final int value; + + private MessageCase(int value) { + this.value = value; + } + /** @deprecated Use {@link #forNumber(int)} instead. */ + @java.lang.Deprecated + public static MessageCase valueOf(int value) { + return forNumber(value); + } + + public static MessageCase forNumber(int value) { + switch (value) { + case 2: + return RIDE_STARTED; + case 3: + return RIDE_ENDED; + case 4: + return LOCATION_UPDATE; + case 0: + return MESSAGE_NOT_SET; + default: + return null; + } + } + + public int getNumber() { + return this.value; + } + }; + + public MessageCase getMessageCase() { + return MessageCase.forNumber(messageCase_); + } + + public static final int DRIVER_ID_FIELD_NUMBER = 1; + private volatile java.lang.Object driverId_; + /** string driver_id = 1; */ + public java.lang.String getDriverId() { + java.lang.Object ref = driverId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + driverId_ = s; + return s; + } + } + /** string driver_id = 1; */ + public com.google.protobuf.ByteString getDriverIdBytes() { + java.lang.Object ref = driverId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + driverId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int RIDE_STARTED_FIELD_NUMBER = 2; + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideStarted ride_started = 2; + * + */ + public boolean hasRideStarted() { + return messageCase_ == 2; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideStarted ride_started = 2; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted + getRideStarted() { + if (messageCase_ == 2) { + return (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted + .getDefaultInstance(); + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideStarted ride_started = 2; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStartedOrBuilder + getRideStartedOrBuilder() { + if (messageCase_ == 2) { + return (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted + .getDefaultInstance(); + } + + public static final int RIDE_ENDED_FIELD_NUMBER = 3; + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideEnded ride_ended = 3; + * + */ + public boolean hasRideEnded() { + return messageCase_ == 3; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideEnded ride_ended = 3; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + getRideEnded() { + if (messageCase_ == 3) { + return (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + .getDefaultInstance(); + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideEnded ride_ended = 3; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEndedOrBuilder + getRideEndedOrBuilder() { + if (messageCase_ == 3) { + return (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + .getDefaultInstance(); + } + + public static final int LOCATION_UPDATE_FIELD_NUMBER = 4; + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.LocationUpdate location_update = 4; + * + */ + public boolean hasLocationUpdate() { + return messageCase_ == 4; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.LocationUpdate location_update = 4; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.LocationUpdate + getLocationUpdate() { + if (messageCase_ == 4) { + return (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.LocationUpdate + .getDefaultInstance(); + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.LocationUpdate location_update = 4; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdateOrBuilder + getLocationUpdateOrBuilder() { + if (messageCase_ == 4) { + return (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.LocationUpdate + .getDefaultInstance(); + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!getDriverIdBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, driverId_); + } + if (messageCase_ == 2) { + output.writeMessage( + 2, + (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted) + message_); + } + if (messageCase_ == 3) { + output.writeMessage( + 3, + (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded) + message_); + } + if (messageCase_ == 4) { + output.writeMessage( + 4, + (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate) + message_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!getDriverIdBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, driverId_); + } + if (messageCase_ == 2) { + size += + com.google.protobuf.CodedOutputStream.computeMessageSize( + 2, + (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted) + message_); + } + if (messageCase_ == 3) { + size += + com.google.protobuf.CodedOutputStream.computeMessageSize( + 3, + (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded) + message_); + } + if (messageCase_ == 4) { + size += + com.google.protobuf.CodedOutputStream.computeMessageSize( + 4, + (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate) + message_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage)) { + return super.equals(obj); + } + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage other = + (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage) obj; + + if (!getDriverId().equals(other.getDriverId())) return false; + if (!getMessageCase().equals(other.getMessageCase())) return false; + switch (messageCase_) { + case 2: + if (!getRideStarted().equals(other.getRideStarted())) return false; + break; + case 3: + if (!getRideEnded().equals(other.getRideEnded())) return false; + break; + case 4: + if (!getLocationUpdate().equals(other.getLocationUpdate())) return false; + break; + case 0: + default: + } + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + DRIVER_ID_FIELD_NUMBER; + hash = (53 * hash) + getDriverId().hashCode(); + switch (messageCase_) { + case 2: + hash = (37 * hash) + RIDE_STARTED_FIELD_NUMBER; + hash = (53 * hash) + getRideStarted().hashCode(); + break; + case 3: + hash = (37 * hash) + RIDE_ENDED_FIELD_NUMBER; + hash = (53 * hash) + getRideEnded().hashCode(); + break; + case 4: + hash = (37 * hash) + LOCATION_UPDATE_FIELD_NUMBER; + hash = (53 * hash) + getLocationUpdate().hashCode(); + break; + case 0: + default: + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + parseFrom(byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + parseFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + parseFrom(com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * + * + *
+   * the messages that corresponds to the FROM_DRIVER_INGRESS
+   * 
+ * + * Protobuf type {@code com.ververica.statefun.examples.ridesharing.InboundDriverMessage} + */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.ridesharing.InboundDriverMessage) + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessageOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.class, + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.Builder + .class); + } + + // Construct using + // com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + driverId_ = ""; + + messageCase_ = 0; + message_ = null; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + getDefaultInstanceForType() { + return com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage build() { + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage result = + buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + buildPartial() { + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage result = + new com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage(this); + result.driverId_ = driverId_; + if (messageCase_ == 2) { + if (rideStartedBuilder_ == null) { + result.message_ = message_; + } else { + result.message_ = rideStartedBuilder_.build(); + } + } + if (messageCase_ == 3) { + if (rideEndedBuilder_ == null) { + result.message_ = message_; + } else { + result.message_ = rideEndedBuilder_.build(); + } + } + if (messageCase_ == 4) { + if (locationUpdateBuilder_ == null) { + result.message_ = message_; + } else { + result.message_ = locationUpdateBuilder_.build(); + } + } + result.messageCase_ = messageCase_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other + instanceof com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage) { + return mergeFrom( + (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage other) { + if (other + == com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .getDefaultInstance()) return this; + if (!other.getDriverId().isEmpty()) { + driverId_ = other.driverId_; + onChanged(); + } + switch (other.getMessageCase()) { + case RIDE_STARTED: + { + mergeRideStarted(other.getRideStarted()); + break; + } + case RIDE_ENDED: + { + mergeRideEnded(other.getRideEnded()); + break; + } + case LOCATION_UPDATE: + { + mergeLocationUpdate(other.getLocationUpdate()); + break; + } + case MESSAGE_NOT_SET: + { + break; + } + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage parsedMessage = + null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int messageCase_ = 0; + private java.lang.Object message_; + + public MessageCase getMessageCase() { + return MessageCase.forNumber(messageCase_); + } + + public Builder clearMessage() { + messageCase_ = 0; + message_ = null; + onChanged(); + return this; + } + + private java.lang.Object driverId_ = ""; + /** string driver_id = 1; */ + public java.lang.String getDriverId() { + java.lang.Object ref = driverId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + driverId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string driver_id = 1; */ + public com.google.protobuf.ByteString getDriverIdBytes() { + java.lang.Object ref = driverId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + driverId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string driver_id = 1; */ + public Builder setDriverId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + driverId_ = value; + onChanged(); + return this; + } + /** string driver_id = 1; */ + public Builder clearDriverId() { + + driverId_ = getDefaultInstance().getDriverId(); + onChanged(); + return this; + } + /** string driver_id = 1; */ + public Builder setDriverIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + driverId_ = value; + onChanged(); + return this; + } + + private com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted, + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted + .Builder, + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStartedOrBuilder> + rideStartedBuilder_; + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideStarted ride_started = 2; + * + */ + public boolean hasRideStarted() { + return messageCase_ == 2; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideStarted ride_started = 2; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted + getRideStarted() { + if (rideStartedBuilder_ == null) { + if (messageCase_ == 2) { + return (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted.getDefaultInstance(); + } else { + if (messageCase_ == 2) { + return rideStartedBuilder_.getMessage(); + } + return com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted.getDefaultInstance(); + } + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideStarted ride_started = 2; + * + */ + public Builder setRideStarted( + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted + value) { + if (rideStartedBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + message_ = value; + onChanged(); + } else { + rideStartedBuilder_.setMessage(value); + } + messageCase_ = 2; + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideStarted ride_started = 2; + * + */ + public Builder setRideStarted( + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted + .Builder + builderForValue) { + if (rideStartedBuilder_ == null) { + message_ = builderForValue.build(); + onChanged(); + } else { + rideStartedBuilder_.setMessage(builderForValue.build()); + } + messageCase_ = 2; + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideStarted ride_started = 2; + * + */ + public Builder mergeRideStarted( + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted + value) { + if (rideStartedBuilder_ == null) { + if (messageCase_ == 2 + && message_ + != com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted.getDefaultInstance()) { + message_ = + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted + .newBuilder( + (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted) + message_) + .mergeFrom(value) + .buildPartial(); + } else { + message_ = value; + } + onChanged(); + } else { + if (messageCase_ == 2) { + rideStartedBuilder_.mergeFrom(value); + } + rideStartedBuilder_.setMessage(value); + } + messageCase_ = 2; + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideStarted ride_started = 2; + * + */ + public Builder clearRideStarted() { + if (rideStartedBuilder_ == null) { + if (messageCase_ == 2) { + messageCase_ = 0; + message_ = null; + onChanged(); + } + } else { + if (messageCase_ == 2) { + messageCase_ = 0; + message_ = null; + } + rideStartedBuilder_.clear(); + } + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideStarted ride_started = 2; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted + .Builder + getRideStartedBuilder() { + return getRideStartedFieldBuilder().getBuilder(); + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideStarted ride_started = 2; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStartedOrBuilder + getRideStartedOrBuilder() { + if ((messageCase_ == 2) && (rideStartedBuilder_ != null)) { + return rideStartedBuilder_.getMessageOrBuilder(); + } else { + if (messageCase_ == 2) { + return (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted.getDefaultInstance(); + } + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideStarted ride_started = 2; + * + */ + private com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted, + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted + .Builder, + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStartedOrBuilder> + getRideStartedFieldBuilder() { + if (rideStartedBuilder_ == null) { + if (!(messageCase_ == 2)) { + message_ = + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted + .getDefaultInstance(); + } + rideStartedBuilder_ = + new com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted, + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted.Builder, + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStartedOrBuilder>( + (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideStarted) + message_, + getParentForChildren(), + isClean()); + message_ = null; + } + messageCase_ = 2; + onChanged(); + ; + return rideStartedBuilder_; + } + + private com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded, + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + .Builder, + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEndedOrBuilder> + rideEndedBuilder_; + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideEnded ride_ended = 3; + * + */ + public boolean hasRideEnded() { + return messageCase_ == 3; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideEnded ride_ended = 3; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + getRideEnded() { + if (rideEndedBuilder_ == null) { + if (messageCase_ == 3) { + return (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEnded) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + .getDefaultInstance(); + } else { + if (messageCase_ == 3) { + return rideEndedBuilder_.getMessage(); + } + return com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + .getDefaultInstance(); + } + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideEnded ride_ended = 3; + * + */ + public Builder setRideEnded( + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + value) { + if (rideEndedBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + message_ = value; + onChanged(); + } else { + rideEndedBuilder_.setMessage(value); + } + messageCase_ = 3; + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideEnded ride_ended = 3; + * + */ + public Builder setRideEnded( + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded.Builder + builderForValue) { + if (rideEndedBuilder_ == null) { + message_ = builderForValue.build(); + onChanged(); + } else { + rideEndedBuilder_.setMessage(builderForValue.build()); + } + messageCase_ = 3; + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideEnded ride_ended = 3; + * + */ + public Builder mergeRideEnded( + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + value) { + if (rideEndedBuilder_ == null) { + if (messageCase_ == 3 + && message_ + != com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEnded.getDefaultInstance()) { + message_ = + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + .newBuilder( + (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEnded) + message_) + .mergeFrom(value) + .buildPartial(); + } else { + message_ = value; + } + onChanged(); + } else { + if (messageCase_ == 3) { + rideEndedBuilder_.mergeFrom(value); + } + rideEndedBuilder_.setMessage(value); + } + messageCase_ = 3; + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideEnded ride_ended = 3; + * + */ + public Builder clearRideEnded() { + if (rideEndedBuilder_ == null) { + if (messageCase_ == 3) { + messageCase_ = 0; + message_ = null; + onChanged(); + } + } else { + if (messageCase_ == 3) { + messageCase_ = 0; + message_ = null; + } + rideEndedBuilder_.clear(); + } + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideEnded ride_ended = 3; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + .Builder + getRideEndedBuilder() { + return getRideEndedFieldBuilder().getBuilder(); + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideEnded ride_ended = 3; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEndedOrBuilder + getRideEndedOrBuilder() { + if ((messageCase_ == 3) && (rideEndedBuilder_ != null)) { + return rideEndedBuilder_.getMessageOrBuilder(); + } else { + if (messageCase_ == 3) { + return (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEnded) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + .getDefaultInstance(); + } + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideEnded ride_ended = 3; + * + */ + private com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded, + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + .Builder, + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEndedOrBuilder> + getRideEndedFieldBuilder() { + if (rideEndedBuilder_ == null) { + if (!(messageCase_ == 3)) { + message_ = + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + .getDefaultInstance(); + } + rideEndedBuilder_ = + new com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEnded, + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + .Builder, + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEndedOrBuilder>( + (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .RideEnded) + message_, + getParentForChildren(), + isClean()); + message_ = null; + } + messageCase_ = 3; + onChanged(); + ; + return rideEndedBuilder_; + } + + private com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate, + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate.Builder, + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdateOrBuilder> + locationUpdateBuilder_; + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.LocationUpdate location_update = 4; + * + */ + public boolean hasLocationUpdate() { + return messageCase_ == 4; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.LocationUpdate location_update = 4; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.LocationUpdate + getLocationUpdate() { + if (locationUpdateBuilder_ == null) { + if (messageCase_ == 4) { + return (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate.getDefaultInstance(); + } else { + if (messageCase_ == 4) { + return locationUpdateBuilder_.getMessage(); + } + return com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate.getDefaultInstance(); + } + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.LocationUpdate location_update = 4; + * + */ + public Builder setLocationUpdate( + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.LocationUpdate + value) { + if (locationUpdateBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + message_ = value; + onChanged(); + } else { + locationUpdateBuilder_.setMessage(value); + } + messageCase_ = 4; + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.LocationUpdate location_update = 4; + * + */ + public Builder setLocationUpdate( + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.LocationUpdate + .Builder + builderForValue) { + if (locationUpdateBuilder_ == null) { + message_ = builderForValue.build(); + onChanged(); + } else { + locationUpdateBuilder_.setMessage(builderForValue.build()); + } + messageCase_ = 4; + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.LocationUpdate location_update = 4; + * + */ + public Builder mergeLocationUpdate( + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.LocationUpdate + value) { + if (locationUpdateBuilder_ == null) { + if (messageCase_ == 4 + && message_ + != com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate.getDefaultInstance()) { + message_ = + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate.newBuilder( + (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate) + message_) + .mergeFrom(value) + .buildPartial(); + } else { + message_ = value; + } + onChanged(); + } else { + if (messageCase_ == 4) { + locationUpdateBuilder_.mergeFrom(value); + } + locationUpdateBuilder_.setMessage(value); + } + messageCase_ = 4; + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.LocationUpdate location_update = 4; + * + */ + public Builder clearLocationUpdate() { + if (locationUpdateBuilder_ == null) { + if (messageCase_ == 4) { + messageCase_ = 0; + message_ = null; + onChanged(); + } + } else { + if (messageCase_ == 4) { + messageCase_ = 0; + message_ = null; + } + locationUpdateBuilder_.clear(); + } + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.LocationUpdate location_update = 4; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.LocationUpdate + .Builder + getLocationUpdateBuilder() { + return getLocationUpdateFieldBuilder().getBuilder(); + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.LocationUpdate location_update = 4; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdateOrBuilder + getLocationUpdateOrBuilder() { + if ((messageCase_ == 4) && (locationUpdateBuilder_ != null)) { + return locationUpdateBuilder_.getMessageOrBuilder(); + } else { + if (messageCase_ == 4) { + return (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate.getDefaultInstance(); + } + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.LocationUpdate location_update = 4; + * + */ + private com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate, + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate.Builder, + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdateOrBuilder> + getLocationUpdateFieldBuilder() { + if (locationUpdateBuilder_ == null) { + if (!(messageCase_ == 4)) { + message_ = + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate.getDefaultInstance(); + } + locationUpdateBuilder_ = + new com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate, + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate.Builder, + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdateOrBuilder>( + (com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + .LocationUpdate) + message_, + getParentForChildren(), + isClean()); + message_ = null; + } + messageCase_ = 4; + onChanged(); + ; + return locationUpdateBuilder_; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.ridesharing.InboundDriverMessage) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.ridesharing.InboundDriverMessage) + private static final com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage(); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public InboundDriverMessage parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new InboundDriverMessage(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/InboundDriverMessageOrBuilder.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/InboundDriverMessageOrBuilder.java new file mode 100644 index 00000000..f283eaf9 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/InboundDriverMessageOrBuilder.java @@ -0,0 +1,81 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +public interface InboundDriverMessageOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.ridesharing.InboundDriverMessage) + com.google.protobuf.MessageOrBuilder { + + /** string driver_id = 1; */ + java.lang.String getDriverId(); + /** string driver_id = 1; */ + com.google.protobuf.ByteString getDriverIdBytes(); + + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideStarted ride_started = 2; + * + */ + boolean hasRideStarted(); + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideStarted ride_started = 2; + * + */ + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStarted + getRideStarted(); + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideStarted ride_started = 2; + * + */ + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideStartedOrBuilder + getRideStartedOrBuilder(); + + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideEnded ride_ended = 3; + * + */ + boolean hasRideEnded(); + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideEnded ride_ended = 3; + * + */ + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEnded + getRideEnded(); + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.RideEnded ride_ended = 3; + * + */ + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.RideEndedOrBuilder + getRideEndedOrBuilder(); + + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.LocationUpdate location_update = 4; + * + */ + boolean hasLocationUpdate(); + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.LocationUpdate location_update = 4; + * + */ + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.LocationUpdate + getLocationUpdate(); + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundDriverMessage.LocationUpdate location_update = 4; + * + */ + com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.LocationUpdateOrBuilder + getLocationUpdateOrBuilder(); + + public com.ververica.statefun.examples.ridesharing.generated.InboundDriverMessage.MessageCase + getMessageCase(); +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/InboundPassengerMessage.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/InboundPassengerMessage.java new file mode 100644 index 00000000..007282b6 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/InboundPassengerMessage.java @@ -0,0 +1,1548 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +/** + * + * + *
+ * the messages that corresponds to the FROM_PASSENGER_INGRESS
+ * 
+ * + * Protobuf type {@code com.ververica.statefun.examples.ridesharing.InboundPassengerMessage} + */ +public final class InboundPassengerMessage extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.ridesharing.InboundPassengerMessage) + InboundPassengerMessageOrBuilder { + private static final long serialVersionUID = 0L; + // Use InboundPassengerMessage.newBuilder() to construct. + private InboundPassengerMessage(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private InboundPassengerMessage() { + passengerId_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private InboundPassengerMessage( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + java.lang.String s = input.readStringRequireUtf8(); + + passengerId_ = s; + break; + } + case 18: + { + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide.Builder + subBuilder = null; + if (messageCase_ == 2) { + subBuilder = + ((com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide) + message_) + .toBuilder(); + } + message_ = + input.readMessage( + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide.parser(), + extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom( + (com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide) + message_); + message_ = subBuilder.buildPartial(); + } + messageCase_ = 2; + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_InboundPassengerMessage_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_InboundPassengerMessage_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage.class, + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage.Builder + .class); + } + + public interface RequestRideOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.ridesharing.InboundPassengerMessage.RequestRide) + com.google.protobuf.MessageOrBuilder { + + /** int32 start_geo_cell = 2; */ + int getStartGeoCell(); + + /** int32 end_geo_cell = 3; */ + int getEndGeoCell(); + } + /** + * + * + *
+   * A message sent by the physical driver to the ridesharing statefun job, indicating that the user
+   * would like to be picked up from start_geo_cell and dropped off at end_geo_cell
+   * 
+ * + * Protobuf type {@code + * com.ververica.statefun.examples.ridesharing.InboundPassengerMessage.RequestRide} + */ + public static final class RequestRide extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.ridesharing.InboundPassengerMessage.RequestRide) + RequestRideOrBuilder { + private static final long serialVersionUID = 0L; + // Use RequestRide.newBuilder() to construct. + private RequestRide(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private RequestRide() {} + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private RequestRide( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 16: + { + startGeoCell_ = input.readInt32(); + break; + } + case 24: + { + endGeoCell_ = input.readInt32(); + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_InboundPassengerMessage_RequestRide_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_InboundPassengerMessage_RequestRide_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide.class, + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide.Builder.class); + } + + public static final int START_GEO_CELL_FIELD_NUMBER = 2; + private int startGeoCell_; + /** int32 start_geo_cell = 2; */ + public int getStartGeoCell() { + return startGeoCell_; + } + + public static final int END_GEO_CELL_FIELD_NUMBER = 3; + private int endGeoCell_; + /** int32 end_geo_cell = 3; */ + public int getEndGeoCell() { + return endGeoCell_; + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (startGeoCell_ != 0) { + output.writeInt32(2, startGeoCell_); + } + if (endGeoCell_ != 0) { + output.writeInt32(3, endGeoCell_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (startGeoCell_ != 0) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(2, startGeoCell_); + } + if (endGeoCell_ != 0) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(3, endGeoCell_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide)) { + return super.equals(obj); + } + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage.RequestRide + other = + (com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide) + obj; + + if (getStartGeoCell() != other.getStartGeoCell()) return false; + if (getEndGeoCell() != other.getEndGeoCell()) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + START_GEO_CELL_FIELD_NUMBER; + hash = (53 * hash) + getStartGeoCell(); + hash = (37 * hash) + END_GEO_CELL_FIELD_NUMBER; + hash = (53 * hash) + getEndGeoCell(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide + parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide + parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide + parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide + parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide + parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide + parseFrom(byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide + parseFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide + parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide + parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide + parseFrom(com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide + parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage.RequestRide + prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * + * + *
+     * A message sent by the physical driver to the ridesharing statefun job, indicating that the user
+     * would like to be picked up from start_geo_cell and dropped off at end_geo_cell
+     * 
+ * + * Protobuf type {@code + * com.ververica.statefun.examples.ridesharing.InboundPassengerMessage.RequestRide} + */ + public static final class Builder + extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.ridesharing.InboundPassengerMessage.RequestRide) + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRideOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_InboundPassengerMessage_RequestRide_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_InboundPassengerMessage_RequestRide_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide.class, + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide.Builder.class); + } + + // Construct using + // com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage.RequestRide.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + startGeoCell_ = 0; + + endGeoCell_ = 0; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_InboundPassengerMessage_RequestRide_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide + getDefaultInstanceForType() { + return com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide.getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide + build() { + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage.RequestRide + result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide + buildPartial() { + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage.RequestRide + result = + new com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide(this); + result.startGeoCell_ = startGeoCell_; + result.endGeoCell_ = endGeoCell_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other + instanceof + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide) { + return mergeFrom( + (com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide) + other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage.RequestRide + other) { + if (other + == com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide.getDefaultInstance()) return this; + if (other.getStartGeoCell() != 0) { + setStartGeoCell(other.getStartGeoCell()); + } + if (other.getEndGeoCell() != 0) { + setEndGeoCell(other.getEndGeoCell()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage.RequestRide + parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int startGeoCell_; + /** int32 start_geo_cell = 2; */ + public int getStartGeoCell() { + return startGeoCell_; + } + /** int32 start_geo_cell = 2; */ + public Builder setStartGeoCell(int value) { + + startGeoCell_ = value; + onChanged(); + return this; + } + /** int32 start_geo_cell = 2; */ + public Builder clearStartGeoCell() { + + startGeoCell_ = 0; + onChanged(); + return this; + } + + private int endGeoCell_; + /** int32 end_geo_cell = 3; */ + public int getEndGeoCell() { + return endGeoCell_; + } + /** int32 end_geo_cell = 3; */ + public Builder setEndGeoCell(int value) { + + endGeoCell_ = value; + onChanged(); + return this; + } + /** int32 end_geo_cell = 3; */ + public Builder clearEndGeoCell() { + + endGeoCell_ = 0; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.ridesharing.InboundPassengerMessage.RequestRide) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.ridesharing.InboundPassengerMessage.RequestRide) + private static final com.ververica.statefun.examples.ridesharing.generated + .InboundPassengerMessage.RequestRide + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide(); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public RequestRide parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new RequestRide(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage.RequestRide + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + private int messageCase_ = 0; + private java.lang.Object message_; + + public enum MessageCase implements com.google.protobuf.Internal.EnumLite { + REQUEST_RIDE(2), + MESSAGE_NOT_SET(0); + private final int value; + + private MessageCase(int value) { + this.value = value; + } + /** @deprecated Use {@link #forNumber(int)} instead. */ + @java.lang.Deprecated + public static MessageCase valueOf(int value) { + return forNumber(value); + } + + public static MessageCase forNumber(int value) { + switch (value) { + case 2: + return REQUEST_RIDE; + case 0: + return MESSAGE_NOT_SET; + default: + return null; + } + } + + public int getNumber() { + return this.value; + } + }; + + public MessageCase getMessageCase() { + return MessageCase.forNumber(messageCase_); + } + + public static final int PASSENGER_ID_FIELD_NUMBER = 1; + private volatile java.lang.Object passengerId_; + /** string passenger_id = 1; */ + public java.lang.String getPassengerId() { + java.lang.Object ref = passengerId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + passengerId_ = s; + return s; + } + } + /** string passenger_id = 1; */ + public com.google.protobuf.ByteString getPassengerIdBytes() { + java.lang.Object ref = passengerId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + passengerId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int REQUEST_RIDE_FIELD_NUMBER = 2; + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundPassengerMessage.RequestRide request_ride = 2; + * + */ + public boolean hasRequestRide() { + return messageCase_ == 2; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundPassengerMessage.RequestRide request_ride = 2; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage.RequestRide + getRequestRide() { + if (messageCase_ == 2) { + return (com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage.RequestRide + .getDefaultInstance(); + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundPassengerMessage.RequestRide request_ride = 2; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRideOrBuilder + getRequestRideOrBuilder() { + if (messageCase_ == 2) { + return (com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage.RequestRide + .getDefaultInstance(); + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!getPassengerIdBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, passengerId_); + } + if (messageCase_ == 2) { + output.writeMessage( + 2, + (com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide) + message_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!getPassengerIdBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, passengerId_); + } + if (messageCase_ == 2) { + size += + com.google.protobuf.CodedOutputStream.computeMessageSize( + 2, + (com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide) + message_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage)) { + return super.equals(obj); + } + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage other = + (com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage) obj; + + if (!getPassengerId().equals(other.getPassengerId())) return false; + if (!getMessageCase().equals(other.getMessageCase())) return false; + switch (messageCase_) { + case 2: + if (!getRequestRide().equals(other.getRequestRide())) return false; + break; + case 0: + default: + } + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + PASSENGER_ID_FIELD_NUMBER; + hash = (53 * hash) + getPassengerId().hashCode(); + switch (messageCase_) { + case 2: + hash = (37 * hash) + REQUEST_RIDE_FIELD_NUMBER; + hash = (53 * hash) + getRequestRide().hashCode(); + break; + case 0: + default: + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + parseFrom(byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + parseFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + parseFrom(com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * + * + *
+   * the messages that corresponds to the FROM_PASSENGER_INGRESS
+   * 
+ * + * Protobuf type {@code com.ververica.statefun.examples.ridesharing.InboundPassengerMessage} + */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.ridesharing.InboundPassengerMessage) + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessageOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_InboundPassengerMessage_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_InboundPassengerMessage_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage.class, + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage.Builder + .class); + } + + // Construct using + // com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + passengerId_ = ""; + + messageCase_ = 0; + message_ = null; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_InboundPassengerMessage_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + getDefaultInstanceForType() { + return com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage build() { + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage result = + buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + buildPartial() { + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage result = + new com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage(this); + result.passengerId_ = passengerId_; + if (messageCase_ == 2) { + if (requestRideBuilder_ == null) { + result.message_ = message_; + } else { + result.message_ = requestRideBuilder_.build(); + } + } + result.messageCase_ = messageCase_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other + instanceof + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage) { + return mergeFrom( + (com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage other) { + if (other + == com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .getDefaultInstance()) return this; + if (!other.getPassengerId().isEmpty()) { + passengerId_ = other.passengerId_; + onChanged(); + } + switch (other.getMessageCase()) { + case REQUEST_RIDE: + { + mergeRequestRide(other.getRequestRide()); + break; + } + case MESSAGE_NOT_SET: + { + break; + } + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage parsedMessage = + null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int messageCase_ = 0; + private java.lang.Object message_; + + public MessageCase getMessageCase() { + return MessageCase.forNumber(messageCase_); + } + + public Builder clearMessage() { + messageCase_ = 0; + message_ = null; + onChanged(); + return this; + } + + private java.lang.Object passengerId_ = ""; + /** string passenger_id = 1; */ + public java.lang.String getPassengerId() { + java.lang.Object ref = passengerId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + passengerId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string passenger_id = 1; */ + public com.google.protobuf.ByteString getPassengerIdBytes() { + java.lang.Object ref = passengerId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + passengerId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string passenger_id = 1; */ + public Builder setPassengerId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + passengerId_ = value; + onChanged(); + return this; + } + /** string passenger_id = 1; */ + public Builder clearPassengerId() { + + passengerId_ = getDefaultInstance().getPassengerId(); + onChanged(); + return this; + } + /** string passenger_id = 1; */ + public Builder setPassengerIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + passengerId_ = value; + onChanged(); + return this; + } + + private com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide, + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide.Builder, + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRideOrBuilder> + requestRideBuilder_; + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundPassengerMessage.RequestRide request_ride = 2; + * + */ + public boolean hasRequestRide() { + return messageCase_ == 2; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundPassengerMessage.RequestRide request_ride = 2; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage.RequestRide + getRequestRide() { + if (requestRideBuilder_ == null) { + if (messageCase_ == 2) { + return (com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide.getDefaultInstance(); + } else { + if (messageCase_ == 2) { + return requestRideBuilder_.getMessage(); + } + return com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide.getDefaultInstance(); + } + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundPassengerMessage.RequestRide request_ride = 2; + * + */ + public Builder setRequestRide( + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage.RequestRide + value) { + if (requestRideBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + message_ = value; + onChanged(); + } else { + requestRideBuilder_.setMessage(value); + } + messageCase_ = 2; + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundPassengerMessage.RequestRide request_ride = 2; + * + */ + public Builder setRequestRide( + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage.RequestRide + .Builder + builderForValue) { + if (requestRideBuilder_ == null) { + message_ = builderForValue.build(); + onChanged(); + } else { + requestRideBuilder_.setMessage(builderForValue.build()); + } + messageCase_ = 2; + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundPassengerMessage.RequestRide request_ride = 2; + * + */ + public Builder mergeRequestRide( + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage.RequestRide + value) { + if (requestRideBuilder_ == null) { + if (messageCase_ == 2 + && message_ + != com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide.getDefaultInstance()) { + message_ = + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide.newBuilder( + (com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide) + message_) + .mergeFrom(value) + .buildPartial(); + } else { + message_ = value; + } + onChanged(); + } else { + if (messageCase_ == 2) { + requestRideBuilder_.mergeFrom(value); + } + requestRideBuilder_.setMessage(value); + } + messageCase_ = 2; + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundPassengerMessage.RequestRide request_ride = 2; + * + */ + public Builder clearRequestRide() { + if (requestRideBuilder_ == null) { + if (messageCase_ == 2) { + messageCase_ = 0; + message_ = null; + onChanged(); + } + } else { + if (messageCase_ == 2) { + messageCase_ = 0; + message_ = null; + } + requestRideBuilder_.clear(); + } + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundPassengerMessage.RequestRide request_ride = 2; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage.RequestRide + .Builder + getRequestRideBuilder() { + return getRequestRideFieldBuilder().getBuilder(); + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundPassengerMessage.RequestRide request_ride = 2; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRideOrBuilder + getRequestRideOrBuilder() { + if ((messageCase_ == 2) && (requestRideBuilder_ != null)) { + return requestRideBuilder_.getMessageOrBuilder(); + } else { + if (messageCase_ == 2) { + return (com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide.getDefaultInstance(); + } + } + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundPassengerMessage.RequestRide request_ride = 2; + * + */ + private com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide, + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide.Builder, + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRideOrBuilder> + getRequestRideFieldBuilder() { + if (requestRideBuilder_ == null) { + if (!(messageCase_ == 2)) { + message_ = + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide.getDefaultInstance(); + } + requestRideBuilder_ = + new com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide, + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide.Builder, + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRideOrBuilder>( + (com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + .RequestRide) + message_, + getParentForChildren(), + isClean()); + message_ = null; + } + messageCase_ = 2; + onChanged(); + ; + return requestRideBuilder_; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.ridesharing.InboundPassengerMessage) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.ridesharing.InboundPassengerMessage) + private static final com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage(); + } + + public static com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public InboundPassengerMessage parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new InboundPassengerMessage(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/InboundPassengerMessageOrBuilder.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/InboundPassengerMessageOrBuilder.java new file mode 100644 index 00000000..f91119cc --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/InboundPassengerMessageOrBuilder.java @@ -0,0 +1,39 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +public interface InboundPassengerMessageOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.ridesharing.InboundPassengerMessage) + com.google.protobuf.MessageOrBuilder { + + /** string passenger_id = 1; */ + java.lang.String getPassengerId(); + /** string passenger_id = 1; */ + com.google.protobuf.ByteString getPassengerIdBytes(); + + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundPassengerMessage.RequestRide request_ride = 2; + * + */ + boolean hasRequestRide(); + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundPassengerMessage.RequestRide request_ride = 2; + * + */ + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage.RequestRide + getRequestRide(); + /** + * + * .com.ververica.statefun.examples.ridesharing.InboundPassengerMessage.RequestRide request_ride = 2; + * + */ + com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage.RequestRideOrBuilder + getRequestRideOrBuilder(); + + public com.ververica.statefun.examples.ridesharing.generated.InboundPassengerMessage.MessageCase + getMessageCase(); +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/JoinCell.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/JoinCell.java new file mode 100644 index 00000000..21795f79 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/JoinCell.java @@ -0,0 +1,426 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +/** Protobuf type {@code com.ververica.statefun.examples.ridesharing.JoinCell} */ +public final class JoinCell extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.ridesharing.JoinCell) + JoinCellOrBuilder { + private static final long serialVersionUID = 0L; + // Use JoinCell.newBuilder() to construct. + private JoinCell(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private JoinCell() {} + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private JoinCell( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_JoinCell_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_JoinCell_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.JoinCell.class, + com.ververica.statefun.examples.ridesharing.generated.JoinCell.Builder.class); + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.ververica.statefun.examples.ridesharing.generated.JoinCell)) { + return super.equals(obj); + } + com.ververica.statefun.examples.ridesharing.generated.JoinCell other = + (com.ververica.statefun.examples.ridesharing.generated.JoinCell) obj; + + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.ridesharing.generated.JoinCell parseFrom( + java.nio.ByteBuffer data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.JoinCell parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.JoinCell parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.JoinCell parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.JoinCell parseFrom( + byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.JoinCell parseFrom( + byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.JoinCell parseFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.JoinCell parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.JoinCell parseDelimitedFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.JoinCell parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.JoinCell parseFrom( + com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.JoinCell parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.ridesharing.generated.JoinCell prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code com.ververica.statefun.examples.ridesharing.JoinCell} */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.ridesharing.JoinCell) + com.ververica.statefun.examples.ridesharing.generated.JoinCellOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_JoinCell_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_JoinCell_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.JoinCell.class, + com.ververica.statefun.examples.ridesharing.generated.JoinCell.Builder.class); + } + + // Construct using com.ververica.statefun.examples.ridesharing.generated.JoinCell.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_JoinCell_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.JoinCell + getDefaultInstanceForType() { + return com.ververica.statefun.examples.ridesharing.generated.JoinCell.getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.JoinCell build() { + com.ververica.statefun.examples.ridesharing.generated.JoinCell result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.JoinCell buildPartial() { + com.ververica.statefun.examples.ridesharing.generated.JoinCell result = + new com.ververica.statefun.examples.ridesharing.generated.JoinCell(this); + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.ververica.statefun.examples.ridesharing.generated.JoinCell) { + return mergeFrom((com.ververica.statefun.examples.ridesharing.generated.JoinCell) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.ververica.statefun.examples.ridesharing.generated.JoinCell other) { + if (other + == com.ververica.statefun.examples.ridesharing.generated.JoinCell.getDefaultInstance()) + return this; + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.ridesharing.generated.JoinCell parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.ridesharing.generated.JoinCell) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.ridesharing.JoinCell) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.ridesharing.JoinCell) + private static final com.ververica.statefun.examples.ridesharing.generated.JoinCell + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = new com.ververica.statefun.examples.ridesharing.generated.JoinCell(); + } + + public static com.ververica.statefun.examples.ridesharing.generated.JoinCell + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public JoinCell parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new JoinCell(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.JoinCell + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/JoinCellOrBuilder.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/JoinCellOrBuilder.java new file mode 100644 index 00000000..b0518b54 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/JoinCellOrBuilder.java @@ -0,0 +1,9 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +public interface JoinCellOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.ridesharing.JoinCell) + com.google.protobuf.MessageOrBuilder {} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/LeaveCell.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/LeaveCell.java new file mode 100644 index 00000000..6d7f2c78 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/LeaveCell.java @@ -0,0 +1,427 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +/** Protobuf type {@code com.ververica.statefun.examples.ridesharing.LeaveCell} */ +public final class LeaveCell extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.ridesharing.LeaveCell) + LeaveCellOrBuilder { + private static final long serialVersionUID = 0L; + // Use LeaveCell.newBuilder() to construct. + private LeaveCell(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private LeaveCell() {} + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private LeaveCell( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_LeaveCell_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_LeaveCell_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.LeaveCell.class, + com.ververica.statefun.examples.ridesharing.generated.LeaveCell.Builder.class); + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.ververica.statefun.examples.ridesharing.generated.LeaveCell)) { + return super.equals(obj); + } + com.ververica.statefun.examples.ridesharing.generated.LeaveCell other = + (com.ververica.statefun.examples.ridesharing.generated.LeaveCell) obj; + + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.ridesharing.generated.LeaveCell parseFrom( + java.nio.ByteBuffer data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.LeaveCell parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.LeaveCell parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.LeaveCell parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.LeaveCell parseFrom( + byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.LeaveCell parseFrom( + byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.LeaveCell parseFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.LeaveCell parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.LeaveCell parseDelimitedFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.LeaveCell parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.LeaveCell parseFrom( + com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.LeaveCell parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.ridesharing.generated.LeaveCell prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code com.ververica.statefun.examples.ridesharing.LeaveCell} */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.ridesharing.LeaveCell) + com.ververica.statefun.examples.ridesharing.generated.LeaveCellOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_LeaveCell_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_LeaveCell_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.LeaveCell.class, + com.ververica.statefun.examples.ridesharing.generated.LeaveCell.Builder.class); + } + + // Construct using com.ververica.statefun.examples.ridesharing.generated.LeaveCell.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_LeaveCell_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.LeaveCell + getDefaultInstanceForType() { + return com.ververica.statefun.examples.ridesharing.generated.LeaveCell.getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.LeaveCell build() { + com.ververica.statefun.examples.ridesharing.generated.LeaveCell result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.LeaveCell buildPartial() { + com.ververica.statefun.examples.ridesharing.generated.LeaveCell result = + new com.ververica.statefun.examples.ridesharing.generated.LeaveCell(this); + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.ververica.statefun.examples.ridesharing.generated.LeaveCell) { + return mergeFrom((com.ververica.statefun.examples.ridesharing.generated.LeaveCell) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.ridesharing.generated.LeaveCell other) { + if (other + == com.ververica.statefun.examples.ridesharing.generated.LeaveCell.getDefaultInstance()) + return this; + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.ridesharing.generated.LeaveCell parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.ridesharing.generated.LeaveCell) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.ridesharing.LeaveCell) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.ridesharing.LeaveCell) + private static final com.ververica.statefun.examples.ridesharing.generated.LeaveCell + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = new com.ververica.statefun.examples.ridesharing.generated.LeaveCell(); + } + + public static com.ververica.statefun.examples.ridesharing.generated.LeaveCell + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public LeaveCell parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new LeaveCell(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.LeaveCell + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/LeaveCellOrBuilder.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/LeaveCellOrBuilder.java new file mode 100644 index 00000000..ef2b5392 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/LeaveCellOrBuilder.java @@ -0,0 +1,9 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +public interface LeaveCellOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.ridesharing.LeaveCell) + com.google.protobuf.MessageOrBuilder {} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/OutboundDriverMessage.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/OutboundDriverMessage.java new file mode 100644 index 00000000..f97f4e26 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/OutboundDriverMessage.java @@ -0,0 +1,1663 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +/** + * + * + *
+ * the messages that corresponds to the TO_DRIVER_EGRESS
+ * 
+ * + * Protobuf type {@code com.ververica.statefun.examples.ridesharing.OutboundDriverMessage} + */ +public final class OutboundDriverMessage extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.ridesharing.OutboundDriverMessage) + OutboundDriverMessageOrBuilder { + private static final long serialVersionUID = 0L; + // Use OutboundDriverMessage.newBuilder() to construct. + private OutboundDriverMessage(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private OutboundDriverMessage() { + driverId_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private OutboundDriverMessage( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + java.lang.String s = input.readStringRequireUtf8(); + + driverId_ = s; + break; + } + case 18: + { + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger.Builder + subBuilder = null; + if (messageCase_ == 2) { + subBuilder = + ((com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger) + message_) + .toBuilder(); + } + message_ = + input.readMessage( + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger.parser(), + extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom( + (com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger) + message_); + message_ = subBuilder.buildPartial(); + } + messageCase_ = 2; + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundDriverMessage_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundDriverMessage_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage.class, + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage.Builder + .class); + } + + public interface PickupPassengerOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.ridesharing.OutboundDriverMessage.PickupPassenger) + com.google.protobuf.MessageOrBuilder { + + /** string ride_id = 2; */ + java.lang.String getRideId(); + /** string ride_id = 2; */ + com.google.protobuf.ByteString getRideIdBytes(); + + /** int32 start_geo_location = 3; */ + int getStartGeoLocation(); + + /** int32 end_geo_location = 4; */ + int getEndGeoLocation(); + } + /** + * + * + *
+   * A message that is sent to the physical driver to indicate that
+   * a pickup is needed.
+   * a pickup is needed.
+   * 
+ * + * Protobuf type {@code + * com.ververica.statefun.examples.ridesharing.OutboundDriverMessage.PickupPassenger} + */ + public static final class PickupPassenger extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.ridesharing.OutboundDriverMessage.PickupPassenger) + PickupPassengerOrBuilder { + private static final long serialVersionUID = 0L; + // Use PickupPassenger.newBuilder() to construct. + private PickupPassenger(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private PickupPassenger() { + rideId_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private PickupPassenger( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 18: + { + java.lang.String s = input.readStringRequireUtf8(); + + rideId_ = s; + break; + } + case 24: + { + startGeoLocation_ = input.readInt32(); + break; + } + case 32: + { + endGeoLocation_ = input.readInt32(); + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundDriverMessage_PickupPassenger_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundDriverMessage_PickupPassenger_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger.class, + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger.Builder.class); + } + + public static final int RIDE_ID_FIELD_NUMBER = 2; + private volatile java.lang.Object rideId_; + /** string ride_id = 2; */ + public java.lang.String getRideId() { + java.lang.Object ref = rideId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + rideId_ = s; + return s; + } + } + /** string ride_id = 2; */ + public com.google.protobuf.ByteString getRideIdBytes() { + java.lang.Object ref = rideId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + rideId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int START_GEO_LOCATION_FIELD_NUMBER = 3; + private int startGeoLocation_; + /** int32 start_geo_location = 3; */ + public int getStartGeoLocation() { + return startGeoLocation_; + } + + public static final int END_GEO_LOCATION_FIELD_NUMBER = 4; + private int endGeoLocation_; + /** int32 end_geo_location = 4; */ + public int getEndGeoLocation() { + return endGeoLocation_; + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!getRideIdBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 2, rideId_); + } + if (startGeoLocation_ != 0) { + output.writeInt32(3, startGeoLocation_); + } + if (endGeoLocation_ != 0) { + output.writeInt32(4, endGeoLocation_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!getRideIdBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, rideId_); + } + if (startGeoLocation_ != 0) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(3, startGeoLocation_); + } + if (endGeoLocation_ != 0) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(4, endGeoLocation_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger)) { + return super.equals(obj); + } + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage.PickupPassenger + other = + (com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger) + obj; + + if (!getRideId().equals(other.getRideId())) return false; + if (getStartGeoLocation() != other.getStartGeoLocation()) return false; + if (getEndGeoLocation() != other.getEndGeoLocation()) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + RIDE_ID_FIELD_NUMBER; + hash = (53 * hash) + getRideId().hashCode(); + hash = (37 * hash) + START_GEO_LOCATION_FIELD_NUMBER; + hash = (53 * hash) + getStartGeoLocation(); + hash = (37 * hash) + END_GEO_LOCATION_FIELD_NUMBER; + hash = (53 * hash) + getEndGeoLocation(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger + parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger + parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger + parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger + parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger + parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger + parseFrom(byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger + parseFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger + parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger + parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger + parseFrom(com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger + parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage.PickupPassenger + prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * + * + *
+     * A message that is sent to the physical driver to indicate that
+     * a pickup is needed.
+     * a pickup is needed.
+     * 
+ * + * Protobuf type {@code + * com.ververica.statefun.examples.ridesharing.OutboundDriverMessage.PickupPassenger} + */ + public static final class Builder + extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.ridesharing.OutboundDriverMessage.PickupPassenger) + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassengerOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundDriverMessage_PickupPassenger_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundDriverMessage_PickupPassenger_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger.class, + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger.Builder.class); + } + + // Construct using + // com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage.PickupPassenger.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + rideId_ = ""; + + startGeoLocation_ = 0; + + endGeoLocation_ = 0; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundDriverMessage_PickupPassenger_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger + getDefaultInstanceForType() { + return com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger.getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger + build() { + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage.PickupPassenger + result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger + buildPartial() { + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage.PickupPassenger + result = + new com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger(this); + result.rideId_ = rideId_; + result.startGeoLocation_ = startGeoLocation_; + result.endGeoLocation_ = endGeoLocation_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other + instanceof + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger) { + return mergeFrom( + (com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger) + other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger + other) { + if (other + == com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger.getDefaultInstance()) return this; + if (!other.getRideId().isEmpty()) { + rideId_ = other.rideId_; + onChanged(); + } + if (other.getStartGeoLocation() != 0) { + setStartGeoLocation(other.getStartGeoLocation()); + } + if (other.getEndGeoLocation() != 0) { + setEndGeoLocation(other.getEndGeoLocation()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage.PickupPassenger + parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private java.lang.Object rideId_ = ""; + /** string ride_id = 2; */ + public java.lang.String getRideId() { + java.lang.Object ref = rideId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + rideId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string ride_id = 2; */ + public com.google.protobuf.ByteString getRideIdBytes() { + java.lang.Object ref = rideId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + rideId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string ride_id = 2; */ + public Builder setRideId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + rideId_ = value; + onChanged(); + return this; + } + /** string ride_id = 2; */ + public Builder clearRideId() { + + rideId_ = getDefaultInstance().getRideId(); + onChanged(); + return this; + } + /** string ride_id = 2; */ + public Builder setRideIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + rideId_ = value; + onChanged(); + return this; + } + + private int startGeoLocation_; + /** int32 start_geo_location = 3; */ + public int getStartGeoLocation() { + return startGeoLocation_; + } + /** int32 start_geo_location = 3; */ + public Builder setStartGeoLocation(int value) { + + startGeoLocation_ = value; + onChanged(); + return this; + } + /** int32 start_geo_location = 3; */ + public Builder clearStartGeoLocation() { + + startGeoLocation_ = 0; + onChanged(); + return this; + } + + private int endGeoLocation_; + /** int32 end_geo_location = 4; */ + public int getEndGeoLocation() { + return endGeoLocation_; + } + /** int32 end_geo_location = 4; */ + public Builder setEndGeoLocation(int value) { + + endGeoLocation_ = value; + onChanged(); + return this; + } + /** int32 end_geo_location = 4; */ + public Builder clearEndGeoLocation() { + + endGeoLocation_ = 0; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.ridesharing.OutboundDriverMessage.PickupPassenger) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.ridesharing.OutboundDriverMessage.PickupPassenger) + private static final com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger(); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public PickupPassenger parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new PickupPassenger(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + private int messageCase_ = 0; + private java.lang.Object message_; + + public enum MessageCase implements com.google.protobuf.Internal.EnumLite { + PICKUP_PASSENGER(2), + MESSAGE_NOT_SET(0); + private final int value; + + private MessageCase(int value) { + this.value = value; + } + /** @deprecated Use {@link #forNumber(int)} instead. */ + @java.lang.Deprecated + public static MessageCase valueOf(int value) { + return forNumber(value); + } + + public static MessageCase forNumber(int value) { + switch (value) { + case 2: + return PICKUP_PASSENGER; + case 0: + return MESSAGE_NOT_SET; + default: + return null; + } + } + + public int getNumber() { + return this.value; + } + }; + + public MessageCase getMessageCase() { + return MessageCase.forNumber(messageCase_); + } + + public static final int DRIVER_ID_FIELD_NUMBER = 1; + private volatile java.lang.Object driverId_; + /** string driver_id = 1; */ + public java.lang.String getDriverId() { + java.lang.Object ref = driverId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + driverId_ = s; + return s; + } + } + /** string driver_id = 1; */ + public com.google.protobuf.ByteString getDriverIdBytes() { + java.lang.Object ref = driverId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + driverId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int PICKUP_PASSENGER_FIELD_NUMBER = 2; + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundDriverMessage.PickupPassenger pickup_passenger = 2; + * + */ + public boolean hasPickupPassenger() { + return messageCase_ == 2; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundDriverMessage.PickupPassenger pickup_passenger = 2; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage.PickupPassenger + getPickupPassenger() { + if (messageCase_ == 2) { + return (com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger.getDefaultInstance(); + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundDriverMessage.PickupPassenger pickup_passenger = 2; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassengerOrBuilder + getPickupPassengerOrBuilder() { + if (messageCase_ == 2) { + return (com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger.getDefaultInstance(); + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!getDriverIdBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, driverId_); + } + if (messageCase_ == 2) { + output.writeMessage( + 2, + (com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger) + message_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!getDriverIdBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, driverId_); + } + if (messageCase_ == 2) { + size += + com.google.protobuf.CodedOutputStream.computeMessageSize( + 2, + (com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger) + message_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage)) { + return super.equals(obj); + } + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage other = + (com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage) obj; + + if (!getDriverId().equals(other.getDriverId())) return false; + if (!getMessageCase().equals(other.getMessageCase())) return false; + switch (messageCase_) { + case 2: + if (!getPickupPassenger().equals(other.getPickupPassenger())) return false; + break; + case 0: + default: + } + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + DRIVER_ID_FIELD_NUMBER; + hash = (53 * hash) + getDriverId().hashCode(); + switch (messageCase_) { + case 2: + hash = (37 * hash) + PICKUP_PASSENGER_FIELD_NUMBER; + hash = (53 * hash) + getPickupPassenger().hashCode(); + break; + case 0: + default: + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + parseFrom(byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + parseFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + parseFrom(com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * + * + *
+   * the messages that corresponds to the TO_DRIVER_EGRESS
+   * 
+ * + * Protobuf type {@code com.ververica.statefun.examples.ridesharing.OutboundDriverMessage} + */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.ridesharing.OutboundDriverMessage) + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessageOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundDriverMessage_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundDriverMessage_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage.class, + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage.Builder + .class); + } + + // Construct using + // com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + driverId_ = ""; + + messageCase_ = 0; + message_ = null; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundDriverMessage_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + getDefaultInstanceForType() { + return com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage build() { + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage result = + buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + buildPartial() { + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage result = + new com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage(this); + result.driverId_ = driverId_; + if (messageCase_ == 2) { + if (pickupPassengerBuilder_ == null) { + result.message_ = message_; + } else { + result.message_ = pickupPassengerBuilder_.build(); + } + } + result.messageCase_ = messageCase_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other + instanceof com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage) { + return mergeFrom( + (com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage other) { + if (other + == com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .getDefaultInstance()) return this; + if (!other.getDriverId().isEmpty()) { + driverId_ = other.driverId_; + onChanged(); + } + switch (other.getMessageCase()) { + case PICKUP_PASSENGER: + { + mergePickupPassenger(other.getPickupPassenger()); + break; + } + case MESSAGE_NOT_SET: + { + break; + } + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage parsedMessage = + null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int messageCase_ = 0; + private java.lang.Object message_; + + public MessageCase getMessageCase() { + return MessageCase.forNumber(messageCase_); + } + + public Builder clearMessage() { + messageCase_ = 0; + message_ = null; + onChanged(); + return this; + } + + private java.lang.Object driverId_ = ""; + /** string driver_id = 1; */ + public java.lang.String getDriverId() { + java.lang.Object ref = driverId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + driverId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string driver_id = 1; */ + public com.google.protobuf.ByteString getDriverIdBytes() { + java.lang.Object ref = driverId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + driverId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string driver_id = 1; */ + public Builder setDriverId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + driverId_ = value; + onChanged(); + return this; + } + /** string driver_id = 1; */ + public Builder clearDriverId() { + + driverId_ = getDefaultInstance().getDriverId(); + onChanged(); + return this; + } + /** string driver_id = 1; */ + public Builder setDriverIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + driverId_ = value; + onChanged(); + return this; + } + + private com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger, + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger.Builder, + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassengerOrBuilder> + pickupPassengerBuilder_; + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundDriverMessage.PickupPassenger pickup_passenger = 2; + * + */ + public boolean hasPickupPassenger() { + return messageCase_ == 2; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundDriverMessage.PickupPassenger pickup_passenger = 2; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger + getPickupPassenger() { + if (pickupPassengerBuilder_ == null) { + if (messageCase_ == 2) { + return (com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger.getDefaultInstance(); + } else { + if (messageCase_ == 2) { + return pickupPassengerBuilder_.getMessage(); + } + return com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger.getDefaultInstance(); + } + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundDriverMessage.PickupPassenger pickup_passenger = 2; + * + */ + public Builder setPickupPassenger( + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage.PickupPassenger + value) { + if (pickupPassengerBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + message_ = value; + onChanged(); + } else { + pickupPassengerBuilder_.setMessage(value); + } + messageCase_ = 2; + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundDriverMessage.PickupPassenger pickup_passenger = 2; + * + */ + public Builder setPickupPassenger( + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage.PickupPassenger + .Builder + builderForValue) { + if (pickupPassengerBuilder_ == null) { + message_ = builderForValue.build(); + onChanged(); + } else { + pickupPassengerBuilder_.setMessage(builderForValue.build()); + } + messageCase_ = 2; + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundDriverMessage.PickupPassenger pickup_passenger = 2; + * + */ + public Builder mergePickupPassenger( + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage.PickupPassenger + value) { + if (pickupPassengerBuilder_ == null) { + if (messageCase_ == 2 + && message_ + != com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger.getDefaultInstance()) { + message_ = + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger.newBuilder( + (com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger) + message_) + .mergeFrom(value) + .buildPartial(); + } else { + message_ = value; + } + onChanged(); + } else { + if (messageCase_ == 2) { + pickupPassengerBuilder_.mergeFrom(value); + } + pickupPassengerBuilder_.setMessage(value); + } + messageCase_ = 2; + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundDriverMessage.PickupPassenger pickup_passenger = 2; + * + */ + public Builder clearPickupPassenger() { + if (pickupPassengerBuilder_ == null) { + if (messageCase_ == 2) { + messageCase_ = 0; + message_ = null; + onChanged(); + } + } else { + if (messageCase_ == 2) { + messageCase_ = 0; + message_ = null; + } + pickupPassengerBuilder_.clear(); + } + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundDriverMessage.PickupPassenger pickup_passenger = 2; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger.Builder + getPickupPassengerBuilder() { + return getPickupPassengerFieldBuilder().getBuilder(); + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundDriverMessage.PickupPassenger pickup_passenger = 2; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassengerOrBuilder + getPickupPassengerOrBuilder() { + if ((messageCase_ == 2) && (pickupPassengerBuilder_ != null)) { + return pickupPassengerBuilder_.getMessageOrBuilder(); + } else { + if (messageCase_ == 2) { + return (com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger.getDefaultInstance(); + } + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundDriverMessage.PickupPassenger pickup_passenger = 2; + * + */ + private com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger, + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger.Builder, + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassengerOrBuilder> + getPickupPassengerFieldBuilder() { + if (pickupPassengerBuilder_ == null) { + if (!(messageCase_ == 2)) { + message_ = + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger.getDefaultInstance(); + } + pickupPassengerBuilder_ = + new com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger, + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger.Builder, + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassengerOrBuilder>( + (com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassenger) + message_, + getParentForChildren(), + isClean()); + message_ = null; + } + messageCase_ = 2; + onChanged(); + ; + return pickupPassengerBuilder_; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.ridesharing.OutboundDriverMessage) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.ridesharing.OutboundDriverMessage) + private static final com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage(); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public OutboundDriverMessage parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new OutboundDriverMessage(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/OutboundDriverMessageOrBuilder.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/OutboundDriverMessageOrBuilder.java new file mode 100644 index 00000000..3fef5c20 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/OutboundDriverMessageOrBuilder.java @@ -0,0 +1,40 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +public interface OutboundDriverMessageOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.ridesharing.OutboundDriverMessage) + com.google.protobuf.MessageOrBuilder { + + /** string driver_id = 1; */ + java.lang.String getDriverId(); + /** string driver_id = 1; */ + com.google.protobuf.ByteString getDriverIdBytes(); + + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundDriverMessage.PickupPassenger pickup_passenger = 2; + * + */ + boolean hasPickupPassenger(); + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundDriverMessage.PickupPassenger pickup_passenger = 2; + * + */ + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage.PickupPassenger + getPickupPassenger(); + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundDriverMessage.PickupPassenger pickup_passenger = 2; + * + */ + com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage + .PickupPassengerOrBuilder + getPickupPassengerOrBuilder(); + + public com.ververica.statefun.examples.ridesharing.generated.OutboundDriverMessage.MessageCase + getMessageCase(); +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/OutboundPassengerMessage.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/OutboundPassengerMessage.java new file mode 100644 index 00000000..87dd48f0 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/OutboundPassengerMessage.java @@ -0,0 +1,4281 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +/** + * + * + *
+ * the messages that corresponds to the TO_PASSENGER_EGRESS
+ * 
+ * + * Protobuf type {@code com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage} + */ +public final class OutboundPassengerMessage extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage) + OutboundPassengerMessageOrBuilder { + private static final long serialVersionUID = 0L; + // Use OutboundPassengerMessage.newBuilder() to construct. + private OutboundPassengerMessage(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private OutboundPassengerMessage() { + passengerId_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private OutboundPassengerMessage( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + java.lang.String s = input.readStringRequireUtf8(); + + passengerId_ = s; + break; + } + case 18: + { + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound.Builder + subBuilder = null; + if (messageCase_ == 2) { + subBuilder = + ((com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound) + message_) + .toBuilder(); + } + message_ = + input.readMessage( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound.parser(), + extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom( + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound) + message_); + message_ = subBuilder.buildPartial(); + } + messageCase_ = 2; + break; + } + case 26: + { + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed.Builder + subBuilder = null; + if (messageCase_ == 3) { + subBuilder = + ((com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed) + message_) + .toBuilder(); + } + message_ = + input.readMessage( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed.parser(), + extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom( + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed) + message_); + message_ = subBuilder.buildPartial(); + } + messageCase_ = 3; + break; + } + case 34: + { + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted.Builder + subBuilder = null; + if (messageCase_ == 4) { + subBuilder = + ((com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted) + message_) + .toBuilder(); + } + message_ = + input.readMessage( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted.parser(), + extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom( + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted) + message_); + message_ = subBuilder.buildPartial(); + } + messageCase_ = 4; + break; + } + case 42: + { + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded.Builder + subBuilder = null; + if (messageCase_ == 5) { + subBuilder = + ((com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded) + message_) + .toBuilder(); + } + message_ = + input.readMessage( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded.parser(), + extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom( + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded) + message_); + message_ = subBuilder.buildPartial(); + } + messageCase_ = 5; + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.class, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.Builder + .class); + } + + public interface DriverHasBeenFoundOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.DriverHasBeenFound) + com.google.protobuf.MessageOrBuilder { + + /** string driver_id = 2; */ + java.lang.String getDriverId(); + /** string driver_id = 2; */ + com.google.protobuf.ByteString getDriverIdBytes(); + + /** int32 driver_geo_cell = 3; */ + int getDriverGeoCell(); + } + /** + * + * + *
+   * A message that is sent to the passenger, when there is a driver that
+   * is ready to pick them up.
+   * 
+ * + * Protobuf type {@code + * com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.DriverHasBeenFound} + */ + public static final class DriverHasBeenFound extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.DriverHasBeenFound) + DriverHasBeenFoundOrBuilder { + private static final long serialVersionUID = 0L; + // Use DriverHasBeenFound.newBuilder() to construct. + private DriverHasBeenFound(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private DriverHasBeenFound() { + driverId_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private DriverHasBeenFound( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 18: + { + java.lang.String s = input.readStringRequireUtf8(); + + driverId_ = s; + break; + } + case 24: + { + driverGeoCell_ = input.readInt32(); + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_DriverHasBeenFound_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_DriverHasBeenFound_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound.class, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound.Builder.class); + } + + public static final int DRIVER_ID_FIELD_NUMBER = 2; + private volatile java.lang.Object driverId_; + /** string driver_id = 2; */ + public java.lang.String getDriverId() { + java.lang.Object ref = driverId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + driverId_ = s; + return s; + } + } + /** string driver_id = 2; */ + public com.google.protobuf.ByteString getDriverIdBytes() { + java.lang.Object ref = driverId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + driverId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int DRIVER_GEO_CELL_FIELD_NUMBER = 3; + private int driverGeoCell_; + /** int32 driver_geo_cell = 3; */ + public int getDriverGeoCell() { + return driverGeoCell_; + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!getDriverIdBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 2, driverId_); + } + if (driverGeoCell_ != 0) { + output.writeInt32(3, driverGeoCell_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!getDriverIdBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, driverId_); + } + if (driverGeoCell_ != 0) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(3, driverGeoCell_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound)) { + return super.equals(obj); + } + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound + other = + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound) + obj; + + if (!getDriverId().equals(other.getDriverId())) return false; + if (getDriverGeoCell() != other.getDriverGeoCell()) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + DRIVER_ID_FIELD_NUMBER; + hash = (53 * hash) + getDriverId().hashCode(); + hash = (37 * hash) + DRIVER_GEO_CELL_FIELD_NUMBER; + hash = (53 * hash) + getDriverGeoCell(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound + parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound + parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound + parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound + parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound + parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound + parseFrom(byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound + parseFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound + parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound + parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound + parseFrom(com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound + parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound + prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * + * + *
+     * A message that is sent to the passenger, when there is a driver that
+     * is ready to pick them up.
+     * 
+ * + * Protobuf type {@code + * com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.DriverHasBeenFound} + */ + public static final class Builder + extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.DriverHasBeenFound) + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFoundOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_DriverHasBeenFound_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_DriverHasBeenFound_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound.class, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound.Builder.class); + } + + // Construct using + // com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.DriverHasBeenFound.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + driverId_ = ""; + + driverGeoCell_ = 0; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_DriverHasBeenFound_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound + getDefaultInstanceForType() { + return com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound.getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound + build() { + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound + result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound + buildPartial() { + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound + result = + new com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound(this); + result.driverId_ = driverId_; + result.driverGeoCell_ = driverGeoCell_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other + instanceof + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound) { + return mergeFrom( + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound) + other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound + other) { + if (other + == com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound.getDefaultInstance()) return this; + if (!other.getDriverId().isEmpty()) { + driverId_ = other.driverId_; + onChanged(); + } + if (other.getDriverGeoCell() != 0) { + setDriverGeoCell(other.getDriverGeoCell()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound + parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private java.lang.Object driverId_ = ""; + /** string driver_id = 2; */ + public java.lang.String getDriverId() { + java.lang.Object ref = driverId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + driverId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string driver_id = 2; */ + public com.google.protobuf.ByteString getDriverIdBytes() { + java.lang.Object ref = driverId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + driverId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string driver_id = 2; */ + public Builder setDriverId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + driverId_ = value; + onChanged(); + return this; + } + /** string driver_id = 2; */ + public Builder clearDriverId() { + + driverId_ = getDefaultInstance().getDriverId(); + onChanged(); + return this; + } + /** string driver_id = 2; */ + public Builder setDriverIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + driverId_ = value; + onChanged(); + return this; + } + + private int driverGeoCell_; + /** int32 driver_geo_cell = 3; */ + public int getDriverGeoCell() { + return driverGeoCell_; + } + /** int32 driver_geo_cell = 3; */ + public Builder setDriverGeoCell(int value) { + + driverGeoCell_ = value; + onChanged(); + return this; + } + /** int32 driver_geo_cell = 3; */ + public Builder clearDriverGeoCell() { + + driverGeoCell_ = 0; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.DriverHasBeenFound) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.DriverHasBeenFound) + private static final com.ververica.statefun.examples.ridesharing.generated + .OutboundPassengerMessage.DriverHasBeenFound + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound(); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public DriverHasBeenFound parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new DriverHasBeenFound(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + public interface RideFailedOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideFailed) + com.google.protobuf.MessageOrBuilder { + + /** string ride_id = 2; */ + java.lang.String getRideId(); + /** string ride_id = 2; */ + com.google.protobuf.ByteString getRideIdBytes(); + } + /** + * + * + *
+   * A message that is sent to the passenger when the system couldn't find in reasonable time
+   * a driver to pickup the passenger.
+   * 
+ * + * Protobuf type {@code + * com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideFailed} + */ + public static final class RideFailed extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideFailed) + RideFailedOrBuilder { + private static final long serialVersionUID = 0L; + // Use RideFailed.newBuilder() to construct. + private RideFailed(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private RideFailed() { + rideId_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private RideFailed( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 18: + { + java.lang.String s = input.readStringRequireUtf8(); + + rideId_ = s; + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_RideFailed_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_RideFailed_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed.class, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed.Builder.class); + } + + public static final int RIDE_ID_FIELD_NUMBER = 2; + private volatile java.lang.Object rideId_; + /** string ride_id = 2; */ + public java.lang.String getRideId() { + java.lang.Object ref = rideId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + rideId_ = s; + return s; + } + } + /** string ride_id = 2; */ + public com.google.protobuf.ByteString getRideIdBytes() { + java.lang.Object ref = rideId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + rideId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!getRideIdBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 2, rideId_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!getRideIdBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, rideId_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed)) { + return super.equals(obj); + } + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideFailed + other = + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed) + obj; + + if (!getRideId().equals(other.getRideId())) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + RIDE_ID_FIELD_NUMBER; + hash = (53 * hash) + getRideId().hashCode(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed + parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed + parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed + parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed + parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed + parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed + parseFrom(byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed + parseFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed + parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed + parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed + parseFrom(com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed + parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideFailed + prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * + * + *
+     * A message that is sent to the passenger when the system couldn't find in reasonable time
+     * a driver to pickup the passenger.
+     * 
+ * + * Protobuf type {@code + * com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideFailed} + */ + public static final class Builder + extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideFailed) + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailedOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_RideFailed_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_RideFailed_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed.class, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed.Builder.class); + } + + // Construct using + // com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideFailed.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + rideId_ = ""; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_RideFailed_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed + getDefaultInstanceForType() { + return com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed.getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed + build() { + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideFailed + result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed + buildPartial() { + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideFailed + result = + new com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed(this); + result.rideId_ = rideId_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other + instanceof + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed) { + return mergeFrom( + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed) + other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideFailed + other) { + if (other + == com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed.getDefaultInstance()) return this; + if (!other.getRideId().isEmpty()) { + rideId_ = other.rideId_; + onChanged(); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideFailed + parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private java.lang.Object rideId_ = ""; + /** string ride_id = 2; */ + public java.lang.String getRideId() { + java.lang.Object ref = rideId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + rideId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string ride_id = 2; */ + public com.google.protobuf.ByteString getRideIdBytes() { + java.lang.Object ref = rideId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + rideId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string ride_id = 2; */ + public Builder setRideId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + rideId_ = value; + onChanged(); + return this; + } + /** string ride_id = 2; */ + public Builder clearRideId() { + + rideId_ = getDefaultInstance().getRideId(); + onChanged(); + return this; + } + /** string ride_id = 2; */ + public Builder setRideIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + rideId_ = value; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideFailed) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideFailed) + private static final com.ververica.statefun.examples.ridesharing.generated + .OutboundPassengerMessage.RideFailed + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed(); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public RideFailed parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new RideFailed(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideFailed + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + public interface RideStartedOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideStarted) + com.google.protobuf.MessageOrBuilder { + + /** string driver_id = 2; */ + java.lang.String getDriverId(); + /** string driver_id = 2; */ + com.google.protobuf.ByteString getDriverIdBytes(); + } + /** + * + * + *
+   * A notification that is sent to the passenger indicating that the ride has started.
+   * 
+ * + * Protobuf type {@code + * com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideStarted} + */ + public static final class RideStarted extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideStarted) + RideStartedOrBuilder { + private static final long serialVersionUID = 0L; + // Use RideStarted.newBuilder() to construct. + private RideStarted(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private RideStarted() { + driverId_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private RideStarted( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 18: + { + java.lang.String s = input.readStringRequireUtf8(); + + driverId_ = s; + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_RideStarted_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_RideStarted_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted.class, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted.Builder.class); + } + + public static final int DRIVER_ID_FIELD_NUMBER = 2; + private volatile java.lang.Object driverId_; + /** string driver_id = 2; */ + public java.lang.String getDriverId() { + java.lang.Object ref = driverId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + driverId_ = s; + return s; + } + } + /** string driver_id = 2; */ + public com.google.protobuf.ByteString getDriverIdBytes() { + java.lang.Object ref = driverId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + driverId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!getDriverIdBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 2, driverId_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!getDriverIdBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, driverId_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted)) { + return super.equals(obj); + } + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideStarted + other = + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted) + obj; + + if (!getDriverId().equals(other.getDriverId())) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + DRIVER_ID_FIELD_NUMBER; + hash = (53 * hash) + getDriverId().hashCode(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted + parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted + parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted + parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted + parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted + parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted + parseFrom(byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted + parseFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted + parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted + parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted + parseFrom(com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted + parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideStarted + prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * + * + *
+     * A notification that is sent to the passenger indicating that the ride has started.
+     * 
+ * + * Protobuf type {@code + * com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideStarted} + */ + public static final class Builder + extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideStarted) + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStartedOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_RideStarted_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_RideStarted_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted.class, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted.Builder.class); + } + + // Construct using + // com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideStarted.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + driverId_ = ""; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_RideStarted_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted + getDefaultInstanceForType() { + return com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted.getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted + build() { + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideStarted + result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted + buildPartial() { + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideStarted + result = + new com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted(this); + result.driverId_ = driverId_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other + instanceof + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted) { + return mergeFrom( + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted) + other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideStarted + other) { + if (other + == com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted.getDefaultInstance()) return this; + if (!other.getDriverId().isEmpty()) { + driverId_ = other.driverId_; + onChanged(); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideStarted + parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private java.lang.Object driverId_ = ""; + /** string driver_id = 2; */ + public java.lang.String getDriverId() { + java.lang.Object ref = driverId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + driverId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string driver_id = 2; */ + public com.google.protobuf.ByteString getDriverIdBytes() { + java.lang.Object ref = driverId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + driverId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string driver_id = 2; */ + public Builder setDriverId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + driverId_ = value; + onChanged(); + return this; + } + /** string driver_id = 2; */ + public Builder clearDriverId() { + + driverId_ = getDefaultInstance().getDriverId(); + onChanged(); + return this; + } + /** string driver_id = 2; */ + public Builder setDriverIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + driverId_ = value; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideStarted) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideStarted) + private static final com.ververica.statefun.examples.ridesharing.generated + .OutboundPassengerMessage.RideStarted + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted(); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public RideStarted parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new RideStarted(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + public interface RideEndedOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideEnded) + com.google.protobuf.MessageOrBuilder {} + /** + * + * + *
+   * A notification that is sent to the passenger indicating that the ride has end.
+   * 
+ * + * Protobuf type {@code + * com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideEnded} + */ + public static final class RideEnded extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideEnded) + RideEndedOrBuilder { + private static final long serialVersionUID = 0L; + // Use RideEnded.newBuilder() to construct. + private RideEnded(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private RideEnded() {} + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private RideEnded( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_RideEnded_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_RideEnded_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded.class, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded.Builder.class); + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded)) { + return super.equals(obj); + } + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideEnded + other = + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded) + obj; + + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded + parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded + parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded + parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded + parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded + parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded + parseFrom(byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded + parseFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded + parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded + parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded + parseFrom(com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded + parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideEnded + prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * + * + *
+     * A notification that is sent to the passenger indicating that the ride has end.
+     * 
+ * + * Protobuf type {@code + * com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideEnded} + */ + public static final class Builder + extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideEnded) + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEndedOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_RideEnded_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_RideEnded_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded.class, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded.Builder.class); + } + + // Construct using + // com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideEnded.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_RideEnded_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded + getDefaultInstanceForType() { + return com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded.getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded + build() { + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideEnded + result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded + buildPartial() { + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideEnded + result = + new com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded(this); + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other + instanceof + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded) { + return mergeFrom( + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded) + other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideEnded + other) { + if (other + == com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded.getDefaultInstance()) return this; + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideEnded + parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideEnded) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideEnded) + private static final com.ververica.statefun.examples.ridesharing.generated + .OutboundPassengerMessage.RideEnded + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded(); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public RideEnded parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new RideEnded(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideEnded + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + private int messageCase_ = 0; + private java.lang.Object message_; + + public enum MessageCase implements com.google.protobuf.Internal.EnumLite { + DRIVER_FOUND(2), + RIDE_FAILED(3), + RIDE_STARTED(4), + RIDE_ENDED(5), + MESSAGE_NOT_SET(0); + private final int value; + + private MessageCase(int value) { + this.value = value; + } + /** @deprecated Use {@link #forNumber(int)} instead. */ + @java.lang.Deprecated + public static MessageCase valueOf(int value) { + return forNumber(value); + } + + public static MessageCase forNumber(int value) { + switch (value) { + case 2: + return DRIVER_FOUND; + case 3: + return RIDE_FAILED; + case 4: + return RIDE_STARTED; + case 5: + return RIDE_ENDED; + case 0: + return MESSAGE_NOT_SET; + default: + return null; + } + } + + public int getNumber() { + return this.value; + } + }; + + public MessageCase getMessageCase() { + return MessageCase.forNumber(messageCase_); + } + + public static final int PASSENGER_ID_FIELD_NUMBER = 1; + private volatile java.lang.Object passengerId_; + /** string passenger_id = 1; */ + public java.lang.String getPassengerId() { + java.lang.Object ref = passengerId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + passengerId_ = s; + return s; + } + } + /** string passenger_id = 1; */ + public com.google.protobuf.ByteString getPassengerIdBytes() { + java.lang.Object ref = passengerId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + passengerId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int DRIVER_FOUND_FIELD_NUMBER = 2; + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.DriverHasBeenFound driver_found = 2; + * + */ + public boolean hasDriverFound() { + return messageCase_ == 2; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.DriverHasBeenFound driver_found = 2; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound + getDriverFound() { + if (messageCase_ == 2) { + return (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound.getDefaultInstance(); + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.DriverHasBeenFound driver_found = 2; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFoundOrBuilder + getDriverFoundOrBuilder() { + if (messageCase_ == 2) { + return (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound.getDefaultInstance(); + } + + public static final int RIDE_FAILED_FIELD_NUMBER = 3; + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideFailed ride_failed = 3; + * + */ + public boolean hasRideFailed() { + return messageCase_ == 3; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideFailed ride_failed = 3; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideFailed + getRideFailed() { + if (messageCase_ == 3) { + return (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideFailed + .getDefaultInstance(); + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideFailed ride_failed = 3; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailedOrBuilder + getRideFailedOrBuilder() { + if (messageCase_ == 3) { + return (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideFailed + .getDefaultInstance(); + } + + public static final int RIDE_STARTED_FIELD_NUMBER = 4; + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideStarted ride_started = 4; + * + */ + public boolean hasRideStarted() { + return messageCase_ == 4; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideStarted ride_started = 4; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideStarted + getRideStarted() { + if (messageCase_ == 4) { + return (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted.getDefaultInstance(); + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideStarted ride_started = 4; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStartedOrBuilder + getRideStartedOrBuilder() { + if (messageCase_ == 4) { + return (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted.getDefaultInstance(); + } + + public static final int RIDE_ENDED_FIELD_NUMBER = 5; + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideEnded ride_ended = 5; + * + */ + public boolean hasRideEnded() { + return messageCase_ == 5; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideEnded ride_ended = 5; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideEnded + getRideEnded() { + if (messageCase_ == 5) { + return (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideEnded + .getDefaultInstance(); + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideEnded ride_ended = 5; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEndedOrBuilder + getRideEndedOrBuilder() { + if (messageCase_ == 5) { + return (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideEnded + .getDefaultInstance(); + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!getPassengerIdBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, passengerId_); + } + if (messageCase_ == 2) { + output.writeMessage( + 2, + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound) + message_); + } + if (messageCase_ == 3) { + output.writeMessage( + 3, + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed) + message_); + } + if (messageCase_ == 4) { + output.writeMessage( + 4, + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted) + message_); + } + if (messageCase_ == 5) { + output.writeMessage( + 5, + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideEnded) + message_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!getPassengerIdBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, passengerId_); + } + if (messageCase_ == 2) { + size += + com.google.protobuf.CodedOutputStream.computeMessageSize( + 2, + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound) + message_); + } + if (messageCase_ == 3) { + size += + com.google.protobuf.CodedOutputStream.computeMessageSize( + 3, + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed) + message_); + } + if (messageCase_ == 4) { + size += + com.google.protobuf.CodedOutputStream.computeMessageSize( + 4, + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted) + message_); + } + if (messageCase_ == 5) { + size += + com.google.protobuf.CodedOutputStream.computeMessageSize( + 5, + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded) + message_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage)) { + return super.equals(obj); + } + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage other = + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage) obj; + + if (!getPassengerId().equals(other.getPassengerId())) return false; + if (!getMessageCase().equals(other.getMessageCase())) return false; + switch (messageCase_) { + case 2: + if (!getDriverFound().equals(other.getDriverFound())) return false; + break; + case 3: + if (!getRideFailed().equals(other.getRideFailed())) return false; + break; + case 4: + if (!getRideStarted().equals(other.getRideStarted())) return false; + break; + case 5: + if (!getRideEnded().equals(other.getRideEnded())) return false; + break; + case 0: + default: + } + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + PASSENGER_ID_FIELD_NUMBER; + hash = (53 * hash) + getPassengerId().hashCode(); + switch (messageCase_) { + case 2: + hash = (37 * hash) + DRIVER_FOUND_FIELD_NUMBER; + hash = (53 * hash) + getDriverFound().hashCode(); + break; + case 3: + hash = (37 * hash) + RIDE_FAILED_FIELD_NUMBER; + hash = (53 * hash) + getRideFailed().hashCode(); + break; + case 4: + hash = (37 * hash) + RIDE_STARTED_FIELD_NUMBER; + hash = (53 * hash) + getRideStarted().hashCode(); + break; + case 5: + hash = (37 * hash) + RIDE_ENDED_FIELD_NUMBER; + hash = (53 * hash) + getRideEnded().hashCode(); + break; + case 0: + default: + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + parseFrom(byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + parseFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + parseFrom(com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * + * + *
+   * the messages that corresponds to the TO_PASSENGER_EGRESS
+   * 
+ * + * Protobuf type {@code com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage} + */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage) + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessageOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.class, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.Builder + .class); + } + + // Construct using + // com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + passengerId_ = ""; + + messageCase_ = 0; + message_ = null; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + getDefaultInstanceForType() { + return com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage build() { + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage result = + buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + buildPartial() { + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage result = + new com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage(this); + result.passengerId_ = passengerId_; + if (messageCase_ == 2) { + if (driverFoundBuilder_ == null) { + result.message_ = message_; + } else { + result.message_ = driverFoundBuilder_.build(); + } + } + if (messageCase_ == 3) { + if (rideFailedBuilder_ == null) { + result.message_ = message_; + } else { + result.message_ = rideFailedBuilder_.build(); + } + } + if (messageCase_ == 4) { + if (rideStartedBuilder_ == null) { + result.message_ = message_; + } else { + result.message_ = rideStartedBuilder_.build(); + } + } + if (messageCase_ == 5) { + if (rideEndedBuilder_ == null) { + result.message_ = message_; + } else { + result.message_ = rideEndedBuilder_.build(); + } + } + result.messageCase_ = messageCase_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other + instanceof + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage) { + return mergeFrom( + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage other) { + if (other + == com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .getDefaultInstance()) return this; + if (!other.getPassengerId().isEmpty()) { + passengerId_ = other.passengerId_; + onChanged(); + } + switch (other.getMessageCase()) { + case DRIVER_FOUND: + { + mergeDriverFound(other.getDriverFound()); + break; + } + case RIDE_FAILED: + { + mergeRideFailed(other.getRideFailed()); + break; + } + case RIDE_STARTED: + { + mergeRideStarted(other.getRideStarted()); + break; + } + case RIDE_ENDED: + { + mergeRideEnded(other.getRideEnded()); + break; + } + case MESSAGE_NOT_SET: + { + break; + } + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage parsedMessage = + null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int messageCase_ = 0; + private java.lang.Object message_; + + public MessageCase getMessageCase() { + return MessageCase.forNumber(messageCase_); + } + + public Builder clearMessage() { + messageCase_ = 0; + message_ = null; + onChanged(); + return this; + } + + private java.lang.Object passengerId_ = ""; + /** string passenger_id = 1; */ + public java.lang.String getPassengerId() { + java.lang.Object ref = passengerId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + passengerId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string passenger_id = 1; */ + public com.google.protobuf.ByteString getPassengerIdBytes() { + java.lang.Object ref = passengerId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + passengerId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string passenger_id = 1; */ + public Builder setPassengerId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + passengerId_ = value; + onChanged(); + return this; + } + /** string passenger_id = 1; */ + public Builder clearPassengerId() { + + passengerId_ = getDefaultInstance().getPassengerId(); + onChanged(); + return this; + } + /** string passenger_id = 1; */ + public Builder setPassengerIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + passengerId_ = value; + onChanged(); + return this; + } + + private com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound.Builder, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFoundOrBuilder> + driverFoundBuilder_; + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.DriverHasBeenFound driver_found = 2; + * + */ + public boolean hasDriverFound() { + return messageCase_ == 2; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.DriverHasBeenFound driver_found = 2; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound + getDriverFound() { + if (driverFoundBuilder_ == null) { + if (messageCase_ == 2) { + return (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound.getDefaultInstance(); + } else { + if (messageCase_ == 2) { + return driverFoundBuilder_.getMessage(); + } + return com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound.getDefaultInstance(); + } + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.DriverHasBeenFound driver_found = 2; + * + */ + public Builder setDriverFound( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound + value) { + if (driverFoundBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + message_ = value; + onChanged(); + } else { + driverFoundBuilder_.setMessage(value); + } + messageCase_ = 2; + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.DriverHasBeenFound driver_found = 2; + * + */ + public Builder setDriverFound( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound.Builder + builderForValue) { + if (driverFoundBuilder_ == null) { + message_ = builderForValue.build(); + onChanged(); + } else { + driverFoundBuilder_.setMessage(builderForValue.build()); + } + messageCase_ = 2; + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.DriverHasBeenFound driver_found = 2; + * + */ + public Builder mergeDriverFound( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound + value) { + if (driverFoundBuilder_ == null) { + if (messageCase_ == 2 + && message_ + != com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound.getDefaultInstance()) { + message_ = + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound.newBuilder( + (com.ververica.statefun.examples.ridesharing.generated + .OutboundPassengerMessage.DriverHasBeenFound) + message_) + .mergeFrom(value) + .buildPartial(); + } else { + message_ = value; + } + onChanged(); + } else { + if (messageCase_ == 2) { + driverFoundBuilder_.mergeFrom(value); + } + driverFoundBuilder_.setMessage(value); + } + messageCase_ = 2; + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.DriverHasBeenFound driver_found = 2; + * + */ + public Builder clearDriverFound() { + if (driverFoundBuilder_ == null) { + if (messageCase_ == 2) { + messageCase_ = 0; + message_ = null; + onChanged(); + } + } else { + if (messageCase_ == 2) { + messageCase_ = 0; + message_ = null; + } + driverFoundBuilder_.clear(); + } + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.DriverHasBeenFound driver_found = 2; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound.Builder + getDriverFoundBuilder() { + return getDriverFoundFieldBuilder().getBuilder(); + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.DriverHasBeenFound driver_found = 2; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFoundOrBuilder + getDriverFoundOrBuilder() { + if ((messageCase_ == 2) && (driverFoundBuilder_ != null)) { + return driverFoundBuilder_.getMessageOrBuilder(); + } else { + if (messageCase_ == 2) { + return (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound.getDefaultInstance(); + } + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.DriverHasBeenFound driver_found = 2; + * + */ + private com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound.Builder, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFoundOrBuilder> + getDriverFoundFieldBuilder() { + if (driverFoundBuilder_ == null) { + if (!(messageCase_ == 2)) { + message_ = + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound.getDefaultInstance(); + } + driverFoundBuilder_ = + new com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound.Builder, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFoundOrBuilder>( + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFound) + message_, + getParentForChildren(), + isClean()); + message_ = null; + } + messageCase_ = 2; + onChanged(); + ; + return driverFoundBuilder_; + } + + private com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed.Builder, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailedOrBuilder> + rideFailedBuilder_; + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideFailed ride_failed = 3; + * + */ + public boolean hasRideFailed() { + return messageCase_ == 3; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideFailed ride_failed = 3; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideFailed + getRideFailed() { + if (rideFailedBuilder_ == null) { + if (messageCase_ == 3) { + return (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed.getDefaultInstance(); + } else { + if (messageCase_ == 3) { + return rideFailedBuilder_.getMessage(); + } + return com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed.getDefaultInstance(); + } + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideFailed ride_failed = 3; + * + */ + public Builder setRideFailed( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideFailed + value) { + if (rideFailedBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + message_ = value; + onChanged(); + } else { + rideFailedBuilder_.setMessage(value); + } + messageCase_ = 3; + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideFailed ride_failed = 3; + * + */ + public Builder setRideFailed( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideFailed + .Builder + builderForValue) { + if (rideFailedBuilder_ == null) { + message_ = builderForValue.build(); + onChanged(); + } else { + rideFailedBuilder_.setMessage(builderForValue.build()); + } + messageCase_ = 3; + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideFailed ride_failed = 3; + * + */ + public Builder mergeRideFailed( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideFailed + value) { + if (rideFailedBuilder_ == null) { + if (messageCase_ == 3 + && message_ + != com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed.getDefaultInstance()) { + message_ = + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed.newBuilder( + (com.ververica.statefun.examples.ridesharing.generated + .OutboundPassengerMessage.RideFailed) + message_) + .mergeFrom(value) + .buildPartial(); + } else { + message_ = value; + } + onChanged(); + } else { + if (messageCase_ == 3) { + rideFailedBuilder_.mergeFrom(value); + } + rideFailedBuilder_.setMessage(value); + } + messageCase_ = 3; + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideFailed ride_failed = 3; + * + */ + public Builder clearRideFailed() { + if (rideFailedBuilder_ == null) { + if (messageCase_ == 3) { + messageCase_ = 0; + message_ = null; + onChanged(); + } + } else { + if (messageCase_ == 3) { + messageCase_ = 0; + message_ = null; + } + rideFailedBuilder_.clear(); + } + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideFailed ride_failed = 3; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideFailed + .Builder + getRideFailedBuilder() { + return getRideFailedFieldBuilder().getBuilder(); + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideFailed ride_failed = 3; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailedOrBuilder + getRideFailedOrBuilder() { + if ((messageCase_ == 3) && (rideFailedBuilder_ != null)) { + return rideFailedBuilder_.getMessageOrBuilder(); + } else { + if (messageCase_ == 3) { + return (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed.getDefaultInstance(); + } + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideFailed ride_failed = 3; + * + */ + private com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed.Builder, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailedOrBuilder> + getRideFailedFieldBuilder() { + if (rideFailedBuilder_ == null) { + if (!(messageCase_ == 3)) { + message_ = + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed.getDefaultInstance(); + } + rideFailedBuilder_ = + new com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed.Builder, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailedOrBuilder>( + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideFailed) + message_, + getParentForChildren(), + isClean()); + message_ = null; + } + messageCase_ = 3; + onChanged(); + ; + return rideFailedBuilder_; + } + + private com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted.Builder, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStartedOrBuilder> + rideStartedBuilder_; + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideStarted ride_started = 4; + * + */ + public boolean hasRideStarted() { + return messageCase_ == 4; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideStarted ride_started = 4; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted + getRideStarted() { + if (rideStartedBuilder_ == null) { + if (messageCase_ == 4) { + return (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted.getDefaultInstance(); + } else { + if (messageCase_ == 4) { + return rideStartedBuilder_.getMessage(); + } + return com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted.getDefaultInstance(); + } + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideStarted ride_started = 4; + * + */ + public Builder setRideStarted( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideStarted + value) { + if (rideStartedBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + message_ = value; + onChanged(); + } else { + rideStartedBuilder_.setMessage(value); + } + messageCase_ = 4; + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideStarted ride_started = 4; + * + */ + public Builder setRideStarted( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideStarted + .Builder + builderForValue) { + if (rideStartedBuilder_ == null) { + message_ = builderForValue.build(); + onChanged(); + } else { + rideStartedBuilder_.setMessage(builderForValue.build()); + } + messageCase_ = 4; + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideStarted ride_started = 4; + * + */ + public Builder mergeRideStarted( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideStarted + value) { + if (rideStartedBuilder_ == null) { + if (messageCase_ == 4 + && message_ + != com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted.getDefaultInstance()) { + message_ = + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted.newBuilder( + (com.ververica.statefun.examples.ridesharing.generated + .OutboundPassengerMessage.RideStarted) + message_) + .mergeFrom(value) + .buildPartial(); + } else { + message_ = value; + } + onChanged(); + } else { + if (messageCase_ == 4) { + rideStartedBuilder_.mergeFrom(value); + } + rideStartedBuilder_.setMessage(value); + } + messageCase_ = 4; + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideStarted ride_started = 4; + * + */ + public Builder clearRideStarted() { + if (rideStartedBuilder_ == null) { + if (messageCase_ == 4) { + messageCase_ = 0; + message_ = null; + onChanged(); + } + } else { + if (messageCase_ == 4) { + messageCase_ = 0; + message_ = null; + } + rideStartedBuilder_.clear(); + } + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideStarted ride_started = 4; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted.Builder + getRideStartedBuilder() { + return getRideStartedFieldBuilder().getBuilder(); + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideStarted ride_started = 4; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStartedOrBuilder + getRideStartedOrBuilder() { + if ((messageCase_ == 4) && (rideStartedBuilder_ != null)) { + return rideStartedBuilder_.getMessageOrBuilder(); + } else { + if (messageCase_ == 4) { + return (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted.getDefaultInstance(); + } + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideStarted ride_started = 4; + * + */ + private com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted.Builder, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStartedOrBuilder> + getRideStartedFieldBuilder() { + if (rideStartedBuilder_ == null) { + if (!(messageCase_ == 4)) { + message_ = + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted.getDefaultInstance(); + } + rideStartedBuilder_ = + new com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted.Builder, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStartedOrBuilder>( + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStarted) + message_, + getParentForChildren(), + isClean()); + message_ = null; + } + messageCase_ = 4; + onChanged(); + ; + return rideStartedBuilder_; + } + + private com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideEnded + .Builder, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEndedOrBuilder> + rideEndedBuilder_; + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideEnded ride_ended = 5; + * + */ + public boolean hasRideEnded() { + return messageCase_ == 5; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideEnded ride_ended = 5; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideEnded + getRideEnded() { + if (rideEndedBuilder_ == null) { + if (messageCase_ == 5) { + return (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded.getDefaultInstance(); + } else { + if (messageCase_ == 5) { + return rideEndedBuilder_.getMessage(); + } + return com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded.getDefaultInstance(); + } + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideEnded ride_ended = 5; + * + */ + public Builder setRideEnded( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideEnded + value) { + if (rideEndedBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + message_ = value; + onChanged(); + } else { + rideEndedBuilder_.setMessage(value); + } + messageCase_ = 5; + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideEnded ride_ended = 5; + * + */ + public Builder setRideEnded( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideEnded + .Builder + builderForValue) { + if (rideEndedBuilder_ == null) { + message_ = builderForValue.build(); + onChanged(); + } else { + rideEndedBuilder_.setMessage(builderForValue.build()); + } + messageCase_ = 5; + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideEnded ride_ended = 5; + * + */ + public Builder mergeRideEnded( + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideEnded + value) { + if (rideEndedBuilder_ == null) { + if (messageCase_ == 5 + && message_ + != com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded.getDefaultInstance()) { + message_ = + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded.newBuilder( + (com.ververica.statefun.examples.ridesharing.generated + .OutboundPassengerMessage.RideEnded) + message_) + .mergeFrom(value) + .buildPartial(); + } else { + message_ = value; + } + onChanged(); + } else { + if (messageCase_ == 5) { + rideEndedBuilder_.mergeFrom(value); + } + rideEndedBuilder_.setMessage(value); + } + messageCase_ = 5; + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideEnded ride_ended = 5; + * + */ + public Builder clearRideEnded() { + if (rideEndedBuilder_ == null) { + if (messageCase_ == 5) { + messageCase_ = 0; + message_ = null; + onChanged(); + } + } else { + if (messageCase_ == 5) { + messageCase_ = 0; + message_ = null; + } + rideEndedBuilder_.clear(); + } + return this; + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideEnded ride_ended = 5; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideEnded + .Builder + getRideEndedBuilder() { + return getRideEndedFieldBuilder().getBuilder(); + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideEnded ride_ended = 5; + * + */ + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEndedOrBuilder + getRideEndedOrBuilder() { + if ((messageCase_ == 5) && (rideEndedBuilder_ != null)) { + return rideEndedBuilder_.getMessageOrBuilder(); + } else { + if (messageCase_ == 5) { + return (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded) + message_; + } + return com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded.getDefaultInstance(); + } + } + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideEnded ride_ended = 5; + * + */ + private com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideEnded + .Builder, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEndedOrBuilder> + getRideEndedFieldBuilder() { + if (rideEndedBuilder_ == null) { + if (!(messageCase_ == 5)) { + message_ = + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded.getDefaultInstance(); + } + rideEndedBuilder_ = + new com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded.Builder, + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEndedOrBuilder>( + (com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideEnded) + message_, + getParentForChildren(), + isClean()); + message_ = null; + } + messageCase_ = 5; + onChanged(); + ; + return rideEndedBuilder_; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage) + private static final com.ververica.statefun.examples.ridesharing.generated + .OutboundPassengerMessage + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage(); + } + + public static com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public OutboundPassengerMessage parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new OutboundPassengerMessage(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/OutboundPassengerMessageOrBuilder.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/OutboundPassengerMessageOrBuilder.java new file mode 100644 index 00000000..5f8160bf --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/OutboundPassengerMessageOrBuilder.java @@ -0,0 +1,104 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +public interface OutboundPassengerMessageOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage) + com.google.protobuf.MessageOrBuilder { + + /** string passenger_id = 1; */ + java.lang.String getPassengerId(); + /** string passenger_id = 1; */ + com.google.protobuf.ByteString getPassengerIdBytes(); + + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.DriverHasBeenFound driver_found = 2; + * + */ + boolean hasDriverFound(); + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.DriverHasBeenFound driver_found = 2; + * + */ + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.DriverHasBeenFound + getDriverFound(); + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.DriverHasBeenFound driver_found = 2; + * + */ + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .DriverHasBeenFoundOrBuilder + getDriverFoundOrBuilder(); + + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideFailed ride_failed = 3; + * + */ + boolean hasRideFailed(); + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideFailed ride_failed = 3; + * + */ + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideFailed + getRideFailed(); + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideFailed ride_failed = 3; + * + */ + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideFailedOrBuilder + getRideFailedOrBuilder(); + + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideStarted ride_started = 4; + * + */ + boolean hasRideStarted(); + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideStarted ride_started = 4; + * + */ + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideStarted + getRideStarted(); + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideStarted ride_started = 4; + * + */ + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage + .RideStartedOrBuilder + getRideStartedOrBuilder(); + + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideEnded ride_ended = 5; + * + */ + boolean hasRideEnded(); + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideEnded ride_ended = 5; + * + */ + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideEnded + getRideEnded(); + /** + * + * .com.ververica.statefun.examples.ridesharing.OutboundPassengerMessage.RideEnded ride_ended = 5; + * + */ + com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.RideEndedOrBuilder + getRideEndedOrBuilder(); + + public com.ververica.statefun.examples.ridesharing.generated.OutboundPassengerMessage.MessageCase + getMessageCase(); +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/PassengerJoinsRide.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/PassengerJoinsRide.java new file mode 100644 index 00000000..65d80ebd --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/PassengerJoinsRide.java @@ -0,0 +1,637 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +/** Protobuf type {@code com.ververica.statefun.examples.ridesharing.PassengerJoinsRide} */ +public final class PassengerJoinsRide extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.ridesharing.PassengerJoinsRide) + PassengerJoinsRideOrBuilder { + private static final long serialVersionUID = 0L; + // Use PassengerJoinsRide.newBuilder() to construct. + private PassengerJoinsRide(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private PassengerJoinsRide() { + passengerId_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private PassengerJoinsRide( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + java.lang.String s = input.readStringRequireUtf8(); + + passengerId_ = s; + break; + } + case 16: + { + startGeoCell_ = input.readInt32(); + break; + } + case 24: + { + endGeoCell_ = input.readInt32(); + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_PassengerJoinsRide_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_PassengerJoinsRide_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide.class, + com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide.Builder.class); + } + + public static final int PASSENGER_ID_FIELD_NUMBER = 1; + private volatile java.lang.Object passengerId_; + /** string passenger_id = 1; */ + public java.lang.String getPassengerId() { + java.lang.Object ref = passengerId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + passengerId_ = s; + return s; + } + } + /** string passenger_id = 1; */ + public com.google.protobuf.ByteString getPassengerIdBytes() { + java.lang.Object ref = passengerId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + passengerId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int START_GEO_CELL_FIELD_NUMBER = 2; + private int startGeoCell_; + /** int32 start_geo_cell = 2; */ + public int getStartGeoCell() { + return startGeoCell_; + } + + public static final int END_GEO_CELL_FIELD_NUMBER = 3; + private int endGeoCell_; + /** int32 end_geo_cell = 3; */ + public int getEndGeoCell() { + return endGeoCell_; + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!getPassengerIdBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, passengerId_); + } + if (startGeoCell_ != 0) { + output.writeInt32(2, startGeoCell_); + } + if (endGeoCell_ != 0) { + output.writeInt32(3, endGeoCell_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!getPassengerIdBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, passengerId_); + } + if (startGeoCell_ != 0) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(2, startGeoCell_); + } + if (endGeoCell_ != 0) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(3, endGeoCell_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide)) { + return super.equals(obj); + } + com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide other = + (com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide) obj; + + if (!getPassengerId().equals(other.getPassengerId())) return false; + if (getStartGeoCell() != other.getStartGeoCell()) return false; + if (getEndGeoCell() != other.getEndGeoCell()) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + PASSENGER_ID_FIELD_NUMBER; + hash = (53 * hash) + getPassengerId().hashCode(); + hash = (37 * hash) + START_GEO_CELL_FIELD_NUMBER; + hash = (53 * hash) + getStartGeoCell(); + hash = (37 * hash) + END_GEO_CELL_FIELD_NUMBER; + hash = (53 * hash) + getEndGeoCell(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide parseFrom( + java.nio.ByteBuffer data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide parseFrom( + byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide parseFrom( + byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide parseFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide + parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide parseFrom( + com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code com.ververica.statefun.examples.ridesharing.PassengerJoinsRide} */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.ridesharing.PassengerJoinsRide) + com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRideOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_PassengerJoinsRide_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_PassengerJoinsRide_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide.class, + com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide.Builder + .class); + } + + // Construct using + // com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + passengerId_ = ""; + + startGeoCell_ = 0; + + endGeoCell_ = 0; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_PassengerJoinsRide_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide + getDefaultInstanceForType() { + return com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide + .getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide build() { + com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide result = + buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide buildPartial() { + com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide result = + new com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide(this); + result.passengerId_ = passengerId_; + result.startGeoCell_ = startGeoCell_; + result.endGeoCell_ = endGeoCell_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other + instanceof com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide) { + return mergeFrom( + (com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide other) { + if (other + == com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide + .getDefaultInstance()) return this; + if (!other.getPassengerId().isEmpty()) { + passengerId_ = other.passengerId_; + onChanged(); + } + if (other.getStartGeoCell() != 0) { + setStartGeoCell(other.getStartGeoCell()); + } + if (other.getEndGeoCell() != 0) { + setEndGeoCell(other.getEndGeoCell()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private java.lang.Object passengerId_ = ""; + /** string passenger_id = 1; */ + public java.lang.String getPassengerId() { + java.lang.Object ref = passengerId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + passengerId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string passenger_id = 1; */ + public com.google.protobuf.ByteString getPassengerIdBytes() { + java.lang.Object ref = passengerId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + passengerId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string passenger_id = 1; */ + public Builder setPassengerId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + passengerId_ = value; + onChanged(); + return this; + } + /** string passenger_id = 1; */ + public Builder clearPassengerId() { + + passengerId_ = getDefaultInstance().getPassengerId(); + onChanged(); + return this; + } + /** string passenger_id = 1; */ + public Builder setPassengerIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + passengerId_ = value; + onChanged(); + return this; + } + + private int startGeoCell_; + /** int32 start_geo_cell = 2; */ + public int getStartGeoCell() { + return startGeoCell_; + } + /** int32 start_geo_cell = 2; */ + public Builder setStartGeoCell(int value) { + + startGeoCell_ = value; + onChanged(); + return this; + } + /** int32 start_geo_cell = 2; */ + public Builder clearStartGeoCell() { + + startGeoCell_ = 0; + onChanged(); + return this; + } + + private int endGeoCell_; + /** int32 end_geo_cell = 3; */ + public int getEndGeoCell() { + return endGeoCell_; + } + /** int32 end_geo_cell = 3; */ + public Builder setEndGeoCell(int value) { + + endGeoCell_ = value; + onChanged(); + return this; + } + /** int32 end_geo_cell = 3; */ + public Builder clearEndGeoCell() { + + endGeoCell_ = 0; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.ridesharing.PassengerJoinsRide) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.ridesharing.PassengerJoinsRide) + private static final com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide(); + } + + public static com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public PassengerJoinsRide parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new PassengerJoinsRide(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.PassengerJoinsRide + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/PassengerJoinsRideOrBuilder.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/PassengerJoinsRideOrBuilder.java new file mode 100644 index 00000000..7cbd44a2 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/PassengerJoinsRideOrBuilder.java @@ -0,0 +1,21 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +public interface PassengerJoinsRideOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.ridesharing.PassengerJoinsRide) + com.google.protobuf.MessageOrBuilder { + + /** string passenger_id = 1; */ + java.lang.String getPassengerId(); + /** string passenger_id = 1; */ + com.google.protobuf.ByteString getPassengerIdBytes(); + + /** int32 start_geo_cell = 2; */ + int getStartGeoCell(); + + /** int32 end_geo_cell = 3; */ + int getEndGeoCell(); +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/PickupPassenger.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/PickupPassenger.java new file mode 100644 index 00000000..9b10e761 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/PickupPassenger.java @@ -0,0 +1,737 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +/** Protobuf type {@code com.ververica.statefun.examples.ridesharing.PickupPassenger} */ +public final class PickupPassenger extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.ridesharing.PickupPassenger) + PickupPassengerOrBuilder { + private static final long serialVersionUID = 0L; + // Use PickupPassenger.newBuilder() to construct. + private PickupPassenger(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private PickupPassenger() { + driverId_ = ""; + passengerId_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private PickupPassenger( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + java.lang.String s = input.readStringRequireUtf8(); + + driverId_ = s; + break; + } + case 18: + { + java.lang.String s = input.readStringRequireUtf8(); + + passengerId_ = s; + break; + } + case 24: + { + passengerStartCell_ = input.readInt32(); + break; + } + case 32: + { + passengerEndCell_ = input.readInt32(); + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_PickupPassenger_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_PickupPassenger_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.PickupPassenger.class, + com.ververica.statefun.examples.ridesharing.generated.PickupPassenger.Builder.class); + } + + public static final int DRIVER_ID_FIELD_NUMBER = 1; + private volatile java.lang.Object driverId_; + /** string driver_id = 1; */ + public java.lang.String getDriverId() { + java.lang.Object ref = driverId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + driverId_ = s; + return s; + } + } + /** string driver_id = 1; */ + public com.google.protobuf.ByteString getDriverIdBytes() { + java.lang.Object ref = driverId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + driverId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int PASSENGER_ID_FIELD_NUMBER = 2; + private volatile java.lang.Object passengerId_; + /** string passenger_id = 2; */ + public java.lang.String getPassengerId() { + java.lang.Object ref = passengerId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + passengerId_ = s; + return s; + } + } + /** string passenger_id = 2; */ + public com.google.protobuf.ByteString getPassengerIdBytes() { + java.lang.Object ref = passengerId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + passengerId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int PASSENGER_START_CELL_FIELD_NUMBER = 3; + private int passengerStartCell_; + /** int32 passenger_start_cell = 3; */ + public int getPassengerStartCell() { + return passengerStartCell_; + } + + public static final int PASSENGER_END_CELL_FIELD_NUMBER = 4; + private int passengerEndCell_; + /** int32 passenger_end_cell = 4; */ + public int getPassengerEndCell() { + return passengerEndCell_; + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!getDriverIdBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, driverId_); + } + if (!getPassengerIdBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 2, passengerId_); + } + if (passengerStartCell_ != 0) { + output.writeInt32(3, passengerStartCell_); + } + if (passengerEndCell_ != 0) { + output.writeInt32(4, passengerEndCell_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!getDriverIdBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, driverId_); + } + if (!getPassengerIdBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, passengerId_); + } + if (passengerStartCell_ != 0) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(3, passengerStartCell_); + } + if (passengerEndCell_ != 0) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(4, passengerEndCell_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.ververica.statefun.examples.ridesharing.generated.PickupPassenger)) { + return super.equals(obj); + } + com.ververica.statefun.examples.ridesharing.generated.PickupPassenger other = + (com.ververica.statefun.examples.ridesharing.generated.PickupPassenger) obj; + + if (!getDriverId().equals(other.getDriverId())) return false; + if (!getPassengerId().equals(other.getPassengerId())) return false; + if (getPassengerStartCell() != other.getPassengerStartCell()) return false; + if (getPassengerEndCell() != other.getPassengerEndCell()) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + DRIVER_ID_FIELD_NUMBER; + hash = (53 * hash) + getDriverId().hashCode(); + hash = (37 * hash) + PASSENGER_ID_FIELD_NUMBER; + hash = (53 * hash) + getPassengerId().hashCode(); + hash = (37 * hash) + PASSENGER_START_CELL_FIELD_NUMBER; + hash = (53 * hash) + getPassengerStartCell(); + hash = (37 * hash) + PASSENGER_END_CELL_FIELD_NUMBER; + hash = (53 * hash) + getPassengerEndCell(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.ridesharing.generated.PickupPassenger parseFrom( + java.nio.ByteBuffer data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.PickupPassenger parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.PickupPassenger parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.PickupPassenger parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.PickupPassenger parseFrom( + byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.PickupPassenger parseFrom( + byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.PickupPassenger parseFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.PickupPassenger parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.PickupPassenger + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.PickupPassenger + parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.PickupPassenger parseFrom( + com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.PickupPassenger parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.ridesharing.generated.PickupPassenger prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code com.ververica.statefun.examples.ridesharing.PickupPassenger} */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.ridesharing.PickupPassenger) + com.ververica.statefun.examples.ridesharing.generated.PickupPassengerOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_PickupPassenger_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_PickupPassenger_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.PickupPassenger.class, + com.ververica.statefun.examples.ridesharing.generated.PickupPassenger.Builder.class); + } + + // Construct using + // com.ververica.statefun.examples.ridesharing.generated.PickupPassenger.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + driverId_ = ""; + + passengerId_ = ""; + + passengerStartCell_ = 0; + + passengerEndCell_ = 0; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_PickupPassenger_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.PickupPassenger + getDefaultInstanceForType() { + return com.ververica.statefun.examples.ridesharing.generated.PickupPassenger + .getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.PickupPassenger build() { + com.ververica.statefun.examples.ridesharing.generated.PickupPassenger result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.PickupPassenger buildPartial() { + com.ververica.statefun.examples.ridesharing.generated.PickupPassenger result = + new com.ververica.statefun.examples.ridesharing.generated.PickupPassenger(this); + result.driverId_ = driverId_; + result.passengerId_ = passengerId_; + result.passengerStartCell_ = passengerStartCell_; + result.passengerEndCell_ = passengerEndCell_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.ververica.statefun.examples.ridesharing.generated.PickupPassenger) { + return mergeFrom( + (com.ververica.statefun.examples.ridesharing.generated.PickupPassenger) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.ridesharing.generated.PickupPassenger other) { + if (other + == com.ververica.statefun.examples.ridesharing.generated.PickupPassenger + .getDefaultInstance()) return this; + if (!other.getDriverId().isEmpty()) { + driverId_ = other.driverId_; + onChanged(); + } + if (!other.getPassengerId().isEmpty()) { + passengerId_ = other.passengerId_; + onChanged(); + } + if (other.getPassengerStartCell() != 0) { + setPassengerStartCell(other.getPassengerStartCell()); + } + if (other.getPassengerEndCell() != 0) { + setPassengerEndCell(other.getPassengerEndCell()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.ridesharing.generated.PickupPassenger parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.ridesharing.generated.PickupPassenger) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private java.lang.Object driverId_ = ""; + /** string driver_id = 1; */ + public java.lang.String getDriverId() { + java.lang.Object ref = driverId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + driverId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string driver_id = 1; */ + public com.google.protobuf.ByteString getDriverIdBytes() { + java.lang.Object ref = driverId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + driverId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string driver_id = 1; */ + public Builder setDriverId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + driverId_ = value; + onChanged(); + return this; + } + /** string driver_id = 1; */ + public Builder clearDriverId() { + + driverId_ = getDefaultInstance().getDriverId(); + onChanged(); + return this; + } + /** string driver_id = 1; */ + public Builder setDriverIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + driverId_ = value; + onChanged(); + return this; + } + + private java.lang.Object passengerId_ = ""; + /** string passenger_id = 2; */ + public java.lang.String getPassengerId() { + java.lang.Object ref = passengerId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + passengerId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string passenger_id = 2; */ + public com.google.protobuf.ByteString getPassengerIdBytes() { + java.lang.Object ref = passengerId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + passengerId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string passenger_id = 2; */ + public Builder setPassengerId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + passengerId_ = value; + onChanged(); + return this; + } + /** string passenger_id = 2; */ + public Builder clearPassengerId() { + + passengerId_ = getDefaultInstance().getPassengerId(); + onChanged(); + return this; + } + /** string passenger_id = 2; */ + public Builder setPassengerIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + passengerId_ = value; + onChanged(); + return this; + } + + private int passengerStartCell_; + /** int32 passenger_start_cell = 3; */ + public int getPassengerStartCell() { + return passengerStartCell_; + } + /** int32 passenger_start_cell = 3; */ + public Builder setPassengerStartCell(int value) { + + passengerStartCell_ = value; + onChanged(); + return this; + } + /** int32 passenger_start_cell = 3; */ + public Builder clearPassengerStartCell() { + + passengerStartCell_ = 0; + onChanged(); + return this; + } + + private int passengerEndCell_; + /** int32 passenger_end_cell = 4; */ + public int getPassengerEndCell() { + return passengerEndCell_; + } + /** int32 passenger_end_cell = 4; */ + public Builder setPassengerEndCell(int value) { + + passengerEndCell_ = value; + onChanged(); + return this; + } + /** int32 passenger_end_cell = 4; */ + public Builder clearPassengerEndCell() { + + passengerEndCell_ = 0; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.ridesharing.PickupPassenger) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.ridesharing.PickupPassenger) + private static final com.ververica.statefun.examples.ridesharing.generated.PickupPassenger + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = new com.ververica.statefun.examples.ridesharing.generated.PickupPassenger(); + } + + public static com.ververica.statefun.examples.ridesharing.generated.PickupPassenger + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public PickupPassenger parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new PickupPassenger(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.PickupPassenger + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/PickupPassengerOrBuilder.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/PickupPassengerOrBuilder.java new file mode 100644 index 00000000..dabcac4b --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/PickupPassengerOrBuilder.java @@ -0,0 +1,26 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +public interface PickupPassengerOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.ridesharing.PickupPassenger) + com.google.protobuf.MessageOrBuilder { + + /** string driver_id = 1; */ + java.lang.String getDriverId(); + /** string driver_id = 1; */ + com.google.protobuf.ByteString getDriverIdBytes(); + + /** string passenger_id = 2; */ + java.lang.String getPassengerId(); + /** string passenger_id = 2; */ + com.google.protobuf.ByteString getPassengerIdBytes(); + + /** int32 passenger_start_cell = 3; */ + int getPassengerStartCell(); + + /** int32 passenger_end_cell = 4; */ + int getPassengerEndCell(); +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/RideEnded.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/RideEnded.java new file mode 100644 index 00000000..80a436f7 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/RideEnded.java @@ -0,0 +1,427 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +/** Protobuf type {@code com.ververica.statefun.examples.ridesharing.RideEnded} */ +public final class RideEnded extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.ridesharing.RideEnded) + RideEndedOrBuilder { + private static final long serialVersionUID = 0L; + // Use RideEnded.newBuilder() to construct. + private RideEnded(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private RideEnded() {} + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private RideEnded( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_RideEnded_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_RideEnded_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.RideEnded.class, + com.ververica.statefun.examples.ridesharing.generated.RideEnded.Builder.class); + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.ververica.statefun.examples.ridesharing.generated.RideEnded)) { + return super.equals(obj); + } + com.ververica.statefun.examples.ridesharing.generated.RideEnded other = + (com.ververica.statefun.examples.ridesharing.generated.RideEnded) obj; + + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideEnded parseFrom( + java.nio.ByteBuffer data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideEnded parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideEnded parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideEnded parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideEnded parseFrom( + byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideEnded parseFrom( + byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideEnded parseFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideEnded parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideEnded parseDelimitedFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideEnded parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideEnded parseFrom( + com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideEnded parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.ridesharing.generated.RideEnded prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code com.ververica.statefun.examples.ridesharing.RideEnded} */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.ridesharing.RideEnded) + com.ververica.statefun.examples.ridesharing.generated.RideEndedOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_RideEnded_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_RideEnded_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.RideEnded.class, + com.ververica.statefun.examples.ridesharing.generated.RideEnded.Builder.class); + } + + // Construct using com.ververica.statefun.examples.ridesharing.generated.RideEnded.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_RideEnded_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.RideEnded + getDefaultInstanceForType() { + return com.ververica.statefun.examples.ridesharing.generated.RideEnded.getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.RideEnded build() { + com.ververica.statefun.examples.ridesharing.generated.RideEnded result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.RideEnded buildPartial() { + com.ververica.statefun.examples.ridesharing.generated.RideEnded result = + new com.ververica.statefun.examples.ridesharing.generated.RideEnded(this); + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.ververica.statefun.examples.ridesharing.generated.RideEnded) { + return mergeFrom((com.ververica.statefun.examples.ridesharing.generated.RideEnded) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.ridesharing.generated.RideEnded other) { + if (other + == com.ververica.statefun.examples.ridesharing.generated.RideEnded.getDefaultInstance()) + return this; + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.ridesharing.generated.RideEnded parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.ridesharing.generated.RideEnded) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.ridesharing.RideEnded) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.ridesharing.RideEnded) + private static final com.ververica.statefun.examples.ridesharing.generated.RideEnded + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = new com.ververica.statefun.examples.ridesharing.generated.RideEnded(); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideEnded + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public RideEnded parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new RideEnded(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.RideEnded + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/RideEndedOrBuilder.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/RideEndedOrBuilder.java new file mode 100644 index 00000000..4bb87092 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/RideEndedOrBuilder.java @@ -0,0 +1,9 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +public interface RideEndedOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.ridesharing.RideEnded) + com.google.protobuf.MessageOrBuilder {} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/RideFailed.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/RideFailed.java new file mode 100644 index 00000000..2ab9cc90 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/RideFailed.java @@ -0,0 +1,534 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +/** Protobuf type {@code com.ververica.statefun.examples.ridesharing.RideFailed} */ +public final class RideFailed extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.ridesharing.RideFailed) + RideFailedOrBuilder { + private static final long serialVersionUID = 0L; + // Use RideFailed.newBuilder() to construct. + private RideFailed(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private RideFailed() { + rideId_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private RideFailed( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + java.lang.String s = input.readStringRequireUtf8(); + + rideId_ = s; + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_RideFailed_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_RideFailed_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.RideFailed.class, + com.ververica.statefun.examples.ridesharing.generated.RideFailed.Builder.class); + } + + public static final int RIDE_ID_FIELD_NUMBER = 1; + private volatile java.lang.Object rideId_; + /** string ride_id = 1; */ + public java.lang.String getRideId() { + java.lang.Object ref = rideId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + rideId_ = s; + return s; + } + } + /** string ride_id = 1; */ + public com.google.protobuf.ByteString getRideIdBytes() { + java.lang.Object ref = rideId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + rideId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!getRideIdBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, rideId_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!getRideIdBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, rideId_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.ververica.statefun.examples.ridesharing.generated.RideFailed)) { + return super.equals(obj); + } + com.ververica.statefun.examples.ridesharing.generated.RideFailed other = + (com.ververica.statefun.examples.ridesharing.generated.RideFailed) obj; + + if (!getRideId().equals(other.getRideId())) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + RIDE_ID_FIELD_NUMBER; + hash = (53 * hash) + getRideId().hashCode(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideFailed parseFrom( + java.nio.ByteBuffer data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideFailed parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideFailed parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideFailed parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideFailed parseFrom( + byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideFailed parseFrom( + byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideFailed parseFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideFailed parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideFailed parseDelimitedFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideFailed parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideFailed parseFrom( + com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideFailed parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.ridesharing.generated.RideFailed prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code com.ververica.statefun.examples.ridesharing.RideFailed} */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.ridesharing.RideFailed) + com.ververica.statefun.examples.ridesharing.generated.RideFailedOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_RideFailed_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_RideFailed_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.RideFailed.class, + com.ververica.statefun.examples.ridesharing.generated.RideFailed.Builder.class); + } + + // Construct using com.ververica.statefun.examples.ridesharing.generated.RideFailed.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + rideId_ = ""; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_RideFailed_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.RideFailed + getDefaultInstanceForType() { + return com.ververica.statefun.examples.ridesharing.generated.RideFailed.getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.RideFailed build() { + com.ververica.statefun.examples.ridesharing.generated.RideFailed result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.RideFailed buildPartial() { + com.ververica.statefun.examples.ridesharing.generated.RideFailed result = + new com.ververica.statefun.examples.ridesharing.generated.RideFailed(this); + result.rideId_ = rideId_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.ververica.statefun.examples.ridesharing.generated.RideFailed) { + return mergeFrom((com.ververica.statefun.examples.ridesharing.generated.RideFailed) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.ridesharing.generated.RideFailed other) { + if (other + == com.ververica.statefun.examples.ridesharing.generated.RideFailed.getDefaultInstance()) + return this; + if (!other.getRideId().isEmpty()) { + rideId_ = other.rideId_; + onChanged(); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.ridesharing.generated.RideFailed parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.ridesharing.generated.RideFailed) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private java.lang.Object rideId_ = ""; + /** string ride_id = 1; */ + public java.lang.String getRideId() { + java.lang.Object ref = rideId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + rideId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string ride_id = 1; */ + public com.google.protobuf.ByteString getRideIdBytes() { + java.lang.Object ref = rideId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + rideId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string ride_id = 1; */ + public Builder setRideId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + rideId_ = value; + onChanged(); + return this; + } + /** string ride_id = 1; */ + public Builder clearRideId() { + + rideId_ = getDefaultInstance().getRideId(); + onChanged(); + return this; + } + /** string ride_id = 1; */ + public Builder setRideIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + rideId_ = value; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.ridesharing.RideFailed) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.ridesharing.RideFailed) + private static final com.ververica.statefun.examples.ridesharing.generated.RideFailed + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = new com.ververica.statefun.examples.ridesharing.generated.RideFailed(); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideFailed + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public RideFailed parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new RideFailed(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.RideFailed + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/RideFailedOrBuilder.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/RideFailedOrBuilder.java new file mode 100644 index 00000000..40298fe4 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/RideFailedOrBuilder.java @@ -0,0 +1,15 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +public interface RideFailedOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.ridesharing.RideFailed) + com.google.protobuf.MessageOrBuilder { + + /** string ride_id = 1; */ + java.lang.String getRideId(); + /** string ride_id = 1; */ + com.google.protobuf.ByteString getRideIdBytes(); +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/RideStarted.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/RideStarted.java new file mode 100644 index 00000000..366f43e6 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/RideStarted.java @@ -0,0 +1,583 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +/** Protobuf type {@code com.ververica.statefun.examples.ridesharing.RideStarted} */ +public final class RideStarted extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.ridesharing.RideStarted) + RideStartedOrBuilder { + private static final long serialVersionUID = 0L; + // Use RideStarted.newBuilder() to construct. + private RideStarted(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private RideStarted() { + driverId_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private RideStarted( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + java.lang.String s = input.readStringRequireUtf8(); + + driverId_ = s; + break; + } + case 16: + { + driverGeoCell_ = input.readInt32(); + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_RideStarted_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_RideStarted_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.RideStarted.class, + com.ververica.statefun.examples.ridesharing.generated.RideStarted.Builder.class); + } + + public static final int DRIVER_ID_FIELD_NUMBER = 1; + private volatile java.lang.Object driverId_; + /** string driver_id = 1; */ + public java.lang.String getDriverId() { + java.lang.Object ref = driverId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + driverId_ = s; + return s; + } + } + /** string driver_id = 1; */ + public com.google.protobuf.ByteString getDriverIdBytes() { + java.lang.Object ref = driverId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + driverId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int DRIVER_GEO_CELL_FIELD_NUMBER = 2; + private int driverGeoCell_; + /** int32 driver_geo_cell = 2; */ + public int getDriverGeoCell() { + return driverGeoCell_; + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!getDriverIdBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, driverId_); + } + if (driverGeoCell_ != 0) { + output.writeInt32(2, driverGeoCell_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!getDriverIdBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, driverId_); + } + if (driverGeoCell_ != 0) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(2, driverGeoCell_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.ververica.statefun.examples.ridesharing.generated.RideStarted)) { + return super.equals(obj); + } + com.ververica.statefun.examples.ridesharing.generated.RideStarted other = + (com.ververica.statefun.examples.ridesharing.generated.RideStarted) obj; + + if (!getDriverId().equals(other.getDriverId())) return false; + if (getDriverGeoCell() != other.getDriverGeoCell()) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + DRIVER_ID_FIELD_NUMBER; + hash = (53 * hash) + getDriverId().hashCode(); + hash = (37 * hash) + DRIVER_GEO_CELL_FIELD_NUMBER; + hash = (53 * hash) + getDriverGeoCell(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideStarted parseFrom( + java.nio.ByteBuffer data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideStarted parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideStarted parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideStarted parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideStarted parseFrom( + byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideStarted parseFrom( + byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideStarted parseFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideStarted parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideStarted + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideStarted + parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideStarted parseFrom( + com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideStarted parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.ridesharing.generated.RideStarted prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code com.ververica.statefun.examples.ridesharing.RideStarted} */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.ridesharing.RideStarted) + com.ververica.statefun.examples.ridesharing.generated.RideStartedOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_RideStarted_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_RideStarted_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.ridesharing.generated.RideStarted.class, + com.ververica.statefun.examples.ridesharing.generated.RideStarted.Builder.class); + } + + // Construct using + // com.ververica.statefun.examples.ridesharing.generated.RideStarted.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + driverId_ = ""; + + driverGeoCell_ = 0; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.ridesharing.generated.Ridesharing + .internal_static_com_ververica_statefun_examples_ridesharing_RideStarted_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.RideStarted + getDefaultInstanceForType() { + return com.ververica.statefun.examples.ridesharing.generated.RideStarted.getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.RideStarted build() { + com.ververica.statefun.examples.ridesharing.generated.RideStarted result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.RideStarted buildPartial() { + com.ververica.statefun.examples.ridesharing.generated.RideStarted result = + new com.ververica.statefun.examples.ridesharing.generated.RideStarted(this); + result.driverId_ = driverId_; + result.driverGeoCell_ = driverGeoCell_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.ververica.statefun.examples.ridesharing.generated.RideStarted) { + return mergeFrom((com.ververica.statefun.examples.ridesharing.generated.RideStarted) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.ridesharing.generated.RideStarted other) { + if (other + == com.ververica.statefun.examples.ridesharing.generated.RideStarted.getDefaultInstance()) + return this; + if (!other.getDriverId().isEmpty()) { + driverId_ = other.driverId_; + onChanged(); + } + if (other.getDriverGeoCell() != 0) { + setDriverGeoCell(other.getDriverGeoCell()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.ridesharing.generated.RideStarted parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.ridesharing.generated.RideStarted) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private java.lang.Object driverId_ = ""; + /** string driver_id = 1; */ + public java.lang.String getDriverId() { + java.lang.Object ref = driverId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + driverId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string driver_id = 1; */ + public com.google.protobuf.ByteString getDriverIdBytes() { + java.lang.Object ref = driverId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + driverId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string driver_id = 1; */ + public Builder setDriverId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + driverId_ = value; + onChanged(); + return this; + } + /** string driver_id = 1; */ + public Builder clearDriverId() { + + driverId_ = getDefaultInstance().getDriverId(); + onChanged(); + return this; + } + /** string driver_id = 1; */ + public Builder setDriverIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + driverId_ = value; + onChanged(); + return this; + } + + private int driverGeoCell_; + /** int32 driver_geo_cell = 2; */ + public int getDriverGeoCell() { + return driverGeoCell_; + } + /** int32 driver_geo_cell = 2; */ + public Builder setDriverGeoCell(int value) { + + driverGeoCell_ = value; + onChanged(); + return this; + } + /** int32 driver_geo_cell = 2; */ + public Builder clearDriverGeoCell() { + + driverGeoCell_ = 0; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.ridesharing.RideStarted) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.ridesharing.RideStarted) + private static final com.ververica.statefun.examples.ridesharing.generated.RideStarted + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = new com.ververica.statefun.examples.ridesharing.generated.RideStarted(); + } + + public static com.ververica.statefun.examples.ridesharing.generated.RideStarted + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public RideStarted parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new RideStarted(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.ridesharing.generated.RideStarted + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/RideStartedOrBuilder.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/RideStartedOrBuilder.java new file mode 100644 index 00000000..4aa5d3b2 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/RideStartedOrBuilder.java @@ -0,0 +1,18 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +public interface RideStartedOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.ridesharing.RideStarted) + com.google.protobuf.MessageOrBuilder { + + /** string driver_id = 1; */ + java.lang.String getDriverId(); + /** string driver_id = 1; */ + com.google.protobuf.ByteString getDriverIdBytes(); + + /** int32 driver_geo_cell = 2; */ + int getDriverGeoCell(); +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/Ridesharing.java b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/Ridesharing.java new file mode 100644 index 00000000..57946ff0 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/java/com/ververica/statefun/examples/ridesharing/generated/Ridesharing.java @@ -0,0 +1,402 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/ridesharing.proto + +package com.ververica.statefun.examples.ridesharing.generated; + +public final class Ridesharing { + private Ridesharing() {} + + public static void registerAllExtensions(com.google.protobuf.ExtensionRegistryLite registry) {} + + public static void registerAllExtensions(com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions((com.google.protobuf.ExtensionRegistryLite) registry); + } + + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_ridesharing_InboundPassengerMessage_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_ridesharing_InboundPassengerMessage_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_ridesharing_InboundPassengerMessage_RequestRide_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_ridesharing_InboundPassengerMessage_RequestRide_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_DriverHasBeenFound_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_DriverHasBeenFound_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_RideFailed_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_RideFailed_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_RideStarted_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_RideStarted_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_RideEnded_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_RideEnded_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_RideStarted_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_RideStarted_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_RideEnded_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_RideEnded_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_LocationUpdate_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_LocationUpdate_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_ridesharing_OutboundDriverMessage_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_ridesharing_OutboundDriverMessage_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_ridesharing_OutboundDriverMessage_PickupPassenger_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_ridesharing_OutboundDriverMessage_PickupPassenger_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_ridesharing_DriverJoinsRide_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_ridesharing_DriverJoinsRide_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_ridesharing_PassengerJoinsRide_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_ridesharing_PassengerJoinsRide_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_ridesharing_RideStarted_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_ridesharing_RideStarted_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_ridesharing_RideFailed_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_ridesharing_RideFailed_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_ridesharing_RideEnded_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_ridesharing_RideEnded_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_ridesharing_JoinCell_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_ridesharing_JoinCell_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_ridesharing_LeaveCell_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_ridesharing_LeaveCell_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_ridesharing_GetDriver_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_ridesharing_GetDriver_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_ridesharing_DriverInCell_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_ridesharing_DriverInCell_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_ridesharing_PickupPassenger_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_ridesharing_PickupPassenger_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_ridesharing_DriverRejectsPickup_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_ridesharing_DriverRejectsPickup_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_ridesharing_GeoCellState_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_ridesharing_GeoCellState_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { + return descriptor; + } + + private static com.google.protobuf.Descriptors.FileDescriptor descriptor; + + static { + java.lang.String[] descriptorData = { + "\n#src/main/protobuf/ridesharing.proto\022+c" + + "om.ververica.statefun.examples.rideshari" + + "ng\"\341\001\n\027InboundPassengerMessage\022\024\n\014passen" + + "ger_id\030\001 \001(\t\022h\n\014request_ride\030\002 \001(\0132P.com" + + ".ververica.statefun.examples.ridesharing" + + ".InboundPassengerMessage.RequestRideH\000\032;" + + "\n\013RequestRide\022\026\n\016start_geo_cell\030\002 \001(\005\022\024\n" + + "\014end_geo_cell\030\003 \001(\005B\t\n\007message\"\370\004\n\030Outbo" + + "undPassengerMessage\022\024\n\014passenger_id\030\001 \001(" + + "\t\022p\n\014driver_found\030\002 \001(\0132X.com.ververica." + + "statefun.examples.ridesharing.OutboundPa" + + "ssengerMessage.DriverHasBeenFoundH\000\022g\n\013r" + + "ide_failed\030\003 \001(\0132P.com.ververica.statefu" + + "n.examples.ridesharing.OutboundPassenger" + + "Message.RideFailedH\000\022i\n\014ride_started\030\004 \001" + + "(\0132Q.com.ververica.statefun.examples.rid" + + "esharing.OutboundPassengerMessage.RideSt" + + "artedH\000\022e\n\nride_ended\030\005 \001(\0132O.com.verver" + + "ica.statefun.examples.ridesharing.Outbou" + + "ndPassengerMessage.RideEndedH\000\032@\n\022Driver" + + "HasBeenFound\022\021\n\tdriver_id\030\002 \001(\t\022\027\n\017drive" + + "r_geo_cell\030\003 \001(\005\032\035\n\nRideFailed\022\017\n\007ride_i" + + "d\030\002 \001(\t\032 \n\013RideStarted\022\021\n\tdriver_id\030\002 \001(" + + "\t\032\013\n\tRideEndedB\t\n\007message\"\304\003\n\024InboundDri" + + "verMessage\022\021\n\tdriver_id\030\001 \001(\t\022e\n\014ride_st" + + "arted\030\002 \001(\0132M.com.ververica.statefun.exa" + + "mples.ridesharing.InboundDriverMessage.R" + + "ideStartedH\000\022a\n\nride_ended\030\003 \001(\0132K.com.v" + + "erverica.statefun.examples.ridesharing.I" + + "nboundDriverMessage.RideEndedH\000\022k\n\017locat" + + "ion_update\030\004 \001(\0132P.com.ververica.statefu" + + "n.examples.ridesharing.InboundDriverMess" + + "age.LocationUpdateH\000\032\r\n\013RideStarted\032\034\n\tR" + + "ideEnded\022\017\n\007ride_id\030\002 \001(\t\032*\n\016LocationUpd" + + "ate\022\030\n\020current_geo_cell\030\002 \001(\005B\t\n\007message" + + "\"\377\001\n\025OutboundDriverMessage\022\021\n\tdriver_id\030" + + "\001 \001(\t\022n\n\020pickup_passenger\030\002 \001(\0132R.com.ve" + + "rverica.statefun.examples.ridesharing.Ou" + + "tboundDriverMessage.PickupPassengerH\000\032X\n" + + "\017PickupPassenger\022\017\n\007ride_id\030\002 \001(\t\022\032\n\022sta" + + "rt_geo_location\030\003 \001(\005\022\030\n\020end_geo_locatio" + + "n\030\004 \001(\005B\t\n\007message\"S\n\017DriverJoinsRide\022\021\n" + + "\tdriver_id\030\001 \001(\t\022\024\n\014passenger_id\030\002 \001(\t\022\027" + + "\n\017driver_location\030\003 \001(\005\"X\n\022PassengerJoin" + + "sRide\022\024\n\014passenger_id\030\001 \001(\t\022\026\n\016start_geo" + + "_cell\030\002 \001(\005\022\024\n\014end_geo_cell\030\003 \001(\005\"9\n\013Rid" + + "eStarted\022\021\n\tdriver_id\030\001 \001(\t\022\027\n\017driver_ge" + + "o_cell\030\002 \001(\005\"\035\n\nRideFailed\022\017\n\007ride_id\030\001 " + + "\001(\t\"\013\n\tRideEnded\"\n\n\010JoinCell\"\013\n\tLeaveCel" + + "l\"\013\n\tGetDriver\"!\n\014DriverInCell\022\021\n\tdriver" + + "_id\030\001 \001(\t\"t\n\017PickupPassenger\022\021\n\tdriver_i" + + "d\030\001 \001(\t\022\024\n\014passenger_id\030\002 \001(\t\022\034\n\024passeng" + + "er_start_cell\030\003 \001(\005\022\032\n\022passenger_end_cel" + + "l\030\004 \001(\005\"9\n\023DriverRejectsPickup\022\021\n\tdriver" + + "_id\030\001 \001(\t\022\017\n\007ride_id\030\002 \001(\t\"!\n\014GeoCellSta" + + "te\022\021\n\tdriver_id\030\001 \003(\tB?\n5com.ververica.s" + + "tatefun.examples.ridesharing.generatedP\001" + + "Z\004mainb\006proto3" + }; + com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = + new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { + public com.google.protobuf.ExtensionRegistry assignDescriptors( + com.google.protobuf.Descriptors.FileDescriptor root) { + descriptor = root; + return null; + } + }; + com.google.protobuf.Descriptors.FileDescriptor.internalBuildGeneratedFileFrom( + descriptorData, new com.google.protobuf.Descriptors.FileDescriptor[] {}, assigner); + internal_static_com_ververica_statefun_examples_ridesharing_InboundPassengerMessage_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_com_ververica_statefun_examples_ridesharing_InboundPassengerMessage_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_ridesharing_InboundPassengerMessage_descriptor, + new java.lang.String[] { + "PassengerId", "RequestRide", "Message", + }); + internal_static_com_ververica_statefun_examples_ridesharing_InboundPassengerMessage_RequestRide_descriptor = + internal_static_com_ververica_statefun_examples_ridesharing_InboundPassengerMessage_descriptor + .getNestedTypes() + .get(0); + internal_static_com_ververica_statefun_examples_ridesharing_InboundPassengerMessage_RequestRide_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_ridesharing_InboundPassengerMessage_RequestRide_descriptor, + new java.lang.String[] { + "StartGeoCell", "EndGeoCell", + }); + internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_descriptor = + getDescriptor().getMessageTypes().get(1); + internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_descriptor, + new java.lang.String[] { + "PassengerId", "DriverFound", "RideFailed", "RideStarted", "RideEnded", "Message", + }); + internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_DriverHasBeenFound_descriptor = + internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_descriptor + .getNestedTypes() + .get(0); + internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_DriverHasBeenFound_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_DriverHasBeenFound_descriptor, + new java.lang.String[] { + "DriverId", "DriverGeoCell", + }); + internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_RideFailed_descriptor = + internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_descriptor + .getNestedTypes() + .get(1); + internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_RideFailed_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_RideFailed_descriptor, + new java.lang.String[] { + "RideId", + }); + internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_RideStarted_descriptor = + internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_descriptor + .getNestedTypes() + .get(2); + internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_RideStarted_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_RideStarted_descriptor, + new java.lang.String[] { + "DriverId", + }); + internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_RideEnded_descriptor = + internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_descriptor + .getNestedTypes() + .get(3); + internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_RideEnded_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_ridesharing_OutboundPassengerMessage_RideEnded_descriptor, + new java.lang.String[] {}); + internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_descriptor = + getDescriptor().getMessageTypes().get(2); + internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_descriptor, + new java.lang.String[] { + "DriverId", "RideStarted", "RideEnded", "LocationUpdate", "Message", + }); + internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_RideStarted_descriptor = + internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_descriptor + .getNestedTypes() + .get(0); + internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_RideStarted_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_RideStarted_descriptor, + new java.lang.String[] {}); + internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_RideEnded_descriptor = + internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_descriptor + .getNestedTypes() + .get(1); + internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_RideEnded_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_RideEnded_descriptor, + new java.lang.String[] { + "RideId", + }); + internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_LocationUpdate_descriptor = + internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_descriptor + .getNestedTypes() + .get(2); + internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_LocationUpdate_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_ridesharing_InboundDriverMessage_LocationUpdate_descriptor, + new java.lang.String[] { + "CurrentGeoCell", + }); + internal_static_com_ververica_statefun_examples_ridesharing_OutboundDriverMessage_descriptor = + getDescriptor().getMessageTypes().get(3); + internal_static_com_ververica_statefun_examples_ridesharing_OutboundDriverMessage_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_ridesharing_OutboundDriverMessage_descriptor, + new java.lang.String[] { + "DriverId", "PickupPassenger", "Message", + }); + internal_static_com_ververica_statefun_examples_ridesharing_OutboundDriverMessage_PickupPassenger_descriptor = + internal_static_com_ververica_statefun_examples_ridesharing_OutboundDriverMessage_descriptor + .getNestedTypes() + .get(0); + internal_static_com_ververica_statefun_examples_ridesharing_OutboundDriverMessage_PickupPassenger_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_ridesharing_OutboundDriverMessage_PickupPassenger_descriptor, + new java.lang.String[] { + "RideId", "StartGeoLocation", "EndGeoLocation", + }); + internal_static_com_ververica_statefun_examples_ridesharing_DriverJoinsRide_descriptor = + getDescriptor().getMessageTypes().get(4); + internal_static_com_ververica_statefun_examples_ridesharing_DriverJoinsRide_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_ridesharing_DriverJoinsRide_descriptor, + new java.lang.String[] { + "DriverId", "PassengerId", "DriverLocation", + }); + internal_static_com_ververica_statefun_examples_ridesharing_PassengerJoinsRide_descriptor = + getDescriptor().getMessageTypes().get(5); + internal_static_com_ververica_statefun_examples_ridesharing_PassengerJoinsRide_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_ridesharing_PassengerJoinsRide_descriptor, + new java.lang.String[] { + "PassengerId", "StartGeoCell", "EndGeoCell", + }); + internal_static_com_ververica_statefun_examples_ridesharing_RideStarted_descriptor = + getDescriptor().getMessageTypes().get(6); + internal_static_com_ververica_statefun_examples_ridesharing_RideStarted_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_ridesharing_RideStarted_descriptor, + new java.lang.String[] { + "DriverId", "DriverGeoCell", + }); + internal_static_com_ververica_statefun_examples_ridesharing_RideFailed_descriptor = + getDescriptor().getMessageTypes().get(7); + internal_static_com_ververica_statefun_examples_ridesharing_RideFailed_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_ridesharing_RideFailed_descriptor, + new java.lang.String[] { + "RideId", + }); + internal_static_com_ververica_statefun_examples_ridesharing_RideEnded_descriptor = + getDescriptor().getMessageTypes().get(8); + internal_static_com_ververica_statefun_examples_ridesharing_RideEnded_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_ridesharing_RideEnded_descriptor, + new java.lang.String[] {}); + internal_static_com_ververica_statefun_examples_ridesharing_JoinCell_descriptor = + getDescriptor().getMessageTypes().get(9); + internal_static_com_ververica_statefun_examples_ridesharing_JoinCell_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_ridesharing_JoinCell_descriptor, + new java.lang.String[] {}); + internal_static_com_ververica_statefun_examples_ridesharing_LeaveCell_descriptor = + getDescriptor().getMessageTypes().get(10); + internal_static_com_ververica_statefun_examples_ridesharing_LeaveCell_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_ridesharing_LeaveCell_descriptor, + new java.lang.String[] {}); + internal_static_com_ververica_statefun_examples_ridesharing_GetDriver_descriptor = + getDescriptor().getMessageTypes().get(11); + internal_static_com_ververica_statefun_examples_ridesharing_GetDriver_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_ridesharing_GetDriver_descriptor, + new java.lang.String[] {}); + internal_static_com_ververica_statefun_examples_ridesharing_DriverInCell_descriptor = + getDescriptor().getMessageTypes().get(12); + internal_static_com_ververica_statefun_examples_ridesharing_DriverInCell_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_ridesharing_DriverInCell_descriptor, + new java.lang.String[] { + "DriverId", + }); + internal_static_com_ververica_statefun_examples_ridesharing_PickupPassenger_descriptor = + getDescriptor().getMessageTypes().get(13); + internal_static_com_ververica_statefun_examples_ridesharing_PickupPassenger_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_ridesharing_PickupPassenger_descriptor, + new java.lang.String[] { + "DriverId", "PassengerId", "PassengerStartCell", "PassengerEndCell", + }); + internal_static_com_ververica_statefun_examples_ridesharing_DriverRejectsPickup_descriptor = + getDescriptor().getMessageTypes().get(14); + internal_static_com_ververica_statefun_examples_ridesharing_DriverRejectsPickup_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_ridesharing_DriverRejectsPickup_descriptor, + new java.lang.String[] { + "DriverId", "RideId", + }); + internal_static_com_ververica_statefun_examples_ridesharing_GeoCellState_descriptor = + getDescriptor().getMessageTypes().get(15); + internal_static_com_ververica_statefun_examples_ridesharing_GeoCellState_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_ridesharing_GeoCellState_descriptor, + new java.lang.String[] { + "DriverId", + }); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/protobuf/ridesharing.proto b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/protobuf/ridesharing.proto new file mode 100644 index 00000000..c5a5a60a --- /dev/null +++ b/stateful-functions-examples/stateful-functions-ridesharing-example/stateful-functions-ridesharing-protocol/src/main/protobuf/ridesharing.proto @@ -0,0 +1,183 @@ +// +// Copyright 2019 Ververica GmbH. +// +// 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"; + +package com.ververica.statefun.examples.ridesharing; + +option java_package = "com.ververica.statefun.examples.ridesharing.generated"; +option java_multiple_files = true; + +option go_package = "main"; + +// ----------------------------------------------------------------------------------------- +// External messages +// ----------------------------------------------------------------------------------------- + +// the messages that corresponds to the FROM_PASSENGER_INGRESS +message InboundPassengerMessage { + + // A message sent by the physical driver to the ridesharing statefun job, indicating that the user + // would like to be picked up from start_geo_cell and dropped off at end_geo_cell + message RequestRide { + int32 start_geo_cell = 2; + int32 end_geo_cell = 3; + } + + string passenger_id = 1; + + oneof message { + RequestRide request_ride = 2; + } +} + +// the messages that corresponds to the TO_PASSENGER_EGRESS +message OutboundPassengerMessage { + + // A message that is sent to the passenger, when there is a driver that + // is ready to pick them up. + message DriverHasBeenFound { + string driver_id = 2; + int32 driver_geo_cell = 3; + } + // A message that is sent to the passenger when the system couldn't find in reasonable time + // a driver to pickup the passenger. + message RideFailed { + string ride_id = 2; + } + // A notification that is sent to the passenger indicating that the ride has started. + message RideStarted { + string driver_id = 2; + } + // A notification that is sent to the passenger indicating that the ride has end. + message RideEnded { + } + + string passenger_id = 1; + + oneof message { + DriverHasBeenFound driver_found = 2; + RideFailed ride_failed = 3; + RideStarted ride_started = 4; + RideEnded ride_ended = 5; + } +} + +// the messages that corresponds to the FROM_DRIVER_INGRESS +message InboundDriverMessage { + + // A message the driver sends once they pickup the passenger + message RideStarted { + } + + // A message the driver sends when they drop off the passenger. + message RideEnded { + string ride_id = 2; + } + + // A periodic location update message + message LocationUpdate { + int32 current_geo_cell = 2; + } + + string driver_id = 1; + + oneof message { + RideStarted ride_started = 2; + RideEnded ride_ended = 3; + LocationUpdate location_update = 4; + } +} + +// the messages that corresponds to the TO_DRIVER_EGRESS +message OutboundDriverMessage { + + // A message that is sent to the physical driver to indicate that + // a pickup is needed. + // a pickup is needed. + message PickupPassenger { + string ride_id = 2; + int32 start_geo_location = 3; + int32 end_geo_location = 4; + } + + string driver_id = 1; + + oneof message { + PickupPassenger pickup_passenger = 2; + } +} + +// ----------------------------------------------------------------------------------------- +// Internal messages +// ----------------------------------------------------------------------------------------- + +message DriverJoinsRide { + string driver_id = 1; + string passenger_id = 2; + int32 driver_location = 3; +} + +message PassengerJoinsRide { + string passenger_id = 1; + int32 start_geo_cell = 2; + int32 end_geo_cell = 3; +} + +message RideStarted { + string driver_id = 1; + int32 driver_geo_cell = 2; +} + +message RideFailed { + string ride_id = 1; +} + +message RideEnded { +} + +message JoinCell { +} + +message LeaveCell { +} + +message GetDriver { +} + +message DriverInCell { + string driver_id = 1; +} + +message PickupPassenger { + string driver_id = 1; + string passenger_id = 2; + int32 passenger_start_cell = 3; + int32 passenger_end_cell = 4; +} + +message DriverRejectsPickup { + string driver_id = 1; + string ride_id = 2; +} + +// ----------------------------------------------------------------------------------------- +// Internal States +// ----------------------------------------------------------------------------------------- + +message GeoCellState { + repeated string driver_id = 1; +} \ No newline at end of file diff --git a/stateful-functions-examples/stateful-functions-shopping-cart-example/Dockerfile b/stateful-functions-examples/stateful-functions-shopping-cart-example/Dockerfile new file mode 100644 index 00000000..db2ae1b6 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-shopping-cart-example/Dockerfile @@ -0,0 +1,20 @@ +# +# Copyright 2019 Ververica GmbH. +# +# 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. +# + +FROM stateful-functions + +RUN mkdir -p /opt/stateful-functions/modules/stateful-functions-shopping-cart-example +COPY target/stateful-functions-shopping-cart-example*jar /opt/stateful-functions/modules/stateful-functions-shopping-cart-example/ diff --git a/stateful-functions-examples/stateful-functions-shopping-cart-example/pom.xml b/stateful-functions-examples/stateful-functions-shopping-cart-example/pom.xml new file mode 100644 index 00000000..adab9627 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-shopping-cart-example/pom.xml @@ -0,0 +1,56 @@ + + + + + stateful-functions-examples + com.ververica + 1.0-SNAPSHOT + .. + + 4.0.0 + + stateful-functions-shopping-cart-example + + + + com.ververica + stateful-functions-sdk + ${project.version} + + + com.ververica + stateful-functions-kafka-io + ${project.version} + + + com.google.protobuf + protobuf-java + 3.8.0 + + + + + junit + junit + 4.12 + test + + + + diff --git a/stateful-functions-examples/stateful-functions-shopping-cart-example/src/main/java/com/ververica/statefun/examples/shoppingcart/Identifiers.java b/stateful-functions-examples/stateful-functions-shopping-cart-example/src/main/java/com/ververica/statefun/examples/shoppingcart/Identifiers.java new file mode 100644 index 00000000..08eb3ee4 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-shopping-cart-example/src/main/java/com/ververica/statefun/examples/shoppingcart/Identifiers.java @@ -0,0 +1,37 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.shoppingcart; + +import com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages; +import com.ververica.statefun.sdk.FunctionType; +import com.ververica.statefun.sdk.io.EgressIdentifier; +import com.ververica.statefun.sdk.io.IngressIdentifier; + +final class Identifiers { + + private Identifiers() {} + + static final FunctionType USER = new FunctionType("shopping-cart", "user"); + + static final FunctionType INVENTORY = new FunctionType("shopping-cart", "inventory"); + + static final IngressIdentifier RESTOCK = + new IngressIdentifier<>(ProtobufMessages.RestockItem.class, "shopping-cart", "restock-item"); + + static final EgressIdentifier RECEIPT = + new EgressIdentifier<>("shopping-cart", "receipt", ProtobufMessages.Receipt.class); +} diff --git a/stateful-functions-examples/stateful-functions-shopping-cart-example/src/main/java/com/ververica/statefun/examples/shoppingcart/Inventory.java b/stateful-functions-examples/stateful-functions-shopping-cart-example/src/main/java/com/ververica/statefun/examples/shoppingcart/Inventory.java new file mode 100644 index 00000000..b9f85e7b --- /dev/null +++ b/stateful-functions-examples/stateful-functions-shopping-cart-example/src/main/java/com/ververica/statefun/examples/shoppingcart/Inventory.java @@ -0,0 +1,54 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.shoppingcart; + +import com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ItemAvailability; +import com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RequestItem; +import com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RestockItem; +import com.ververica.statefun.sdk.Context; +import com.ververica.statefun.sdk.StatefulFunction; +import com.ververica.statefun.sdk.annotations.Persisted; +import com.ververica.statefun.sdk.state.PersistedValue; + +final class Inventory implements StatefulFunction { + + @Persisted + private final PersistedValue inventory = PersistedValue.of("inventory", Integer.class); + + @Override + public void invoke(Context context, Object message) { + if (message instanceof RestockItem) { + int quantity = inventory.getOrDefault(0) + ((RestockItem) message).getQuantity(); + inventory.set(quantity); + } else if (message instanceof RequestItem) { + int quantity = inventory.getOrDefault(0); + int requestedAmount = ((RequestItem) message).getQuantity(); + + ItemAvailability.Builder availability = + ItemAvailability.newBuilder().setQuantity(requestedAmount); + + if (quantity >= requestedAmount) { + inventory.set(quantity - requestedAmount); + availability.setStatus(ItemAvailability.Status.INSTOCK); + } else { + availability.setStatus(ItemAvailability.Status.OUTOFSTOCK); + } + + context.send(context.caller(), availability.build()); + } + } +} diff --git a/stateful-functions-examples/stateful-functions-shopping-cart-example/src/main/java/com/ververica/statefun/examples/shoppingcart/RestockRouter.java b/stateful-functions-examples/stateful-functions-shopping-cart-example/src/main/java/com/ververica/statefun/examples/shoppingcart/RestockRouter.java new file mode 100644 index 00000000..3fd3e9b4 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-shopping-cart-example/src/main/java/com/ververica/statefun/examples/shoppingcart/RestockRouter.java @@ -0,0 +1,27 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.shoppingcart; + +import com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RestockItem; +import com.ververica.statefun.sdk.io.Router; + +final class RestockRouter implements Router { + @Override + public void route(RestockItem message, Downstream downstream) { + downstream.forward(Identifiers.INVENTORY, message.getItemId(), message); + } +} diff --git a/stateful-functions-examples/stateful-functions-shopping-cart-example/src/main/java/com/ververica/statefun/examples/shoppingcart/ShoppingCartModule.java b/stateful-functions-examples/stateful-functions-shopping-cart-example/src/main/java/com/ververica/statefun/examples/shoppingcart/ShoppingCartModule.java new file mode 100644 index 00000000..4561e496 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-shopping-cart-example/src/main/java/com/ververica/statefun/examples/shoppingcart/ShoppingCartModule.java @@ -0,0 +1,48 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.shoppingcart; + +import com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt; +import com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RestockItem; +import com.ververica.statefun.sdk.io.EgressSpec; +import com.ververica.statefun.sdk.io.IngressSpec; +import com.ververica.statefun.sdk.kafka.KafkaEgressBuilder; +import com.ververica.statefun.sdk.kafka.KafkaIngressBuilder; +import com.ververica.statefun.sdk.spi.StatefulFunctionModule; +import java.util.Map; + +public class ShoppingCartModule implements StatefulFunctionModule { + @Override + public void configure(Map globalConfiguration, Binder binder) { + // bind functions + binder.bindFunctionProvider(Identifiers.USER, unused -> new UserShoppingCart()); + binder.bindFunctionProvider(Identifiers.INVENTORY, unused -> new Inventory()); + + // For ingress and egress pretend I filled in the details :) + + IngressSpec restockSpec = + KafkaIngressBuilder.forIdentifier(Identifiers.RESTOCK).build(); + + binder.bindIngress(restockSpec); + + binder.bindIngressRouter(Identifiers.RESTOCK, new RestockRouter()); + + EgressSpec receiptSpec = KafkaEgressBuilder.forIdentifier(Identifiers.RECEIPT).build(); + + binder.bindEgress(receiptSpec); + } +} diff --git a/stateful-functions-examples/stateful-functions-shopping-cart-example/src/main/java/com/ververica/statefun/examples/shoppingcart/UserShoppingCart.java b/stateful-functions-examples/stateful-functions-shopping-cart-example/src/main/java/com/ververica/statefun/examples/shoppingcart/UserShoppingCart.java new file mode 100644 index 00000000..1e2f2f6e --- /dev/null +++ b/stateful-functions-examples/stateful-functions-shopping-cart-example/src/main/java/com/ververica/statefun/examples/shoppingcart/UserShoppingCart.java @@ -0,0 +1,93 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.examples.shoppingcart; + +import com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart; +import com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket; +import com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout; +import com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart; +import com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ItemAvailability; +import com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt; +import com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RequestItem; +import com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RestockItem; +import com.ververica.statefun.sdk.Address; +import com.ververica.statefun.sdk.Context; +import com.ververica.statefun.sdk.StatefulFunction; +import com.ververica.statefun.sdk.annotations.Persisted; +import com.ververica.statefun.sdk.state.PersistedValue; +import java.util.Map; + +final class UserShoppingCart implements StatefulFunction { + + @Persisted + private final PersistedValue userBasket = PersistedValue.of("basket", Basket.class); + + @Override + public void invoke(Context context, Object input) { + if (input instanceof AddToCart) { + AddToCart addToCart = (AddToCart) input; + RequestItem request = RequestItem.newBuilder().setQuantity(addToCart.getQuantity()).build(); + Address address = new Address(Identifiers.INVENTORY, addToCart.getItemId()); + context.send(address, request); + } + + if (input instanceof ItemAvailability) { + ItemAvailability availability = (ItemAvailability) input; + + if (availability.getStatus() == ItemAvailability.Status.INSTOCK) { + Basket basket = userBasket.getOrDefault(() -> Basket.newBuilder().build()); + basket.getItemsMap().put(context.caller().id(), availability.getQuantity()); + } + } + + if (input instanceof ClearCart) { + Basket basket = userBasket.get(); + if (basket == null) { + return; + } + + for (Map.Entry entry : basket.getItemsMap().entrySet()) { + RestockItem item = + RestockItem.newBuilder() + .setItemId(entry.getKey()) + .setQuantity(entry.getValue()) + .build(); + + Address address = new Address(Identifiers.INVENTORY, entry.getKey()); + context.send(address, item); + } + + userBasket.clear(); + } + + if (input instanceof Checkout) { + Basket basket = userBasket.get(); + if (basket == null) { + return; + } + + Receipt receipt = + Receipt.newBuilder() + .setUserId(context.self().id()) + .setDetails("You bought " + basket.getItemsCount() + " items") + .build(); + + context.send(Identifiers.RECEIPT, receipt); + userBasket.clear(); + } + } +} diff --git a/stateful-functions-examples/stateful-functions-shopping-cart-example/src/main/java/com/ververica/statefun/examples/shoppingcart/generated/ProtobufMessages.java b/stateful-functions-examples/stateful-functions-shopping-cart-example/src/main/java/com/ververica/statefun/examples/shoppingcart/generated/ProtobufMessages.java new file mode 100644 index 00000000..4ce217fc --- /dev/null +++ b/stateful-functions-examples/stateful-functions-shopping-cart-example/src/main/java/com/ververica/statefun/examples/shoppingcart/generated/ProtobufMessages.java @@ -0,0 +1,5332 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/shoppingcart.proto + +package com.ververica.statefun.examples.shoppingcart.generated; + +public final class ProtobufMessages { + private ProtobufMessages() {} + + public static void registerAllExtensions(com.google.protobuf.ExtensionRegistryLite registry) {} + + public static void registerAllExtensions(com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions((com.google.protobuf.ExtensionRegistryLite) registry); + } + + public interface AddToCartOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.shoppingcart.AddToCart) + com.google.protobuf.MessageOrBuilder { + + /** string user_id = 1; */ + java.lang.String getUserId(); + /** string user_id = 1; */ + com.google.protobuf.ByteString getUserIdBytes(); + + /** string item_id = 2; */ + java.lang.String getItemId(); + /** string item_id = 2; */ + com.google.protobuf.ByteString getItemIdBytes(); + + /** int32 quantity = 3; */ + int getQuantity(); + } + /** Protobuf type {@code com.ververica.statefun.examples.shoppingcart.AddToCart} */ + public static final class AddToCart extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.shoppingcart.AddToCart) + AddToCartOrBuilder { + private static final long serialVersionUID = 0L; + // Use AddToCart.newBuilder() to construct. + private AddToCart(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private AddToCart() { + userId_ = ""; + itemId_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private AddToCart( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + java.lang.String s = input.readStringRequireUtf8(); + + userId_ = s; + break; + } + case 18: + { + java.lang.String s = input.readStringRequireUtf8(); + + itemId_ = s; + break; + } + case 24: + { + quantity_ = input.readInt32(); + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_AddToCart_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_AddToCart_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart + .class, + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart + .Builder.class); + } + + public static final int USER_ID_FIELD_NUMBER = 1; + private volatile java.lang.Object userId_; + /** string user_id = 1; */ + public java.lang.String getUserId() { + java.lang.Object ref = userId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + userId_ = s; + return s; + } + } + /** string user_id = 1; */ + public com.google.protobuf.ByteString getUserIdBytes() { + java.lang.Object ref = userId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + userId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int ITEM_ID_FIELD_NUMBER = 2; + private volatile java.lang.Object itemId_; + /** string item_id = 2; */ + public java.lang.String getItemId() { + java.lang.Object ref = itemId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + itemId_ = s; + return s; + } + } + /** string item_id = 2; */ + public com.google.protobuf.ByteString getItemIdBytes() { + java.lang.Object ref = itemId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + itemId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int QUANTITY_FIELD_NUMBER = 3; + private int quantity_; + /** int32 quantity = 3; */ + public int getQuantity() { + return quantity_; + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!getUserIdBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, userId_); + } + if (!getItemIdBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 2, itemId_); + } + if (quantity_ != 0) { + output.writeInt32(3, quantity_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!getUserIdBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, userId_); + } + if (!getItemIdBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, itemId_); + } + if (quantity_ != 0) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(3, quantity_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart)) { + return super.equals(obj); + } + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart other = + (com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart) obj; + + if (!getUserId().equals(other.getUserId())) return false; + if (!getItemId().equals(other.getItemId())) return false; + if (getQuantity() != other.getQuantity()) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + USER_ID_FIELD_NUMBER; + hash = (53 * hash) + getUserId().hashCode(); + hash = (37 * hash) + ITEM_ID_FIELD_NUMBER; + hash = (53 * hash) + getItemId().hashCode(); + hash = (37 * hash) + QUANTITY_FIELD_NUMBER; + hash = (53 * hash) + getQuantity(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart + parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart + parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart + parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart + parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart + parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart + parseFrom(byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart + parseFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart + parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart + parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart + parseFrom(com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart + parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart + prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code com.ververica.statefun.examples.shoppingcart.AddToCart} */ + public static final class Builder + extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.shoppingcart.AddToCart) + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCartOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_AddToCart_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_AddToCart_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart + .class, + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart + .Builder.class); + } + + // Construct using + // com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + userId_ = ""; + + itemId_ = ""; + + quantity_ = 0; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_AddToCart_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart + getDefaultInstanceForType() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart + .getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart + build() { + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart result = + buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart + buildPartial() { + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart result = + new com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart( + this); + result.userId_ = userId_; + result.itemId_ = itemId_; + result.quantity_ = quantity_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other + instanceof + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart) { + return mergeFrom( + (com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart) + other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart other) { + if (other + == com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart + .getDefaultInstance()) return this; + if (!other.getUserId().isEmpty()) { + userId_ = other.userId_; + onChanged(); + } + if (!other.getItemId().isEmpty()) { + itemId_ = other.itemId_; + onChanged(); + } + if (other.getQuantity() != 0) { + setQuantity(other.getQuantity()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart + parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private java.lang.Object userId_ = ""; + /** string user_id = 1; */ + public java.lang.String getUserId() { + java.lang.Object ref = userId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + userId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string user_id = 1; */ + public com.google.protobuf.ByteString getUserIdBytes() { + java.lang.Object ref = userId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + userId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string user_id = 1; */ + public Builder setUserId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + userId_ = value; + onChanged(); + return this; + } + /** string user_id = 1; */ + public Builder clearUserId() { + + userId_ = getDefaultInstance().getUserId(); + onChanged(); + return this; + } + /** string user_id = 1; */ + public Builder setUserIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + userId_ = value; + onChanged(); + return this; + } + + private java.lang.Object itemId_ = ""; + /** string item_id = 2; */ + public java.lang.String getItemId() { + java.lang.Object ref = itemId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + itemId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string item_id = 2; */ + public com.google.protobuf.ByteString getItemIdBytes() { + java.lang.Object ref = itemId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + itemId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string item_id = 2; */ + public Builder setItemId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + itemId_ = value; + onChanged(); + return this; + } + /** string item_id = 2; */ + public Builder clearItemId() { + + itemId_ = getDefaultInstance().getItemId(); + onChanged(); + return this; + } + /** string item_id = 2; */ + public Builder setItemIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + itemId_ = value; + onChanged(); + return this; + } + + private int quantity_; + /** int32 quantity = 3; */ + public int getQuantity() { + return quantity_; + } + /** int32 quantity = 3; */ + public Builder setQuantity(int value) { + + quantity_ = value; + onChanged(); + return this; + } + /** int32 quantity = 3; */ + public Builder clearQuantity() { + + quantity_ = 0; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.shoppingcart.AddToCart) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.shoppingcart.AddToCart) + private static final com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .AddToCart + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart(); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public AddToCart parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new AddToCart(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.AddToCart + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + public interface ClearCartOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.shoppingcart.ClearCart) + com.google.protobuf.MessageOrBuilder { + + /** string user_id = 1; */ + java.lang.String getUserId(); + /** string user_id = 1; */ + com.google.protobuf.ByteString getUserIdBytes(); + } + /** Protobuf type {@code com.ververica.statefun.examples.shoppingcart.ClearCart} */ + public static final class ClearCart extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.shoppingcart.ClearCart) + ClearCartOrBuilder { + private static final long serialVersionUID = 0L; + // Use ClearCart.newBuilder() to construct. + private ClearCart(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private ClearCart() { + userId_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private ClearCart( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + java.lang.String s = input.readStringRequireUtf8(); + + userId_ = s; + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_ClearCart_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_ClearCart_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart + .class, + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart + .Builder.class); + } + + public static final int USER_ID_FIELD_NUMBER = 1; + private volatile java.lang.Object userId_; + /** string user_id = 1; */ + public java.lang.String getUserId() { + java.lang.Object ref = userId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + userId_ = s; + return s; + } + } + /** string user_id = 1; */ + public com.google.protobuf.ByteString getUserIdBytes() { + java.lang.Object ref = userId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + userId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!getUserIdBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, userId_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!getUserIdBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, userId_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart)) { + return super.equals(obj); + } + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart other = + (com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart) obj; + + if (!getUserId().equals(other.getUserId())) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + USER_ID_FIELD_NUMBER; + hash = (53 * hash) + getUserId().hashCode(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart + parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart + parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart + parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart + parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart + parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart + parseFrom(byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart + parseFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart + parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart + parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart + parseFrom(com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart + parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart + prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code com.ververica.statefun.examples.shoppingcart.ClearCart} */ + public static final class Builder + extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.shoppingcart.ClearCart) + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCartOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_ClearCart_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_ClearCart_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart + .class, + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart + .Builder.class); + } + + // Construct using + // com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + userId_ = ""; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_ClearCart_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart + getDefaultInstanceForType() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart + .getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart + build() { + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart result = + buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart + buildPartial() { + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart result = + new com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart( + this); + result.userId_ = userId_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other + instanceof + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart) { + return mergeFrom( + (com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart) + other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart other) { + if (other + == com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart + .getDefaultInstance()) return this; + if (!other.getUserId().isEmpty()) { + userId_ = other.userId_; + onChanged(); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart + parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private java.lang.Object userId_ = ""; + /** string user_id = 1; */ + public java.lang.String getUserId() { + java.lang.Object ref = userId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + userId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string user_id = 1; */ + public com.google.protobuf.ByteString getUserIdBytes() { + java.lang.Object ref = userId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + userId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string user_id = 1; */ + public Builder setUserId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + userId_ = value; + onChanged(); + return this; + } + /** string user_id = 1; */ + public Builder clearUserId() { + + userId_ = getDefaultInstance().getUserId(); + onChanged(); + return this; + } + /** string user_id = 1; */ + public Builder setUserIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + userId_ = value; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.shoppingcart.ClearCart) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.shoppingcart.ClearCart) + private static final com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ClearCart + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart(); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public ClearCart parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new ClearCart(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ClearCart + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + public interface CheckoutOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.shoppingcart.Checkout) + com.google.protobuf.MessageOrBuilder { + + /** string user_id = 1; */ + java.lang.String getUserId(); + /** string user_id = 1; */ + com.google.protobuf.ByteString getUserIdBytes(); + } + /** Protobuf type {@code com.ververica.statefun.examples.shoppingcart.Checkout} */ + public static final class Checkout extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.shoppingcart.Checkout) + CheckoutOrBuilder { + private static final long serialVersionUID = 0L; + // Use Checkout.newBuilder() to construct. + private Checkout(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private Checkout() { + userId_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private Checkout( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + java.lang.String s = input.readStringRequireUtf8(); + + userId_ = s; + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_Checkout_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_Checkout_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout + .class, + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout + .Builder.class); + } + + public static final int USER_ID_FIELD_NUMBER = 1; + private volatile java.lang.Object userId_; + /** string user_id = 1; */ + public java.lang.String getUserId() { + java.lang.Object ref = userId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + userId_ = s; + return s; + } + } + /** string user_id = 1; */ + public com.google.protobuf.ByteString getUserIdBytes() { + java.lang.Object ref = userId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + userId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!getUserIdBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, userId_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!getUserIdBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, userId_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout)) { + return super.equals(obj); + } + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout other = + (com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout) obj; + + if (!getUserId().equals(other.getUserId())) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + USER_ID_FIELD_NUMBER; + hash = (53 * hash) + getUserId().hashCode(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout + parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout + parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout + parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout + parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout + parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout + parseFrom(byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout + parseFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout + parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout + parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout + parseFrom(com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout + parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout + prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code com.ververica.statefun.examples.shoppingcart.Checkout} */ + public static final class Builder + extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.shoppingcart.Checkout) + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.CheckoutOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_Checkout_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_Checkout_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout + .class, + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout + .Builder.class); + } + + // Construct using + // com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + userId_ = ""; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_Checkout_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout + getDefaultInstanceForType() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout + .getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout + build() { + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout result = + buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout + buildPartial() { + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout result = + new com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout( + this); + result.userId_ = userId_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other + instanceof + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout) { + return mergeFrom( + (com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout) + other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout other) { + if (other + == com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout + .getDefaultInstance()) return this; + if (!other.getUserId().isEmpty()) { + userId_ = other.userId_; + onChanged(); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout + parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private java.lang.Object userId_ = ""; + /** string user_id = 1; */ + public java.lang.String getUserId() { + java.lang.Object ref = userId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + userId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string user_id = 1; */ + public com.google.protobuf.ByteString getUserIdBytes() { + java.lang.Object ref = userId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + userId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string user_id = 1; */ + public Builder setUserId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + userId_ = value; + onChanged(); + return this; + } + /** string user_id = 1; */ + public Builder clearUserId() { + + userId_ = getDefaultInstance().getUserId(); + onChanged(); + return this; + } + /** string user_id = 1; */ + public Builder setUserIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + userId_ = value; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.shoppingcart.Checkout) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.shoppingcart.Checkout) + private static final com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .Checkout + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout(); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public Checkout parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new Checkout(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Checkout + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + public interface ReceiptOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.shoppingcart.Receipt) + com.google.protobuf.MessageOrBuilder { + + /** string user_id = 1; */ + java.lang.String getUserId(); + /** string user_id = 1; */ + com.google.protobuf.ByteString getUserIdBytes(); + + /** string details = 2; */ + java.lang.String getDetails(); + /** string details = 2; */ + com.google.protobuf.ByteString getDetailsBytes(); + } + /** Protobuf type {@code com.ververica.statefun.examples.shoppingcart.Receipt} */ + public static final class Receipt extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.shoppingcart.Receipt) + ReceiptOrBuilder { + private static final long serialVersionUID = 0L; + // Use Receipt.newBuilder() to construct. + private Receipt(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private Receipt() { + userId_ = ""; + details_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private Receipt( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + java.lang.String s = input.readStringRequireUtf8(); + + userId_ = s; + break; + } + case 18: + { + java.lang.String s = input.readStringRequireUtf8(); + + details_ = s; + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_Receipt_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_Receipt_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt.class, + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt + .Builder.class); + } + + public static final int USER_ID_FIELD_NUMBER = 1; + private volatile java.lang.Object userId_; + /** string user_id = 1; */ + public java.lang.String getUserId() { + java.lang.Object ref = userId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + userId_ = s; + return s; + } + } + /** string user_id = 1; */ + public com.google.protobuf.ByteString getUserIdBytes() { + java.lang.Object ref = userId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + userId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int DETAILS_FIELD_NUMBER = 2; + private volatile java.lang.Object details_; + /** string details = 2; */ + public java.lang.String getDetails() { + java.lang.Object ref = details_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + details_ = s; + return s; + } + } + /** string details = 2; */ + public com.google.protobuf.ByteString getDetailsBytes() { + java.lang.Object ref = details_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + details_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!getUserIdBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, userId_); + } + if (!getDetailsBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 2, details_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!getUserIdBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, userId_); + } + if (!getDetailsBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, details_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt)) { + return super.equals(obj); + } + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt other = + (com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt) obj; + + if (!getUserId().equals(other.getUserId())) return false; + if (!getDetails().equals(other.getDetails())) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + USER_ID_FIELD_NUMBER; + hash = (53 * hash) + getUserId().hashCode(); + hash = (37 * hash) + DETAILS_FIELD_NUMBER; + hash = (53 * hash) + getDetails().hashCode(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt + parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt + parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt + parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt + parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt + parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt + parseFrom(byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt + parseFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt + parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt + parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt + parseFrom(com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt + parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code com.ververica.statefun.examples.shoppingcart.Receipt} */ + public static final class Builder + extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.shoppingcart.Receipt) + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ReceiptOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_Receipt_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_Receipt_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt + .class, + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt + .Builder.class); + } + + // Construct using + // com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + userId_ = ""; + + details_ = ""; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_Receipt_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt + getDefaultInstanceForType() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt + .getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt + build() { + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt result = + buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt + buildPartial() { + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt result = + new com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt( + this); + result.userId_ = userId_; + result.details_ = details_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other + instanceof + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt) { + return mergeFrom( + (com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt) + other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt other) { + if (other + == com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt + .getDefaultInstance()) return this; + if (!other.getUserId().isEmpty()) { + userId_ = other.userId_; + onChanged(); + } + if (!other.getDetails().isEmpty()) { + details_ = other.details_; + onChanged(); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt + parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private java.lang.Object userId_ = ""; + /** string user_id = 1; */ + public java.lang.String getUserId() { + java.lang.Object ref = userId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + userId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string user_id = 1; */ + public com.google.protobuf.ByteString getUserIdBytes() { + java.lang.Object ref = userId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + userId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string user_id = 1; */ + public Builder setUserId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + userId_ = value; + onChanged(); + return this; + } + /** string user_id = 1; */ + public Builder clearUserId() { + + userId_ = getDefaultInstance().getUserId(); + onChanged(); + return this; + } + /** string user_id = 1; */ + public Builder setUserIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + userId_ = value; + onChanged(); + return this; + } + + private java.lang.Object details_ = ""; + /** string details = 2; */ + public java.lang.String getDetails() { + java.lang.Object ref = details_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + details_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string details = 2; */ + public com.google.protobuf.ByteString getDetailsBytes() { + java.lang.Object ref = details_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + details_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string details = 2; */ + public Builder setDetails(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + details_ = value; + onChanged(); + return this; + } + /** string details = 2; */ + public Builder clearDetails() { + + details_ = getDefaultInstance().getDetails(); + onChanged(); + return this; + } + /** string details = 2; */ + public Builder setDetailsBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + details_ = value; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.shoppingcart.Receipt) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.shoppingcart.Receipt) + private static final com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .Receipt + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt(); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public Receipt parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new Receipt(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Receipt + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + public interface RestockItemOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.shoppingcart.RestockItem) + com.google.protobuf.MessageOrBuilder { + + /** string item_id = 1; */ + java.lang.String getItemId(); + /** string item_id = 1; */ + com.google.protobuf.ByteString getItemIdBytes(); + + /** int32 quantity = 2; */ + int getQuantity(); + } + /** Protobuf type {@code com.ververica.statefun.examples.shoppingcart.RestockItem} */ + public static final class RestockItem extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.shoppingcart.RestockItem) + RestockItemOrBuilder { + private static final long serialVersionUID = 0L; + // Use RestockItem.newBuilder() to construct. + private RestockItem(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private RestockItem() { + itemId_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private RestockItem( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + java.lang.String s = input.readStringRequireUtf8(); + + itemId_ = s; + break; + } + case 16: + { + quantity_ = input.readInt32(); + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_RestockItem_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_RestockItem_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RestockItem + .class, + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RestockItem + .Builder.class); + } + + public static final int ITEM_ID_FIELD_NUMBER = 1; + private volatile java.lang.Object itemId_; + /** string item_id = 1; */ + public java.lang.String getItemId() { + java.lang.Object ref = itemId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + itemId_ = s; + return s; + } + } + /** string item_id = 1; */ + public com.google.protobuf.ByteString getItemIdBytes() { + java.lang.Object ref = itemId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + itemId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int QUANTITY_FIELD_NUMBER = 2; + private int quantity_; + /** int32 quantity = 2; */ + public int getQuantity() { + return quantity_; + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!getItemIdBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, itemId_); + } + if (quantity_ != 0) { + output.writeInt32(2, quantity_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!getItemIdBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, itemId_); + } + if (quantity_ != 0) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(2, quantity_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RestockItem)) { + return super.equals(obj); + } + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RestockItem other = + (com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RestockItem) obj; + + if (!getItemId().equals(other.getItemId())) return false; + if (getQuantity() != other.getQuantity()) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + ITEM_ID_FIELD_NUMBER; + hash = (53 * hash) + getItemId().hashCode(); + hash = (37 * hash) + QUANTITY_FIELD_NUMBER; + hash = (53 * hash) + getQuantity(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .RestockItem + parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .RestockItem + parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .RestockItem + parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .RestockItem + parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .RestockItem + parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .RestockItem + parseFrom(byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .RestockItem + parseFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .RestockItem + parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .RestockItem + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .RestockItem + parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .RestockItem + parseFrom(com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .RestockItem + parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RestockItem + prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code com.ververica.statefun.examples.shoppingcart.RestockItem} */ + public static final class Builder + extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.shoppingcart.RestockItem) + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .RestockItemOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_RestockItem_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_RestockItem_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RestockItem + .class, + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RestockItem + .Builder.class); + } + + // Construct using + // com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RestockItem.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + itemId_ = ""; + + quantity_ = 0; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_RestockItem_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RestockItem + getDefaultInstanceForType() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RestockItem + .getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RestockItem + build() { + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RestockItem result = + buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RestockItem + buildPartial() { + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RestockItem result = + new com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RestockItem( + this); + result.itemId_ = itemId_; + result.quantity_ = quantity_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other + instanceof + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RestockItem) { + return mergeFrom( + (com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RestockItem) + other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RestockItem + other) { + if (other + == com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RestockItem + .getDefaultInstance()) return this; + if (!other.getItemId().isEmpty()) { + itemId_ = other.itemId_; + onChanged(); + } + if (other.getQuantity() != 0) { + setQuantity(other.getQuantity()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RestockItem + parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RestockItem) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private java.lang.Object itemId_ = ""; + /** string item_id = 1; */ + public java.lang.String getItemId() { + java.lang.Object ref = itemId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + itemId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string item_id = 1; */ + public com.google.protobuf.ByteString getItemIdBytes() { + java.lang.Object ref = itemId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + itemId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string item_id = 1; */ + public Builder setItemId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + itemId_ = value; + onChanged(); + return this; + } + /** string item_id = 1; */ + public Builder clearItemId() { + + itemId_ = getDefaultInstance().getItemId(); + onChanged(); + return this; + } + /** string item_id = 1; */ + public Builder setItemIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + itemId_ = value; + onChanged(); + return this; + } + + private int quantity_; + /** int32 quantity = 2; */ + public int getQuantity() { + return quantity_; + } + /** int32 quantity = 2; */ + public Builder setQuantity(int value) { + + quantity_ = value; + onChanged(); + return this; + } + /** int32 quantity = 2; */ + public Builder clearQuantity() { + + quantity_ = 0; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.shoppingcart.RestockItem) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.shoppingcart.RestockItem) + private static final com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .RestockItem + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RestockItem(); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .RestockItem + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public RestockItem parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new RestockItem(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RestockItem + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + public interface RequestItemOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.shoppingcart.RequestItem) + com.google.protobuf.MessageOrBuilder { + + /** int32 quantity = 1; */ + int getQuantity(); + } + /** Protobuf type {@code com.ververica.statefun.examples.shoppingcart.RequestItem} */ + public static final class RequestItem extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.shoppingcart.RequestItem) + RequestItemOrBuilder { + private static final long serialVersionUID = 0L; + // Use RequestItem.newBuilder() to construct. + private RequestItem(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private RequestItem() {} + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private RequestItem( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 8: + { + quantity_ = input.readInt32(); + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_RequestItem_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_RequestItem_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RequestItem + .class, + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RequestItem + .Builder.class); + } + + public static final int QUANTITY_FIELD_NUMBER = 1; + private int quantity_; + /** int32 quantity = 1; */ + public int getQuantity() { + return quantity_; + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (quantity_ != 0) { + output.writeInt32(1, quantity_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (quantity_ != 0) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(1, quantity_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RequestItem)) { + return super.equals(obj); + } + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RequestItem other = + (com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RequestItem) obj; + + if (getQuantity() != other.getQuantity()) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + QUANTITY_FIELD_NUMBER; + hash = (53 * hash) + getQuantity(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .RequestItem + parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .RequestItem + parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .RequestItem + parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .RequestItem + parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .RequestItem + parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .RequestItem + parseFrom(byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .RequestItem + parseFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .RequestItem + parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .RequestItem + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .RequestItem + parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .RequestItem + parseFrom(com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .RequestItem + parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RequestItem + prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code com.ververica.statefun.examples.shoppingcart.RequestItem} */ + public static final class Builder + extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.shoppingcart.RequestItem) + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .RequestItemOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_RequestItem_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_RequestItem_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RequestItem + .class, + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RequestItem + .Builder.class); + } + + // Construct using + // com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RequestItem.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + quantity_ = 0; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_RequestItem_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RequestItem + getDefaultInstanceForType() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RequestItem + .getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RequestItem + build() { + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RequestItem result = + buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RequestItem + buildPartial() { + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RequestItem result = + new com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RequestItem( + this); + result.quantity_ = quantity_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other + instanceof + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RequestItem) { + return mergeFrom( + (com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RequestItem) + other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RequestItem + other) { + if (other + == com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RequestItem + .getDefaultInstance()) return this; + if (other.getQuantity() != 0) { + setQuantity(other.getQuantity()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RequestItem + parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RequestItem) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int quantity_; + /** int32 quantity = 1; */ + public int getQuantity() { + return quantity_; + } + /** int32 quantity = 1; */ + public Builder setQuantity(int value) { + + quantity_ = value; + onChanged(); + return this; + } + /** int32 quantity = 1; */ + public Builder clearQuantity() { + + quantity_ = 0; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.shoppingcart.RequestItem) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.shoppingcart.RequestItem) + private static final com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .RequestItem + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RequestItem(); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .RequestItem + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public RequestItem parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new RequestItem(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.RequestItem + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + public interface ItemAvailabilityOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.shoppingcart.ItemAvailability) + com.google.protobuf.MessageOrBuilder { + + /** + * .com.ververica.statefun.examples.shoppingcart.ItemAvailability.Status status = 1; + * + */ + int getStatusValue(); + /** + * .com.ververica.statefun.examples.shoppingcart.ItemAvailability.Status status = 1; + * + */ + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ItemAvailability.Status + getStatus(); + + /** int32 quantity = 2; */ + int getQuantity(); + } + /** Protobuf type {@code com.ververica.statefun.examples.shoppingcart.ItemAvailability} */ + public static final class ItemAvailability extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.shoppingcart.ItemAvailability) + ItemAvailabilityOrBuilder { + private static final long serialVersionUID = 0L; + // Use ItemAvailability.newBuilder() to construct. + private ItemAvailability(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private ItemAvailability() { + status_ = 0; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private ItemAvailability( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 8: + { + int rawValue = input.readEnum(); + + status_ = rawValue; + break; + } + case 16: + { + quantity_ = input.readInt32(); + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_ItemAvailability_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_ItemAvailability_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability.class, + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability.Builder.class); + } + + /** + * Protobuf enum {@code com.ververica.statefun.examples.shoppingcart.ItemAvailability.Status} + */ + public enum Status implements com.google.protobuf.ProtocolMessageEnum { + /** INSTOCK = 0; */ + INSTOCK(0), + /** OUTOFSTOCK = 1; */ + OUTOFSTOCK(1), + UNRECOGNIZED(-1), + ; + + /** INSTOCK = 0; */ + public static final int INSTOCK_VALUE = 0; + /** OUTOFSTOCK = 1; */ + public static final int OUTOFSTOCK_VALUE = 1; + + public final int getNumber() { + if (this == UNRECOGNIZED) { + throw new java.lang.IllegalArgumentException( + "Can't get the number of an unknown enum value."); + } + return value; + } + + /** @deprecated Use {@link #forNumber(int)} instead. */ + @java.lang.Deprecated + public static Status valueOf(int value) { + return forNumber(value); + } + + public static Status forNumber(int value) { + switch (value) { + case 0: + return INSTOCK; + case 1: + return OUTOFSTOCK; + default: + return null; + } + } + + public static com.google.protobuf.Internal.EnumLiteMap internalGetValueMap() { + return internalValueMap; + } + + private static final com.google.protobuf.Internal.EnumLiteMap internalValueMap = + new com.google.protobuf.Internal.EnumLiteMap() { + public Status findValueByNumber(int number) { + return Status.forNumber(number); + } + }; + + public final com.google.protobuf.Descriptors.EnumValueDescriptor getValueDescriptor() { + return getDescriptor().getValues().get(ordinal()); + } + + public final com.google.protobuf.Descriptors.EnumDescriptor getDescriptorForType() { + return getDescriptor(); + } + + public static final com.google.protobuf.Descriptors.EnumDescriptor getDescriptor() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability.getDescriptor() + .getEnumTypes() + .get(0); + } + + private static final Status[] VALUES = values(); + + public static Status valueOf(com.google.protobuf.Descriptors.EnumValueDescriptor desc) { + if (desc.getType() != getDescriptor()) { + throw new java.lang.IllegalArgumentException("EnumValueDescriptor is not for this type."); + } + if (desc.getIndex() == -1) { + return UNRECOGNIZED; + } + return VALUES[desc.getIndex()]; + } + + private final int value; + + private Status(int value) { + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:com.ververica.statefun.examples.shoppingcart.ItemAvailability.Status) + } + + public static final int STATUS_FIELD_NUMBER = 1; + private int status_; + /** + * .com.ververica.statefun.examples.shoppingcart.ItemAvailability.Status status = 1; + * + */ + public int getStatusValue() { + return status_; + } + /** + * .com.ververica.statefun.examples.shoppingcart.ItemAvailability.Status status = 1; + * + */ + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ItemAvailability + .Status + getStatus() { + @SuppressWarnings("deprecation") + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ItemAvailability + .Status + result = + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability.Status.valueOf(status_); + return result == null + ? com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ItemAvailability + .Status.UNRECOGNIZED + : result; + } + + public static final int QUANTITY_FIELD_NUMBER = 2; + private int quantity_; + /** int32 quantity = 2; */ + public int getQuantity() { + return quantity_; + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (status_ + != com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability.Status.INSTOCK + .getNumber()) { + output.writeEnum(1, status_); + } + if (quantity_ != 0) { + output.writeInt32(2, quantity_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (status_ + != com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability.Status.INSTOCK + .getNumber()) { + size += com.google.protobuf.CodedOutputStream.computeEnumSize(1, status_); + } + if (quantity_ != 0) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(2, quantity_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability)) { + return super.equals(obj); + } + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ItemAvailability + other = + (com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability) + obj; + + if (status_ != other.status_) return false; + if (getQuantity() != other.getQuantity()) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + STATUS_FIELD_NUMBER; + hash = (53 * hash) + status_; + hash = (37 * hash) + QUANTITY_FIELD_NUMBER; + hash = (53 * hash) + getQuantity(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability + parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability + parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability + parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability + parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability + parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability + parseFrom(byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability + parseFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability + parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability + parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability + parseFrom(com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability + parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ItemAvailability + prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code com.ververica.statefun.examples.shoppingcart.ItemAvailability} */ + public static final class Builder + extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.shoppingcart.ItemAvailability) + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailabilityOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_ItemAvailability_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_ItemAvailability_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability.class, + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability.Builder.class); + } + + // Construct using + // com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ItemAvailability.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + status_ = 0; + + quantity_ = 0; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_ItemAvailability_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability + getDefaultInstanceForType() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability.getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability + build() { + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ItemAvailability + result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability + buildPartial() { + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ItemAvailability + result = + new com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability(this); + result.status_ = status_; + result.quantity_ = quantity_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other + instanceof + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability) { + return mergeFrom( + (com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability) + other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ItemAvailability + other) { + if (other + == com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability.getDefaultInstance()) return this; + if (other.status_ != 0) { + setStatusValue(other.getStatusValue()); + } + if (other.getQuantity() != 0) { + setQuantity(other.getQuantity()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ItemAvailability + parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int status_ = 0; + /** + * .com.ververica.statefun.examples.shoppingcart.ItemAvailability.Status status = 1; + * + */ + public int getStatusValue() { + return status_; + } + /** + * .com.ververica.statefun.examples.shoppingcart.ItemAvailability.Status status = 1; + * + */ + public Builder setStatusValue(int value) { + status_ = value; + onChanged(); + return this; + } + /** + * .com.ververica.statefun.examples.shoppingcart.ItemAvailability.Status status = 1; + * + */ + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability.Status + getStatus() { + @SuppressWarnings("deprecation") + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ItemAvailability + .Status + result = + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability.Status.valueOf(status_); + return result == null + ? com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability.Status.UNRECOGNIZED + : result; + } + /** + * .com.ververica.statefun.examples.shoppingcart.ItemAvailability.Status status = 1; + * + */ + public Builder setStatus( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ItemAvailability + .Status + value) { + if (value == null) { + throw new NullPointerException(); + } + + status_ = value.getNumber(); + onChanged(); + return this; + } + /** + * .com.ververica.statefun.examples.shoppingcart.ItemAvailability.Status status = 1; + * + */ + public Builder clearStatus() { + + status_ = 0; + onChanged(); + return this; + } + + private int quantity_; + /** int32 quantity = 2; */ + public int getQuantity() { + return quantity_; + } + /** int32 quantity = 2; */ + public Builder setQuantity(int value) { + + quantity_ = value; + onChanged(); + return this; + } + /** int32 quantity = 2; */ + public Builder clearQuantity() { + + quantity_ = 0; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.shoppingcart.ItemAvailability) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.shoppingcart.ItemAvailability) + private static final com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability(); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .ItemAvailability + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public ItemAvailability parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new ItemAvailability(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.ItemAvailability + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + public interface BasketOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.examples.shoppingcart.Basket) + com.google.protobuf.MessageOrBuilder { + + /** map<string, int32> items = 1; */ + int getItemsCount(); + /** map<string, int32> items = 1; */ + boolean containsItems(java.lang.String key); + /** Use {@link #getItemsMap()} instead. */ + @java.lang.Deprecated + java.util.Map getItems(); + /** map<string, int32> items = 1; */ + java.util.Map getItemsMap(); + /** map<string, int32> items = 1; */ + int getItemsOrDefault(java.lang.String key, int defaultValue); + /** map<string, int32> items = 1; */ + int getItemsOrThrow(java.lang.String key); + } + /** Protobuf type {@code com.ververica.statefun.examples.shoppingcart.Basket} */ + public static final class Basket extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.examples.shoppingcart.Basket) + BasketOrBuilder { + private static final long serialVersionUID = 0L; + // Use Basket.newBuilder() to construct. + private Basket(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private Basket() {} + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private Basket( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + if (!((mutable_bitField0_ & 0x00000001) != 0)) { + items_ = + com.google.protobuf.MapField.newMapField( + ItemsDefaultEntryHolder.defaultEntry); + mutable_bitField0_ |= 0x00000001; + } + com.google.protobuf.MapEntry items__ = + input.readMessage( + ItemsDefaultEntryHolder.defaultEntry.getParserForType(), extensionRegistry); + items_.getMutableMap().put(items__.getKey(), items__.getValue()); + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_Basket_descriptor; + } + + @SuppressWarnings({"rawtypes"}) + @java.lang.Override + protected com.google.protobuf.MapField internalGetMapField(int number) { + switch (number) { + case 1: + return internalGetItems(); + default: + throw new RuntimeException("Invalid map field number: " + number); + } + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_Basket_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket.class, + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket.Builder + .class); + } + + public static final int ITEMS_FIELD_NUMBER = 1; + + private static final class ItemsDefaultEntryHolder { + static final com.google.protobuf.MapEntry defaultEntry = + com.google.protobuf.MapEntry.newDefaultInstance( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_Basket_ItemsEntry_descriptor, + com.google.protobuf.WireFormat.FieldType.STRING, + "", + com.google.protobuf.WireFormat.FieldType.INT32, + 0); + } + + private com.google.protobuf.MapField items_; + + private com.google.protobuf.MapField internalGetItems() { + if (items_ == null) { + return com.google.protobuf.MapField.emptyMapField(ItemsDefaultEntryHolder.defaultEntry); + } + return items_; + } + + public int getItemsCount() { + return internalGetItems().getMap().size(); + } + /** map<string, int32> items = 1; */ + public boolean containsItems(java.lang.String key) { + if (key == null) { + throw new java.lang.NullPointerException(); + } + return internalGetItems().getMap().containsKey(key); + } + /** Use {@link #getItemsMap()} instead. */ + @java.lang.Deprecated + public java.util.Map getItems() { + return getItemsMap(); + } + /** map<string, int32> items = 1; */ + public java.util.Map getItemsMap() { + return internalGetItems().getMap(); + } + /** map<string, int32> items = 1; */ + public int getItemsOrDefault(java.lang.String key, int defaultValue) { + if (key == null) { + throw new java.lang.NullPointerException(); + } + java.util.Map map = internalGetItems().getMap(); + return map.containsKey(key) ? map.get(key) : defaultValue; + } + /** map<string, int32> items = 1; */ + public int getItemsOrThrow(java.lang.String key) { + if (key == null) { + throw new java.lang.NullPointerException(); + } + java.util.Map map = internalGetItems().getMap(); + if (!map.containsKey(key)) { + throw new java.lang.IllegalArgumentException(); + } + return map.get(key); + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + com.google.protobuf.GeneratedMessageV3.serializeStringMapTo( + output, internalGetItems(), ItemsDefaultEntryHolder.defaultEntry, 1); + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + for (java.util.Map.Entry entry : + internalGetItems().getMap().entrySet()) { + com.google.protobuf.MapEntry items__ = + ItemsDefaultEntryHolder.defaultEntry + .newBuilderForType() + .setKey(entry.getKey()) + .setValue(entry.getValue()) + .build(); + size += com.google.protobuf.CodedOutputStream.computeMessageSize(1, items__); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket)) { + return super.equals(obj); + } + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket other = + (com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket) obj; + + if (!internalGetItems().equals(other.internalGetItems())) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (!internalGetItems().getMap().isEmpty()) { + hash = (37 * hash) + ITEMS_FIELD_NUMBER; + hash = (53 * hash) + internalGetItems().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket + parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket + parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket + parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket + parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket + parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket + parseFrom(byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket + parseFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket + parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket + parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket + parseFrom(com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket + parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code com.ververica.statefun.examples.shoppingcart.Basket} */ + public static final class Builder + extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.examples.shoppingcart.Basket) + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.BasketOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_Basket_descriptor; + } + + @SuppressWarnings({"rawtypes"}) + protected com.google.protobuf.MapField internalGetMapField(int number) { + switch (number) { + case 1: + return internalGetItems(); + default: + throw new RuntimeException("Invalid map field number: " + number); + } + } + + @SuppressWarnings({"rawtypes"}) + protected com.google.protobuf.MapField internalGetMutableMapField(int number) { + switch (number) { + case 1: + return internalGetMutableItems(); + default: + throw new RuntimeException("Invalid map field number: " + number); + } + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_Basket_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket + .class, + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket + .Builder.class); + } + + // Construct using + // com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + internalGetMutableItems().clear(); + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .internal_static_com_ververica_statefun_examples_shoppingcart_Basket_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket + getDefaultInstanceForType() { + return com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket + .getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket + build() { + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket result = + buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket + buildPartial() { + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket result = + new com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket( + this); + int from_bitField0_ = bitField0_; + result.items_ = internalGetItems(); + result.items_.makeImmutable(); + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other + instanceof + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket) { + return mergeFrom( + (com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket) + other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket other) { + if (other + == com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket + .getDefaultInstance()) return this; + internalGetMutableItems().mergeFrom(other.internalGetItems()); + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket + parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int bitField0_; + + private com.google.protobuf.MapField items_; + + private com.google.protobuf.MapField internalGetItems() { + if (items_ == null) { + return com.google.protobuf.MapField.emptyMapField(ItemsDefaultEntryHolder.defaultEntry); + } + return items_; + } + + private com.google.protobuf.MapField + internalGetMutableItems() { + onChanged(); + ; + if (items_ == null) { + items_ = com.google.protobuf.MapField.newMapField(ItemsDefaultEntryHolder.defaultEntry); + } + if (!items_.isMutable()) { + items_ = items_.copy(); + } + return items_; + } + + public int getItemsCount() { + return internalGetItems().getMap().size(); + } + /** map<string, int32> items = 1; */ + public boolean containsItems(java.lang.String key) { + if (key == null) { + throw new java.lang.NullPointerException(); + } + return internalGetItems().getMap().containsKey(key); + } + /** Use {@link #getItemsMap()} instead. */ + @java.lang.Deprecated + public java.util.Map getItems() { + return getItemsMap(); + } + /** map<string, int32> items = 1; */ + public java.util.Map getItemsMap() { + return internalGetItems().getMap(); + } + /** map<string, int32> items = 1; */ + public int getItemsOrDefault(java.lang.String key, int defaultValue) { + if (key == null) { + throw new java.lang.NullPointerException(); + } + java.util.Map map = internalGetItems().getMap(); + return map.containsKey(key) ? map.get(key) : defaultValue; + } + /** map<string, int32> items = 1; */ + public int getItemsOrThrow(java.lang.String key) { + if (key == null) { + throw new java.lang.NullPointerException(); + } + java.util.Map map = internalGetItems().getMap(); + if (!map.containsKey(key)) { + throw new java.lang.IllegalArgumentException(); + } + return map.get(key); + } + + public Builder clearItems() { + internalGetMutableItems().getMutableMap().clear(); + return this; + } + /** map<string, int32> items = 1; */ + public Builder removeItems(java.lang.String key) { + if (key == null) { + throw new java.lang.NullPointerException(); + } + internalGetMutableItems().getMutableMap().remove(key); + return this; + } + /** Use alternate mutation accessors instead. */ + @java.lang.Deprecated + public java.util.Map getMutableItems() { + return internalGetMutableItems().getMutableMap(); + } + /** map<string, int32> items = 1; */ + public Builder putItems(java.lang.String key, int value) { + if (key == null) { + throw new java.lang.NullPointerException(); + } + + internalGetMutableItems().getMutableMap().put(key, value); + return this; + } + /** map<string, int32> items = 1; */ + public Builder putAllItems(java.util.Map values) { + internalGetMutableItems().getMutableMap().putAll(values); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.examples.shoppingcart.Basket) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.examples.shoppingcart.Basket) + private static final com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages + .Basket + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket(); + } + + public static com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public Basket parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new Basket(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.examples.shoppingcart.generated.ProtobufMessages.Basket + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_shoppingcart_AddToCart_descriptor; + private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_shoppingcart_AddToCart_fieldAccessorTable; + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_shoppingcart_ClearCart_descriptor; + private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_shoppingcart_ClearCart_fieldAccessorTable; + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_shoppingcart_Checkout_descriptor; + private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_shoppingcart_Checkout_fieldAccessorTable; + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_shoppingcart_Receipt_descriptor; + private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_shoppingcart_Receipt_fieldAccessorTable; + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_shoppingcart_RestockItem_descriptor; + private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_shoppingcart_RestockItem_fieldAccessorTable; + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_shoppingcart_RequestItem_descriptor; + private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_shoppingcart_RequestItem_fieldAccessorTable; + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_shoppingcart_ItemAvailability_descriptor; + private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_shoppingcart_ItemAvailability_fieldAccessorTable; + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_shoppingcart_Basket_descriptor; + private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_shoppingcart_Basket_fieldAccessorTable; + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_examples_shoppingcart_Basket_ItemsEntry_descriptor; + private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_examples_shoppingcart_Basket_ItemsEntry_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { + return descriptor; + } + + private static com.google.protobuf.Descriptors.FileDescriptor descriptor; + + static { + java.lang.String[] descriptorData = { + "\n$src/main/protobuf/shoppingcart.proto\022," + + "com.ververica.statefun.examples.shopping" + + "cart\"?\n\tAddToCart\022\017\n\007user_id\030\001 \001(\t\022\017\n\007it" + + "em_id\030\002 \001(\t\022\020\n\010quantity\030\003 \001(\005\"\034\n\tClearCa" + + "rt\022\017\n\007user_id\030\001 \001(\t\"\033\n\010Checkout\022\017\n\007user_" + + "id\030\001 \001(\t\"+\n\007Receipt\022\017\n\007user_id\030\001 \001(\t\022\017\n\007" + + "details\030\002 \001(\t\"0\n\013RestockItem\022\017\n\007item_id\030" + + "\001 \001(\t\022\020\n\010quantity\030\002 \001(\005\"\037\n\013RequestItem\022\020" + + "\n\010quantity\030\001 \001(\005\"\242\001\n\020ItemAvailability\022U\n" + + "\006status\030\001 \001(\0162E.com.ververica.statefun.e" + + "xamples.shoppingcart.ItemAvailability.St" + + "atus\022\020\n\010quantity\030\002 \001(\005\"%\n\006Status\022\013\n\007INST" + + "OCK\020\000\022\016\n\nOUTOFSTOCK\020\001\"\206\001\n\006Basket\022N\n\005item" + + "s\030\001 \003(\0132?.com.ververica.statefun.example" + + "s.shoppingcart.Basket.ItemsEntry\032,\n\nItem" + + "sEntry\022\013\n\003key\030\001 \001(\t\022\r\n\005value\030\002 \001(\005:\0028\001BL" + + "\n6com.ververica.statefun.examples.shoppi" + + "ngcart.generatedB\020ProtobufMessagesP\000b\006pr" + + "oto3" + }; + com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = + new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { + public com.google.protobuf.ExtensionRegistry assignDescriptors( + com.google.protobuf.Descriptors.FileDescriptor root) { + descriptor = root; + return null; + } + }; + com.google.protobuf.Descriptors.FileDescriptor.internalBuildGeneratedFileFrom( + descriptorData, new com.google.protobuf.Descriptors.FileDescriptor[] {}, assigner); + internal_static_com_ververica_statefun_examples_shoppingcart_AddToCart_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_com_ververica_statefun_examples_shoppingcart_AddToCart_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_shoppingcart_AddToCart_descriptor, + new java.lang.String[] { + "UserId", "ItemId", "Quantity", + }); + internal_static_com_ververica_statefun_examples_shoppingcart_ClearCart_descriptor = + getDescriptor().getMessageTypes().get(1); + internal_static_com_ververica_statefun_examples_shoppingcart_ClearCart_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_shoppingcart_ClearCart_descriptor, + new java.lang.String[] { + "UserId", + }); + internal_static_com_ververica_statefun_examples_shoppingcart_Checkout_descriptor = + getDescriptor().getMessageTypes().get(2); + internal_static_com_ververica_statefun_examples_shoppingcart_Checkout_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_shoppingcart_Checkout_descriptor, + new java.lang.String[] { + "UserId", + }); + internal_static_com_ververica_statefun_examples_shoppingcart_Receipt_descriptor = + getDescriptor().getMessageTypes().get(3); + internal_static_com_ververica_statefun_examples_shoppingcart_Receipt_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_shoppingcart_Receipt_descriptor, + new java.lang.String[] { + "UserId", "Details", + }); + internal_static_com_ververica_statefun_examples_shoppingcart_RestockItem_descriptor = + getDescriptor().getMessageTypes().get(4); + internal_static_com_ververica_statefun_examples_shoppingcart_RestockItem_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_shoppingcart_RestockItem_descriptor, + new java.lang.String[] { + "ItemId", "Quantity", + }); + internal_static_com_ververica_statefun_examples_shoppingcart_RequestItem_descriptor = + getDescriptor().getMessageTypes().get(5); + internal_static_com_ververica_statefun_examples_shoppingcart_RequestItem_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_shoppingcart_RequestItem_descriptor, + new java.lang.String[] { + "Quantity", + }); + internal_static_com_ververica_statefun_examples_shoppingcart_ItemAvailability_descriptor = + getDescriptor().getMessageTypes().get(6); + internal_static_com_ververica_statefun_examples_shoppingcart_ItemAvailability_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_shoppingcart_ItemAvailability_descriptor, + new java.lang.String[] { + "Status", "Quantity", + }); + internal_static_com_ververica_statefun_examples_shoppingcart_Basket_descriptor = + getDescriptor().getMessageTypes().get(7); + internal_static_com_ververica_statefun_examples_shoppingcart_Basket_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_shoppingcart_Basket_descriptor, + new java.lang.String[] { + "Items", + }); + internal_static_com_ververica_statefun_examples_shoppingcart_Basket_ItemsEntry_descriptor = + internal_static_com_ververica_statefun_examples_shoppingcart_Basket_descriptor + .getNestedTypes() + .get(0); + internal_static_com_ververica_statefun_examples_shoppingcart_Basket_ItemsEntry_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_examples_shoppingcart_Basket_ItemsEntry_descriptor, + new java.lang.String[] { + "Key", "Value", + }); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/stateful-functions-examples/stateful-functions-shopping-cart-example/src/main/protobuf/shoppingcart.proto b/stateful-functions-examples/stateful-functions-shopping-cart-example/src/main/protobuf/shoppingcart.proto new file mode 100644 index 00000000..f623fd8f --- /dev/null +++ b/stateful-functions-examples/stateful-functions-shopping-cart-example/src/main/protobuf/shoppingcart.proto @@ -0,0 +1,77 @@ +// +// Copyright 2019 Ververica GmbH. +// +// 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"; + +package com.ververica.statefun.examples.shoppingcart; + +option java_package = "com.ververica.statefun.examples.shoppingcart.generated"; +option java_outer_classname = "ProtobufMessages"; +option java_multiple_files = false; + +// --------------------------------------------------------------------- +// External messages +// --------------------------------------------------------------------- + +message AddToCart { + string user_id = 1; + string item_id = 2; + int32 quantity = 3; +} + +message ClearCart { + string user_id = 1; +} + +message Checkout { + string user_id = 1; +} + +message Receipt { + string user_id = 1; + string details = 2; +} + +message RestockItem { + string item_id = 1; + int32 quantity = 2; +} + +// --------------------------------------------------------------------- +// Internal messages +// --------------------------------------------------------------------- + +message RequestItem { + int32 quantity = 1; +} + +message ItemAvailability { + enum Status { + INSTOCK = 0; + OUTOFSTOCK = 1; + } + + Status status = 1; + int32 quantity = 2; +} + +// --------------------------------------------------------------------- +// State +// --------------------------------------------------------------------- + +message Basket { + map items = 1; +} \ No newline at end of file diff --git a/stateful-functions-examples/stateful-functions-shopping-cart-example/src/main/resources/META-INF/services/com.ververica.statefun.sdk.spi.StatefulFunctionModule b/stateful-functions-examples/stateful-functions-shopping-cart-example/src/main/resources/META-INF/services/com.ververica.statefun.sdk.spi.StatefulFunctionModule new file mode 100644 index 00000000..fb5a8a66 --- /dev/null +++ b/stateful-functions-examples/stateful-functions-shopping-cart-example/src/main/resources/META-INF/services/com.ververica.statefun.sdk.spi.StatefulFunctionModule @@ -0,0 +1,17 @@ +# +# Copyright 2019 Ververica GmbH. +# +# 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. +# + +com.ververica.statefun.examples.shoppingcart.ShoppingCartModule \ No newline at end of file diff --git a/stateful-functions-flink/pom.xml b/stateful-functions-flink/pom.xml new file mode 100644 index 00000000..65e7298d --- /dev/null +++ b/stateful-functions-flink/pom.xml @@ -0,0 +1,112 @@ + + + + + + stateful-functions-parent + com.ververica + 1.0-SNAPSHOT + .. + + + 4.0.0 + stateful-functions-flink + pom + + + stateful-functions-flink-launcher + stateful-functions-flink-io + stateful-functions-flink-io-bundle + stateful-functions-flink-core + stateful-functions-flink-distribution + stateful-functions-flink-harness + + + + 2.11 + 3.0.2 + 1.9.0 + 1.21 + 3.8.0 + 1.3.9 + + + + + + + + com.google.code.findbugs + jsr305 + ${jsr305-version} + + + + + org.slf4j + slf4j-api + 1.7.7 + + + org.slf4j + slf4j-log4j12 + 1.7.7 + + + log4j + log4j + 1.2.17 + + + + + com.google.protobuf + protobuf-java + ${protobuf-java.version} + + + + + org.openjdk.jmh + jmh-core + ${jmh.version} + + + + org.apache.commons + commons-math3 + + + + + org.openjdk.jmh + jmh-generator-annprocess + ${jmh.version} + + + + + org.xerial.snappy + snappy-java + 1.1.4 + + + + + diff --git a/stateful-functions-flink/stateful-functions-flink-core/pom.xml b/stateful-functions-flink/stateful-functions-flink-core/pom.xml new file mode 100644 index 00000000..6d58538e --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/pom.xml @@ -0,0 +1,98 @@ + + + + 4.0.0 + + + com.ververica + stateful-functions-flink + 1.0-SNAPSHOT + .. + + + stateful-functions-flink-core + + + + + + com.ververica + stateful-functions-sdk + ${project.version} + + + com.ververica + stateful-functions-flink-io + ${project.version} + + + + + org.apache.flink + flink-streaming-java_2.11 + ${flink.version} + + + + + com.google.protobuf + protobuf-java + + + + + junit + junit + 4.12 + test + + + org.hamcrest + hamcrest-all + 1.3 + test + + + org.apache.flink + flink-core + ${flink.version} + test-jar + test + + + org.apache.flink + flink-test-utils-junit + ${flink.version} + test + + + + + org.openjdk.jmh + jmh-core + test + + + org.openjdk.jmh + jmh-generator-annprocess + test + + + + diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/ModuleSpecs.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/ModuleSpecs.java new file mode 100644 index 00000000..4806fc53 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/ModuleSpecs.java @@ -0,0 +1,135 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core; + +import com.ververica.statefun.flink.core.ModuleSpecs.ModuleSpec; +import java.io.File; +import java.io.IOException; +import java.io.Serializable; +import java.net.URI; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; +import java.util.TreeSet; + +public class ModuleSpecs implements Iterable, Serializable { + + private static final long serialVersionUID = 1L; + private final List specs; + + private ModuleSpecs(List specs) { + this.specs = Objects.requireNonNull(specs); + } + + public static ModuleSpecs fromPath(String rootDirectory) throws IOException { + Objects.requireNonNull(rootDirectory); + + List loadableModules = discoverLoadableArtifacts(rootDirectory); + return new ModuleSpecs(loadableModules); + } + + public static ModuleSpecs fromCollection(ModuleSpec... moduleSpecs) { + List loadableModules = Arrays.asList(moduleSpecs); + return new ModuleSpecs(loadableModules); + } + + /** Scans the given directory and looks for a List of artifacts ( */ + private static List discoverLoadableArtifacts(String rootDirectory) + throws IOException { + File parent = new File(rootDirectory); + if (!parent.exists()) { + throw new IllegalArgumentException(rootDirectory + " does not exists."); + } + if (!parent.isDirectory()) { + throw new RuntimeException(rootDirectory + " is not a directory."); + } + List loadableFunctions = new ArrayList<>(); + for (File subDirectory : nullToEmpty(parent.listFiles())) { + if (subDirectory.isDirectory()) { + ModuleSpec loadableFunction = findLoadableModuleArtifacts(subDirectory.getAbsoluteFile()); + loadableFunctions.add(loadableFunction); + } + } + return loadableFunctions; + } + + private static ModuleSpec findLoadableModuleArtifacts(File subDirectory) throws IOException { + ModuleSpec.Builder builder = ModuleSpec.builder(); + + for (File jarFile : nullToEmpty(subDirectory.listFiles())) { + if (jarFile.isFile() && jarFile.getName().endsWith(".jar")) { + builder.withFile(jarFile.getAbsoluteFile()); + } + } + return builder.build(); + } + + private static File[] nullToEmpty(File[] elements) { + return elements == null ? new File[0] : elements; + } + + public List modules() { + return specs; + } + + @Override + public Iterator iterator() { + return specs.iterator(); + } + + public static final class ModuleSpec implements Serializable { + + private static final long serialVersionUID = 1; + private final List artifactUrls; + + private ModuleSpec(List artifacts) { + this.artifactUrls = Collections.unmodifiableList(artifacts); + } + + static Builder builder() { + return new Builder(); + } + + public List artifactUris() { + return artifactUrls; + } + + static final class Builder { + private final TreeSet artifacts = new TreeSet<>(); + + Builder withFile(File file) throws IOException { + Objects.requireNonNull(file); + artifacts.add(file.getCanonicalFile().toURI()); + return this; + } + + Builder withUri(URI uri) { + Objects.requireNonNull(uri); + artifacts.add(uri); + return this; + } + + ModuleSpec build() { + List sortedCopy = new ArrayList<>(artifacts); + return new ModuleSpec(sortedCopy); + } + } + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/StatefulFunctionsJob.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/StatefulFunctionsJob.java new file mode 100644 index 00000000..95aed8f8 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/StatefulFunctionsJob.java @@ -0,0 +1,88 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core; + +import com.ververica.statefun.flink.core.common.ConfigurationUtil; +import com.ververica.statefun.flink.core.translation.FlinkUniverse; +import java.util.Objects; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + +public class StatefulFunctionsJob { + + public static void main(String... args) throws Exception { + ParameterTool parameterTool = ParameterTool.fromArgs(args); + Configuration configuration = parameterTool.getConfiguration(); + + main(configuration); + } + + public static void main(Configuration configuration) throws Exception { + Objects.requireNonNull(configuration); + + setDefaultContextClassLoaderIfAbsent(); + setDefaultProviderIfAbsent( + configuration, new StatefulFunctionsUniverses.ClassPathUniverseProvider()); + + final StatefulFunctionsUniverse statefulFunctionsUniverse = + StatefulFunctionsUniverses.get( + Thread.currentThread().getContextClassLoader(), configuration); + + final StatefulFunctionsUniverseValidator statefulFunctionsUniverseValidator = + new StatefulFunctionsUniverseValidator(); + statefulFunctionsUniverseValidator.validate(statefulFunctionsUniverse); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + setDefaultConfiguration(configuration, env); + + FlinkUniverse flinkUniverse = new FlinkUniverse(statefulFunctionsUniverse); + flinkUniverse.configure(env); + + String jobName = configuration.getValue(StatefulFunctionsJobConstants.FLINK_JOB_NAME); + env.execute(jobName); + } + + private static void setDefaultConfiguration( + Configuration configuration, StreamExecutionEnvironment env) { + env.getConfig().setGlobalJobParameters(configuration); + env.getConfig().enableObjectReuse(); + final long checkpointingInterval = + configuration.getLong(StatefulFunctionsJobConstants.CHECKPOINTING_INTERVAL); + if (checkpointingInterval > 0) { + env.enableCheckpointing(checkpointingInterval); + } + } + + private static void setDefaultContextClassLoaderIfAbsent() { + ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); + if (classLoader == null) { + Thread.currentThread().setContextClassLoader(StatefulFunctionsJob.class.getClassLoader()); + } + } + + private static void setDefaultProviderIfAbsent( + Configuration configuration, StatefulFunctionsUniverseProvider provider) { + if (!configuration.contains( + StatefulFunctionsJobConstants.STATEFUL_FUNCTIONS_UNIVERSE_INITIALIZER_CLASS_BYTES)) { + ConfigurationUtil.storeSerializedInstance( + configuration, + StatefulFunctionsJobConstants.STATEFUL_FUNCTIONS_UNIVERSE_INITIALIZER_CLASS_BYTES, + provider); + } + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/StatefulFunctionsJobConstants.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/StatefulFunctionsJobConstants.java new file mode 100644 index 00000000..087f3c7e --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/StatefulFunctionsJobConstants.java @@ -0,0 +1,68 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core; + +import com.ververica.statefun.flink.core.message.MessageFactoryType; +import org.apache.flink.annotation.docs.Documentation; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; + +@SuppressWarnings("WeakerAccess") +public final class StatefulFunctionsJobConstants { + + public static final String FUNCTION_OPERATOR_NAME = "functions"; + public static final String FUNCTION_OPERATOR_UID = "functions_uid1"; + public static final String WRITE_BACK_OPERATOR_NAME = "feedback"; + public static final String WRITE_BACK_OPERATOR_UID = "feedback_uid1"; + public static final String ROUTER_NAME = "router"; + + @Documentation.ExcludeFromDocumentation("internal configuration") + public static final ConfigOption STATEFUL_FUNCTIONS_UNIVERSE_INITIALIZER_CLASS_BYTES = + ConfigOptions.key("stateful-functions.internal.core.universe-serialized-initializer") + .defaultValue(new byte[0]); + + public static final ConfigOption TOTAL_MEMORY_USED_FOR_FEEDBACK_CHECKPOINTING = + ConfigOptions.key("stateful-functions.feedback.memory.bytes") + .defaultValue(32 * 1024 * 1024) + .withDescription( + "The number of bytes to use for in memory buffering of the feedback channel, before spilling to disk"); + + public static final ConfigOption CHECKPOINTING_INTERVAL = + ConfigOptions.key("stateful-functions.state.checkpointing-interval-ms") + .defaultValue(30 * 1000L) + .withDescription( + "Flink checkpoint interval in milliseconds, set to -1 to disable checkpointing"); + + public static final ConfigOption MULTIPLEX_FLINK_STATE = + ConfigOptions.key("stateful-functions.state.multiplex-flink-state") + .defaultValue(true) + .withDescription( + "Use a single MapState to multiplex different function types and persisted values," + + "instead of using a ValueState for each combination"); + + public static final ConfigOption USER_MESSAGE_SERIALIZER = + ConfigOptions.key("stateful-functions.message.serializer") + .defaultValue(MessageFactoryType.WITH_PROTOBUF_PAYLOADS.name()) + .withDescription("The serializer to use for on the wire messages."); + + public static final ConfigOption FLINK_JOB_NAME = + ConfigOptions.key("stateful-functions.flink-job-name") + .defaultValue("StatefulFunctions") + .withDescription("The name to display at the Flink-UI"); + + private StatefulFunctionsJobConstants() {} +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/StatefulFunctionsUniverse.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/StatefulFunctionsUniverse.java new file mode 100644 index 00000000..292838bc --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/StatefulFunctionsUniverse.java @@ -0,0 +1,148 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core; + +import com.ververica.statefun.flink.core.message.MessageFactoryType; +import com.ververica.statefun.flink.core.types.StaticallyRegisteredTypes; +import com.ververica.statefun.flink.io.spi.FlinkIoModule; +import com.ververica.statefun.flink.io.spi.SinkProvider; +import com.ververica.statefun.flink.io.spi.SourceProvider; +import com.ververica.statefun.sdk.EgressType; +import com.ververica.statefun.sdk.FunctionType; +import com.ververica.statefun.sdk.IngressType; +import com.ververica.statefun.sdk.StatefulFunctionProvider; +import com.ververica.statefun.sdk.io.EgressIdentifier; +import com.ververica.statefun.sdk.io.EgressSpec; +import com.ververica.statefun.sdk.io.IngressIdentifier; +import com.ververica.statefun.sdk.io.IngressSpec; +import com.ververica.statefun.sdk.io.Router; +import com.ververica.statefun.sdk.spi.StatefulFunctionModule; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import javax.annotation.Nullable; + +public final class StatefulFunctionsUniverse + implements StatefulFunctionModule.Binder, FlinkIoModule.Binder { + + private final Map, IngressSpec> ingress = new HashMap<>(); + private final Map, EgressSpec> egress = new HashMap<>(); + private final Map, List>> routers = new HashMap<>(); + private final Map functions = new HashMap<>(); + private final Map sources = new HashMap<>(); + private final Map sinks = new HashMap<>(); + + private final StaticallyRegisteredTypes types; + private final MessageFactoryType messageFactoryType; + + public StatefulFunctionsUniverse(MessageFactoryType messageFactoryType) { + this.messageFactoryType = messageFactoryType; + this.types = new StaticallyRegisteredTypes(messageFactoryType); + } + + @Override + public void bindIngress(IngressSpec spec) { + Objects.requireNonNull(spec); + IngressIdentifier id = spec.id(); + putAndThrowIfPresent(ingress, id, spec); + types.registerType(id.producedType()); + } + + @Override + public void bindIngressRouter(IngressIdentifier ingressIdentifier, Router router) { + Objects.requireNonNull(ingressIdentifier); + Objects.requireNonNull(router); + + List> ingressRouters = + routers.computeIfAbsent(ingressIdentifier, unused -> new ArrayList<>()); + ingressRouters.add(router); + + types.registerType(ingressIdentifier.producedType()); + } + + @Override + public void bindEgress(EgressSpec spec) { + Objects.requireNonNull(spec); + EgressIdentifier id = spec.id(); + putAndThrowIfPresent(egress, id, spec); + + types.registerType(id.consumedType()); + } + + @Override + public void bindFunctionProvider(FunctionType functionType, StatefulFunctionProvider provider) { + Objects.requireNonNull(functionType); + Objects.requireNonNull(provider); + putAndThrowIfPresent(functions, functionType, provider); + } + + @Override + public void bindSourceProvider(IngressType type, SourceProvider provider) { + Objects.requireNonNull(type); + Objects.requireNonNull(provider); + + putAndThrowIfPresent(sources, type, provider); + } + + @Override + public void bindSinkProvider(EgressType type, SinkProvider provider) { + putAndThrowIfPresent(sinks, type, provider); + } + + public Map, IngressSpec> ingress() { + return ingress; + } + + public Map, EgressSpec> egress() { + return egress; + } + + public Map, List>> routers() { + return routers; + } + + public Map functions() { + return functions; + } + + public Map sources() { + return sources; + } + + public Map sinks() { + return sinks; + } + + public StaticallyRegisteredTypes types() { + return types; + } + + private static void putAndThrowIfPresent(Map map, K key, V value) { + @Nullable V previous = map.put(key, value); + if (previous == null) { + return; + } + throw new IllegalStateException( + String.format("A binding for the key %s was previously defined.", key)); + } + + public MessageFactoryType messageFactoryType() { + return messageFactoryType; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/StatefulFunctionsUniverseProvider.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/StatefulFunctionsUniverseProvider.java new file mode 100644 index 00000000..3ce61f6a --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/StatefulFunctionsUniverseProvider.java @@ -0,0 +1,25 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core; + +import java.io.Serializable; +import org.apache.flink.configuration.Configuration; + +public interface StatefulFunctionsUniverseProvider extends Serializable { + + StatefulFunctionsUniverse get(ClassLoader classLoader, Configuration configuration); +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/StatefulFunctionsUniverseValidator.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/StatefulFunctionsUniverseValidator.java new file mode 100644 index 00000000..6947b9bc --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/StatefulFunctionsUniverseValidator.java @@ -0,0 +1,36 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core; + +final class StatefulFunctionsUniverseValidator { + + void validate(StatefulFunctionsUniverse statefulFunctionsUniverse) { + // TODO: complete this + if (statefulFunctionsUniverse.ingress().isEmpty()) { + throw new IllegalStateException("There are no ingress defined."); + } + if (statefulFunctionsUniverse.sources().isEmpty()) { + throw new IllegalStateException("There are no source providers defined."); + } + if (statefulFunctionsUniverse.routers().isEmpty()) { + throw new IllegalStateException("There are no routers defined."); + } + if (statefulFunctionsUniverse.functions().isEmpty()) { + throw new IllegalStateException("There are no function providers defined."); + } + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/StatefulFunctionsUniverses.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/StatefulFunctionsUniverses.java new file mode 100644 index 00000000..fd3994ad --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/StatefulFunctionsUniverses.java @@ -0,0 +1,64 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core; + +import static java.util.Collections.synchronizedMap; + +import com.ververica.statefun.flink.core.common.ConfigurationUtil; +import com.ververica.statefun.flink.core.spi.Modules; +import java.util.Map; +import java.util.WeakHashMap; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.util.Preconditions; + +public final class StatefulFunctionsUniverses { + + private static final Map universes = + synchronizedMap(new WeakHashMap<>()); + + public static StatefulFunctionsUniverse get( + ClassLoader classLoader, Configuration configuration) { + Preconditions.checkState(classLoader != null, "The class loader was not set."); + Preconditions.checkState(configuration != null, "The configuration was not set."); + + return universes.computeIfAbsent( + classLoader, cl -> initializeFromConfiguration(cl, configuration)); + } + + private static StatefulFunctionsUniverse initializeFromConfiguration( + ClassLoader cl, Configuration configuration) { + + StatefulFunctionsUniverseProvider provider = + ConfigurationUtil.getSerializedInstance( + cl, + configuration, + StatefulFunctionsJobConstants.STATEFUL_FUNCTIONS_UNIVERSE_INITIALIZER_CLASS_BYTES); + + return provider.get(cl, configuration); + } + + static final class ClassPathUniverseProvider implements StatefulFunctionsUniverseProvider { + + private static final long serialVersionUID = 1; + + @Override + public StatefulFunctionsUniverse get(ClassLoader classLoader, Configuration configuration) { + Modules modules = Modules.loadFromClassPath(); + return modules.createStatefulFunctionsUniverse(configuration); + } + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/common/ConfigurationUtil.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/common/ConfigurationUtil.java new file mode 100644 index 00000000..a9dddc89 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/common/ConfigurationUtil.java @@ -0,0 +1,46 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.common; + +import java.io.IOException; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.util.InstantiationUtil; + +public final class ConfigurationUtil { + private ConfigurationUtil() {} + + public static T getSerializedInstance( + ClassLoader classLoader, Configuration configuration, ConfigOption option) { + final byte[] bytes = configuration.getBytes(option.key(), option.defaultValue()); + try { + return InstantiationUtil.deserializeObject(bytes, classLoader, false); + } catch (IOException | ClassNotFoundException e) { + throw new IllegalStateException("Unable to initialize.", e); + } + } + + public static void storeSerializedInstance( + Configuration configuration, ConfigOption option, Object instance) { + try { + byte[] bytes = InstantiationUtil.serializeObject(instance); + configuration.setBytes(option.key(), bytes); + } catch (IOException e) { + throw new IllegalStateException(e); + } + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/common/KeyBy.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/common/KeyBy.java new file mode 100644 index 00000000..065dd8b7 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/common/KeyBy.java @@ -0,0 +1,27 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.common; + +import com.ververica.statefun.sdk.Address; + +public final class KeyBy { + private KeyBy() {} + + public static String apply(Address address) { + return address.id(); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/common/SetContextClassLoader.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/common/SetContextClassLoader.java new file mode 100644 index 00000000..d43358f9 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/common/SetContextClassLoader.java @@ -0,0 +1,34 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.common; + +import java.io.Closeable; +import javax.annotation.Nonnull; + +public final class SetContextClassLoader implements Closeable { + private final ClassLoader originalClassLoader; + + public SetContextClassLoader(@Nonnull Object o) { + this.originalClassLoader = Thread.currentThread().getContextClassLoader(); + Thread.currentThread().setContextClassLoader(o.getClass().getClassLoader()); + } + + @Override + public void close() { + Thread.currentThread().setContextClassLoader(originalClassLoader); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/common/StreamTaskMailbox.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/common/StreamTaskMailbox.java new file mode 100644 index 00000000..c1a7ac47 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/common/StreamTaskMailbox.java @@ -0,0 +1,69 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.common; + +import java.lang.reflect.Field; +import java.util.Objects; +import java.util.concurrent.Executor; +import javax.annotation.Nonnull; +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.runtime.tasks.StreamTask; +import org.apache.flink.streaming.runtime.tasks.mailbox.Mailbox; +import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxSender; + +@Internal +public final class StreamTaskMailbox { + + public static Executor mailboxExecutor(StreamTask containingTask) { + MailboxSender sender = obtainContainingStreamTaskMailboxSenderForFlink19(containingTask); + return new MailboxAdapter(sender); + } + + /** + * In {@code Flink 1.9} the {@link Mailbox} is not exposed yet to the users {@link + * AbstractStreamOperator} but it is defined and functional in the containing {@link StreamTask}. + */ + private static MailboxSender obtainContainingStreamTaskMailboxSenderForFlink19( + StreamTask task) { + try { + Field field = StreamTask.class.getDeclaredField("mailbox"); + field.setAccessible(true); + return (MailboxSender) field.get(task); + } catch (IllegalAccessException | NoSuchFieldException e) { + throw new RuntimeException( + "Stateful Functions is currently only supported with Flink 1.9 version.", e); + } + } + + private static final class MailboxAdapter implements Executor { + private final MailboxSender sender; + + MailboxAdapter(MailboxSender sender) { + this.sender = Objects.requireNonNull(sender); + } + + @Override + public void execute(@Nonnull Runnable command) { + try { + sender.putMail(command); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/di/Inject.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/di/Inject.java new file mode 100644 index 00000000..a645f3c7 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/di/Inject.java @@ -0,0 +1,26 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.di; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +@Retention(RetentionPolicy.RUNTIME) +@Target(ElementType.CONSTRUCTOR) +public @interface Inject {} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/di/Label.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/di/Label.java new file mode 100644 index 00000000..2489884d --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/di/Label.java @@ -0,0 +1,28 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.di; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +@Retention(RetentionPolicy.RUNTIME) +@Target(ElementType.PARAMETER) +public @interface Label { + String value(); +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/di/Lazy.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/di/Lazy.java new file mode 100644 index 00000000..c7017b2d --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/di/Lazy.java @@ -0,0 +1,57 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.di; + +import java.util.Objects; +import javax.annotation.Nullable; + +@SuppressWarnings({"unchecked", "unused", "WeakerAccess"}) +public final class Lazy { + private final Class type; + private final String label; + private ObjectContainer container; + + @Nullable private T instance; + + public Lazy(Class type) { + this(type, null); + } + + public Lazy(Class type, String label) { + this.type = type; + this.label = label; + } + + public Lazy(T instance) { + this((Class) instance.getClass(), null); + this.instance = instance; + } + + Lazy withContainer(ObjectContainer container) { + this.container = Objects.requireNonNull(container); + return this; + } + + public T get() { + @Nullable T instance = this.instance; + if (instance == null) { + instance = container.get(type, label); + this.instance = instance; + } + return instance; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/di/ObjectContainer.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/di/ObjectContainer.java new file mode 100644 index 00000000..e6a67f05 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/di/ObjectContainer.java @@ -0,0 +1,167 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.di; + +import java.lang.annotation.Annotation; +import java.lang.reflect.Constructor; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.function.Supplier; +import javax.annotation.Nullable; +import org.apache.flink.annotation.Internal; + +/** Minimal dependency injection. */ +@Internal +public class ObjectContainer { + + private final Map> factories = new HashMap<>(); + + private final Map instances = new HashMap<>(); + + public void add(T singleton) { + Class type = singleton.getClass(); + factories.put(new Key(type), () -> singleton); + } + + public void add(Class type) { + factories.put(new Key(type), () -> createReflectively(type)); + } + + public void add(String label, Class type, T singleton) { + factories.put(new Key(type, label), () -> singleton); + } + + public void add(String label, Class type, Class actual) { + factories.put(new Key(type, label), () -> createReflectively(actual)); + } + + public void add(String label, Lazy lazyValue) { + factories.put(new Key(Lazy.class, label), () -> lazyValue.withContainer(this)); + } + + public T get(Class type) { + return get(type, null); + } + + public T get(Class type, String label) { + Key key = new Key(type, label); + return getOrCreateInstance(key); + } + + @SuppressWarnings("unchecked") + private T getOrCreateInstance(Key key) { + @Nullable Object instance = instances.get(key); + if (instance == null) { + instances.put(key, instance = create(key)); + } + return (T) instance; + } + + private Object create(Key key) { + Supplier factory = factories.get(key); + if (factory == null) { + throw new IllegalArgumentException("was not able to find a factory for " + key); + } + return factory.get(); + } + + private Object createReflectively(Class type) { + Constructor constructor = findConstructorForInjection(type); + Class[] dependencies = constructor.getParameterTypes(); + Annotation[][] annotations = constructor.getParameterAnnotations(); + Object[] resolvedDependencies = new Object[dependencies.length]; + int i = 0; + for (Class dependency : dependencies) { + @Nullable String label = findLabel(annotations[i]); + Key key = new Key(dependency, label); + resolvedDependencies[i] = getOrCreateInstance(key); + i++; + } + try { + constructor.setAccessible(true); + return constructor.newInstance(resolvedDependencies); + } catch (Throwable e) { + throw new RuntimeException(e); + } + } + + @Nullable + private static String findLabel(Annotation[] annotations) { + for (Annotation annotation : annotations) { + if (annotation.annotationType() == Label.class) { + return ((Label) annotation).value(); + } + } + return null; + } + + private static Constructor findConstructorForInjection(Class type) { + Constructor[] constructors = type.getDeclaredConstructors(); + Constructor defaultCont = null; + for (Constructor constructor : constructors) { + Annotation annotation = constructor.getAnnotation(Inject.class); + if (annotation != null) { + return constructor; + } + if (constructor.getParameterCount() == 0) { + defaultCont = constructor; + } + } + if (defaultCont != null) { + return defaultCont; + } + throw new RuntimeException("not injectable type " + type); + } + + private static final class Key { + final Class type; + + @Nullable final String label; + + Key(Class type, @Nullable String label) { + this.type = type; + this.label = label; + } + + Key(Class type) { + this(type, null); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Key key = (Key) o; + return Objects.equals(type, key.type) && Objects.equals(label, key.label); + } + + @Override + public int hashCode() { + return Objects.hash(type, label); + } + + @Override + public String toString() { + return "Key{" + "type=" + type + ", label='" + label + '\'' + '}'; + } + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/feedback/Feedback.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/feedback/Feedback.java new file mode 100644 index 00000000..a662486e --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/feedback/Feedback.java @@ -0,0 +1,37 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.feedback; + +import com.ververica.statefun.flink.core.message.Message; +import java.util.concurrent.Executor; +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.runtime.tasks.StreamTask; + +@Internal +public final class Feedback { + + public static void registerFeedbackConsumer( + SubtaskFeedbackKey subtaskKey, + StreamTask containingTask, + Executor mailBoxExecutor, + FeedbackConsumer consumer) { + + FeedbackChannelBroker broker = FeedbackChannelBroker.get(); + FeedbackChannel channel = broker.getChannel(subtaskKey); + channel.registerConsumer(consumer, containingTask.getCheckpointLock(), mailBoxExecutor); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/feedback/FeedbackChannel.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/feedback/FeedbackChannel.java new file mode 100644 index 00000000..1e4ab605 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/feedback/FeedbackChannel.java @@ -0,0 +1,148 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.feedback; + +import java.io.Closeable; +import java.util.Deque; +import java.util.Objects; +import java.util.concurrent.Executor; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.flink.util.IOUtils; + +/** Single producer, single consumer channel. */ +public final class FeedbackChannel implements Closeable { + + /** The key that used to identify this channel. */ + private final SubtaskFeedbackKey key; + + /** The underlying queue used to hold the feedback results. */ + private final FeedbackQueue queue; + + /** A single registered consumer */ + private final AtomicReference> consumerRef = new AtomicReference<>(); + + FeedbackChannel(SubtaskFeedbackKey key, FeedbackQueue queue) { + this.key = Objects.requireNonNull(key); + this.queue = Objects.requireNonNull(queue); + } + + // -------------------------------------------------------------------------------------------------------------- + // API + // -------------------------------------------------------------------------------------------------------------- + + /** Adds a feedback result to this channel. */ + public void put(T value) { + if (!queue.addAndCheckIfWasEmpty(value)) { + // successfully added @value into the queue, but the queue wasn't (atomically) drained yet, + // so there is nothing more to do. + return; + } + @SuppressWarnings("resource") + final ConsumerTask consumer = consumerRef.get(); + if (consumer == null) { + // the queue has become non empty at the first time, yet at the same time the (single) + // consumer has not yet registered, so there is nothing to do. + // once the consumer would register a drain would be scheduled for the first time. + return; + } + // the queue was previously empty, and now it is not, therefore we schedule a drain. + consumer.scheduleDrainAll(); + } + + /** + * Register a feedback iteration consumer + * + *

Any invocations to the consumer are guaranteed to happen under the consumerLock. + * + * @param consumer the feedback events consumer. + * @param consumerLock a lock to acquire before calling the consumer. + * @param executor the executor to schedule feedback consumption on. + */ + void registerConsumer( + final FeedbackConsumer consumer, final Object consumerLock, Executor executor) { + Objects.requireNonNull(consumer); + Objects.requireNonNull(consumerLock); + + ConsumerTask consumerTask = new ConsumerTask<>(consumerLock, executor, consumer, queue); + + if (!this.consumerRef.compareAndSet(null, consumerTask)) { + throw new IllegalStateException("There can be only a single consumer in a FeedbackChannel."); + } + // we must try to drain the underlying queue on registration (by scheduling the consumerTask) + // because + // the consumer might be registered after the producer has already started producing data into + // the feedback channel. + consumerTask.scheduleDrainAll(); + } + + // -------------------------------------------------------------------------------------------------------------- + // Internal + // -------------------------------------------------------------------------------------------------------------- + + /** Closes this channel. */ + @Override + public void close() { + ConsumerTask consumer = consumerRef.getAndSet(null); + IOUtils.closeQuietly(consumer); + // remove this channel. + FeedbackChannelBroker broker = FeedbackChannelBroker.get(); + broker.removeChannel(key); + } + + private static final class ConsumerTask implements Runnable, Closeable { + private final Object checkpointLock; + private final Executor executor; + private final FeedbackConsumer consumer; + private final FeedbackQueue queue; + + ConsumerTask( + Object checkpointLock, + Executor executor, + FeedbackConsumer consumer, + FeedbackQueue queue) { + this.checkpointLock = Objects.requireNonNull(checkpointLock); + this.executor = Objects.requireNonNull(executor); + this.consumer = Objects.requireNonNull(consumer); + this.queue = Objects.requireNonNull(queue); + } + + void scheduleDrainAll() { + executor.execute(this); + } + + @Override + public void run() { + // accesses to the consumer are protected with the consumerLock, + // this is so that it can be used with the operators, that requires all the methods to + // enter with the checkpoint lock. + synchronized (checkpointLock) { + final Deque buffer = queue.drainAll(); + try { + T element; + while ((element = buffer.pollFirst()) != null) { + consumer.processFeedback(element); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + } + + @Override + public void close() {} + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/feedback/FeedbackChannelBroker.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/feedback/FeedbackChannelBroker.java new file mode 100644 index 00000000..7187ddd6 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/feedback/FeedbackChannelBroker.java @@ -0,0 +1,62 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.feedback; + +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; + +/** + * HandOffChannelBroker. + * + *

It is used together with the co-location constrain so that two tasks can access the same + * "hand-off" channel, and communicate directly (not via the network stack) by simply passing + * references in one direction. + * + *

To obtain a feedback channel one must first obtain an {@link SubtaskFeedbackKey} and simply + * call {@link #get()}. A channel is removed from this broker on a call to {@link + * FeedbackChannel#close()}. + */ +public final class FeedbackChannelBroker { + + private static final FeedbackChannelBroker INSTANCE = new FeedbackChannelBroker(); + + private final ConcurrentHashMap, FeedbackChannel> channels = + new ConcurrentHashMap<>(); + + public static FeedbackChannelBroker get() { + return INSTANCE; + } + + @SuppressWarnings({"unchecked"}) + public FeedbackChannel getChannel(SubtaskFeedbackKey key) { + Objects.requireNonNull(key); + + FeedbackChannel channel = channels.computeIfAbsent(key, FeedbackChannelBroker::newChannel); + + return (FeedbackChannel) channel; + } + + @SuppressWarnings("resource") + void removeChannel(SubtaskFeedbackKey key) { + channels.remove(key); + } + + private static FeedbackChannel newChannel(SubtaskFeedbackKey key) { + FeedbackQueue queue = new LockFreeBatchFeedbackQueue<>(); + return new FeedbackChannel<>(key, queue); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/feedback/FeedbackConsumer.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/feedback/FeedbackConsumer.java new file mode 100644 index 00000000..b18fe36c --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/feedback/FeedbackConsumer.java @@ -0,0 +1,24 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.feedback; + +/** HandOffConsumer. */ +@FunctionalInterface +public interface FeedbackConsumer { + + void processFeedback(T element) throws Exception; +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/feedback/FeedbackKey.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/feedback/FeedbackKey.java new file mode 100644 index 00000000..a1ac0382 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/feedback/FeedbackKey.java @@ -0,0 +1,59 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.feedback; + +import java.io.Serializable; +import java.util.Objects; + +/** A FeedbackKey without runtime information. */ +public final class FeedbackKey implements Serializable { + + private static final long serialVersionUID = 1; + + private final String pipelineName; + private final long invocationId; + + public FeedbackKey(String pipelineName, long invocationId) { + this.pipelineName = Objects.requireNonNull(pipelineName); + this.invocationId = invocationId; + } + + public SubtaskFeedbackKey withSubTaskIndex(int subTaskIndex) { + return new SubtaskFeedbackKey<>(pipelineName, invocationId, subTaskIndex); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + FeedbackKey that = (FeedbackKey) o; + return invocationId == that.invocationId && Objects.equals(pipelineName, that.pipelineName); + } + + @Override + public int hashCode() { + return Objects.hash(pipelineName, invocationId); + } + + public String asColocationKey() { + return String.format("CO-LOCATION/%s/%d", pipelineName, invocationId); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/feedback/FeedbackQueue.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/feedback/FeedbackQueue.java new file mode 100644 index 00000000..cf5f8e06 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/feedback/FeedbackQueue.java @@ -0,0 +1,47 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.feedback; + +import java.util.Deque; + +/** + * HandOffQueue - is a single producer single consumer (spsc) queue that supports adding and + * draining atomically. + * + *

Implementors of this queue supports atomic addition operation (via {@link + * #addAndCheckIfWasEmpty(Object)} and atomic, bulk retrieving of the content of this queue (via + * {@link #drainAll()})}. + * + * @param element type that is stored in this queue. + */ +interface FeedbackQueue { + + /** + * Adds an element to the queue atomically. + * + * @param element the element to add to the queue. + * @return true, if prior to this addition the queue was empty. + */ + boolean addAndCheckIfWasEmpty(ElementT element); + + /** + * Atomically grabs all that elements of this queue. + * + * @return the elements present at the queue at the moment of this operation. + */ + Deque drainAll(); +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/feedback/FeedbackSinkOperator.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/feedback/FeedbackSinkOperator.java new file mode 100644 index 00000000..58a5c2b6 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/feedback/FeedbackSinkOperator.java @@ -0,0 +1,99 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.feedback; + +import java.util.Objects; +import java.util.function.LongFunction; +import org.apache.flink.metrics.MeterView; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.SimpleCounter; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.util.IOUtils; + +/** IterationSinkOperator. */ +public final class FeedbackSinkOperator extends AbstractStreamOperator + implements OneInputStreamOperator { + + private static final long serialVersionUID = 1; + + // ----- configuration ----- + + private final FeedbackKey key; + private final LongFunction barrierSentinelSupplier; + + // ----- runtime ----- + + private transient FeedbackChannel channel; + private transient SimpleCounter totalProduced; + + public FeedbackSinkOperator(FeedbackKey key, LongFunction barrierSentinelSupplier) { + this.key = Objects.requireNonNull(key); + this.barrierSentinelSupplier = Objects.requireNonNull(barrierSentinelSupplier); + } + + // ---------------------------------------------------------------------------------------------------------- + // Runtime + // ---------------------------------------------------------------------------------------------------------- + + @Override + public void processElement(StreamRecord record) { + V value = record.getValue(); + channel.put(value); + totalProduced.inc(); + } + + // ---------------------------------------------------------------------------------------------------------- + // Operator lifecycle + // ---------------------------------------------------------------------------------------------------------- + + @Override + public void open() throws Exception { + super.open(); + final int indexOfThisSubtask = getRuntimeContext().getIndexOfThisSubtask(); + final SubtaskFeedbackKey key = this.key.withSubTaskIndex(indexOfThisSubtask); + + FeedbackChannelBroker broker = FeedbackChannelBroker.get(); + this.channel = broker.getChannel(key); + + // metrics + MetricGroup metrics = getRuntimeContext().getMetricGroup().addGroup("writeback"); + SimpleCounter produced = metrics.counter("produced", new SimpleCounter()); + metrics.meter("producedRate", new MeterView(produced, 60)); + this.totalProduced = produced; + } + + @Override + public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { + super.prepareSnapshotPreBarrier(checkpointId); + V sentinel = barrierSentinelSupplier.apply(checkpointId); + channel.put(sentinel); + } + + @Override + public void close() throws Exception { + IOUtils.closeQuietly(channel); + super.close(); + } + + @Override + public void dispose() throws Exception { + IOUtils.closeQuietly(channel); + super.dispose(); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/feedback/LockFreeBatchFeedbackQueue.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/feedback/LockFreeBatchFeedbackQueue.java new file mode 100644 index 00000000..b3d53a6b --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/feedback/LockFreeBatchFeedbackQueue.java @@ -0,0 +1,38 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.feedback; + +import com.ververica.statefun.flink.core.queue.Locks; +import com.ververica.statefun.flink.core.queue.MpscQueue; +import java.util.Deque; + +public final class LockFreeBatchFeedbackQueue implements FeedbackQueue { + private static final int INITIAL_BUFFER_SIZE = 32 * 1024; // 32k + + private final MpscQueue queue = new MpscQueue<>(INITIAL_BUFFER_SIZE, Locks.spinLock()); + + @Override + public boolean addAndCheckIfWasEmpty(ElementT element) { + final int size = queue.add(element); + return size == 1; + } + + @Override + public Deque drainAll() { + return queue.drainAll(); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/feedback/SubtaskFeedbackKey.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/feedback/SubtaskFeedbackKey.java new file mode 100644 index 00000000..ac0435db --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/feedback/SubtaskFeedbackKey.java @@ -0,0 +1,56 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.feedback; + +import java.io.Serializable; +import java.util.Objects; + +/** A FeedbackKey bounded to a subtask index. */ +@SuppressWarnings("unused") +public final class SubtaskFeedbackKey implements Serializable { + + private static final long serialVersionUID = 1; + + private final String pipelineName; + private final int subtaskIndex; + private final long invocationId; + + SubtaskFeedbackKey(String pipeline, long invocationId, int subtaskIndex) { + this.pipelineName = Objects.requireNonNull(pipeline); + this.invocationId = invocationId; + this.subtaskIndex = subtaskIndex; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SubtaskFeedbackKey that = (SubtaskFeedbackKey) o; + return subtaskIndex == that.subtaskIndex + && invocationId == that.invocationId + && Objects.equals(pipelineName, that.pipelineName); + } + + @Override + public int hashCode() { + return Objects.hash(pipelineName, subtaskIndex, invocationId); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/ApplyingContext.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/ApplyingContext.java new file mode 100644 index 00000000..65e6d7d2 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/ApplyingContext.java @@ -0,0 +1,25 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.functions; + +import com.ververica.statefun.flink.core.message.Message; +import com.ververica.statefun.sdk.Context; + +public interface ApplyingContext extends Context { + + void apply(LiveFunction function, Message inMessage); +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/AsyncMessageDecorator.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/AsyncMessageDecorator.java new file mode 100644 index 00000000..21ca0b10 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/AsyncMessageDecorator.java @@ -0,0 +1,112 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.functions; + +import com.ververica.statefun.flink.core.message.Message; +import com.ververica.statefun.flink.core.message.MessageFactory; +import com.ververica.statefun.sdk.Address; +import com.ververica.statefun.sdk.AsyncOperationResult; +import com.ververica.statefun.sdk.AsyncOperationResult.Status; +import javax.annotation.Nullable; +import org.apache.flink.api.common.state.MapState; +import org.apache.flink.core.memory.DataOutputView; + +/** + * Wraps the original {@link Message} where it's payload is the user supplied metadata associated + * with an async operation. + */ +final class AsyncMessageDecorator implements Message { + private final MapState pendingAsyncOperations; + private final long futureId; + private final Message message; + private final Throwable throwable; + private final T result; + private final boolean restored; + + AsyncMessageDecorator( + MapState pendingAsyncOperations, + long futureId, + Message message, + T result, + Throwable throwable) { + this.futureId = futureId; + this.pendingAsyncOperations = pendingAsyncOperations; + this.message = message; + this.throwable = throwable; + this.result = result; + this.restored = false; + } + + AsyncMessageDecorator( + MapState asyncOperationState, Long futureId, Message metadataMessage) { + this.futureId = futureId; + this.pendingAsyncOperations = asyncOperationState; + this.message = metadataMessage; + this.throwable = null; + this.result = null; + this.restored = true; + } + + @Nullable + @Override + public Address source() { + return message.source(); + } + + @Override + public Address target() { + return message.target(); + } + + @Override + public Object payload(MessageFactory context, ClassLoader targetClassLoader) { + final Status status; + if (restored) { + status = Status.UNKNOWN; + } else if (throwable == null) { + status = Status.SUCCESS; + } else { + status = Status.FAILURE; + } + Object metadata = message.payload(context, targetClassLoader); + return new AsyncOperationResult<>(metadata, status, result, throwable); + } + + @Override + public boolean isBarrierMessage() { + return false; + } + + @Override + public void postApply() { + try { + pendingAsyncOperations.remove(futureId); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + public Message copy(MessageFactory context) { + throw new UnsupportedOperationException(); + } + + @Override + public void writeTo(MessageFactory context, DataOutputView target) { + throw new UnsupportedOperationException(); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/AsyncOperationFailureNotifier.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/AsyncOperationFailureNotifier.java new file mode 100644 index 00000000..34051a24 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/AsyncOperationFailureNotifier.java @@ -0,0 +1,78 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.functions; + +import com.ververica.statefun.flink.core.message.Message; +import java.util.Map.Entry; +import java.util.Objects; +import org.apache.flink.api.common.state.MapState; +import org.apache.flink.api.common.state.MapStateDescriptor; +import org.apache.flink.runtime.state.KeyedStateBackend; +import org.apache.flink.runtime.state.KeyedStateFunction; +import org.apache.flink.runtime.state.VoidNamespace; +import org.apache.flink.runtime.state.VoidNamespaceSerializer; + +final class AsyncOperationFailureNotifier + implements KeyedStateFunction> { + + static void fireExpiredAsyncOperations( + MapStateDescriptor asyncOperationStateDescriptor, + Reductions reductions, + MapState asyncOperationState, + KeyedStateBackend keyedStateBackend) + throws Exception { + + AsyncOperationFailureNotifier asyncOperationFailureNotifier = + new AsyncOperationFailureNotifier(reductions, asyncOperationState); + + keyedStateBackend.applyToAllKeys( + VoidNamespace.get(), + VoidNamespaceSerializer.INSTANCE, + asyncOperationStateDescriptor, + asyncOperationFailureNotifier); + + if (asyncOperationFailureNotifier.enqueued()) { + reductions.processEnvelopes(); + } + } + + private final Reductions reductions; + private final MapState asyncOperationState; + + private boolean enqueued; + + private AsyncOperationFailureNotifier( + Reductions reductions, MapState asyncOperationState) { + this.reductions = Objects.requireNonNull(reductions); + this.asyncOperationState = Objects.requireNonNull(asyncOperationState); + } + + @Override + public void process(String key, MapState state) throws Exception { + for (Entry entry : state.entries()) { + Long futureId = entry.getKey(); + Message metadataMessage = entry.getValue(); + Message adaptor = new AsyncMessageDecorator(asyncOperationState, futureId, metadataMessage); + reductions.enqueue(adaptor); + enqueued = true; + } + } + + private boolean enqueued() { + return enqueued; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/AsyncSink.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/AsyncSink.java new file mode 100644 index 00000000..b9a4b700 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/AsyncSink.java @@ -0,0 +1,91 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.functions; + +import com.ververica.statefun.flink.core.di.Inject; +import com.ververica.statefun.flink.core.di.Label; +import com.ververica.statefun.flink.core.di.Lazy; +import com.ververica.statefun.flink.core.message.Message; +import com.ververica.statefun.flink.core.queue.Locks; +import com.ververica.statefun.flink.core.queue.MpscQueue; +import java.util.Deque; +import java.util.Objects; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.flink.api.common.state.MapState; + +final class AsyncSink { + private final MapState pendingAsyncOperations; + private final Lazy reductions; + private final Executor asOperator; + private final Executor operatorMailbox; + + private final MpscQueue completed = new MpscQueue<>(32768, Locks.jdkReentrantLock()); + + @Inject + AsyncSink( + @Label("async-operations") MapState pendingAsyncOperations, + @Label("checkpoint-lock-executor") Executor asOperator, + @Label("mailbox-executor") Executor operatorMailbox, + @Label("reductions") Lazy reductions) { + this.pendingAsyncOperations = Objects.requireNonNull(pendingAsyncOperations); + this.asOperator = Objects.requireNonNull(asOperator); + this.reductions = Objects.requireNonNull(reductions); + this.operatorMailbox = Objects.requireNonNull(operatorMailbox); + } + + void accept(Message metadata, CompletableFuture future) { + final long futureId = ThreadLocalRandom.current().nextLong(); // TODO: is this is good enough? + // we keep the message in state (associated with futureId) until either: + // 1. the future successfully completes and the message is processed. The state would be + // cleared by the AsyncMessageDecorator after a successful application. + // 2. after recovery, we clear that state by notifying the owning function that we don't know + // what happened + // with that particular async operation. + try { + pendingAsyncOperations.put(futureId, metadata); + } catch (Exception e) { + throw new RuntimeException(e); + } + future.whenComplete((result, throwable) -> enqueue(metadata, futureId, result, throwable)); + } + + private void enqueue(Message message, long futureId, T result, Throwable throwable) { + AsyncMessageDecorator decoratedMessage = + new AsyncMessageDecorator<>(pendingAsyncOperations, futureId, message, result, throwable); + + final int size = completed.add(decoratedMessage); + if (size == 1) { + // the queue has become non empty, we need to schedule a drain operation. + operatorMailbox.execute(this::drainOnOperatorThreadUnderCheckpointLock); + } + } + + private void drainOnOperatorThreadUnderCheckpointLock() { + asOperator.execute( + () -> { + Deque batchOfCompletedFutures = completed.drainAll(); + Reductions reductions = this.reductions.get(); + Message message; + while ((message = batchOfCompletedFutures.poll()) != null) { + reductions.enqueue(message); + } + reductions.processEnvelopes(); + }); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/DelaySink.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/DelaySink.java new file mode 100644 index 00000000..96d10d72 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/DelaySink.java @@ -0,0 +1,97 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.functions; + +import com.ververica.statefun.flink.core.di.Inject; +import com.ververica.statefun.flink.core.di.Label; +import com.ververica.statefun.flink.core.di.Lazy; +import com.ververica.statefun.flink.core.message.Message; +import java.util.Objects; +import org.apache.flink.runtime.state.VoidNamespace; +import org.apache.flink.streaming.api.operators.InternalTimer; +import org.apache.flink.streaming.api.operators.InternalTimerService; +import org.apache.flink.streaming.api.operators.Triggerable; +import org.apache.flink.util.Preconditions; + +final class DelaySink implements Triggerable { + + private final InternalTimerService delayedMessagesTimerService; + private final DelayedMessagesBuffer delayedMessagesBuffer; + + private final Lazy reductionsSupplier; + private final Partition thisPartition; + private final RemoteSink remoteSink; + + @Inject + DelaySink( + @Label("delayed-messages-buffer") DelayedMessagesBuffer delayedMessagesBuffer, + @Label("delayed-messages-timer-service-factory") + TimerServiceFactory delayedMessagesTimerServiceFactory, + @Label("reductions") Lazy reductionsSupplier, + Partition thisPartition, + RemoteSink remoteSink) { + this.delayedMessagesBuffer = Objects.requireNonNull(delayedMessagesBuffer); + this.reductionsSupplier = Objects.requireNonNull(reductionsSupplier); + this.thisPartition = Objects.requireNonNull(thisPartition); + this.remoteSink = Objects.requireNonNull(remoteSink); + + this.delayedMessagesTimerService = delayedMessagesTimerServiceFactory.createTimerService(this); + } + + void accept(Message message, long delayMillis) { + Objects.requireNonNull(message); + Preconditions.checkArgument(delayMillis >= 0); + + final long triggerTime = delayedMessagesTimerService.currentProcessingTime() + delayMillis; + + delayedMessagesTimerService.registerProcessingTimeTimer(VoidNamespace.INSTANCE, triggerTime); + delayedMessagesBuffer.add(message, triggerTime); + } + + @Override + public void onProcessingTime(InternalTimer timer) throws Exception { + final long triggerTimestamp = timer.getTimestamp(); + final Reductions reductions = reductionsSupplier.get(); + + Iterable delayedMessages = delayedMessagesBuffer.getForTimestamp(triggerTimestamp); + if (delayedMessages == null) { + throw new IllegalStateException( + "A delayed message timer was triggered with timestamp " + + triggerTimestamp + + ", but no messages were buffered for it."); + } + for (Message delayedMessage : delayedMessages) { + if (thisPartition.contains(delayedMessage.target())) { + reductions.enqueue(delayedMessage); + } else { + remoteSink.accept(delayedMessage); + } + } + // we clear the delayedMessageBuffer *before* we process the enqueued local reductions, because + // processing the envelops might actually trigger a delayed message to be sent with the same + // @triggerTimestamp + // so it would be re-enqueued into the delayedMessageBuffer. + delayedMessagesBuffer.clearForTimestamp(triggerTimestamp); + reductions.processEnvelopes(); + } + + @Override + public void onEventTime(InternalTimer timer) throws Exception { + throw new UnsupportedOperationException( + "Delayed messages with event time semantics is not supported."); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/DelayedMessagesBuffer.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/DelayedMessagesBuffer.java new file mode 100644 index 00000000..be01be8d --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/DelayedMessagesBuffer.java @@ -0,0 +1,28 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.functions; + +import com.ververica.statefun.flink.core.message.Message; + +interface DelayedMessagesBuffer { + + void add(Message message, long untilTimestamp); + + Iterable getForTimestamp(long timestamp); + + void clearForTimestamp(long timestamp); +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/FlinkStateDelayedMessagesBuffer.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/FlinkStateDelayedMessagesBuffer.java new file mode 100644 index 00000000..a0c321b1 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/FlinkStateDelayedMessagesBuffer.java @@ -0,0 +1,65 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.functions; + +import com.ververica.statefun.flink.core.di.Inject; +import com.ververica.statefun.flink.core.di.Label; +import com.ververica.statefun.flink.core.message.Message; +import java.util.Objects; +import org.apache.flink.runtime.state.internal.InternalListState; + +final class FlinkStateDelayedMessagesBuffer implements DelayedMessagesBuffer { + + static final String BUFFER_STATE_NAME = "delayed-messages-buffer"; + + private final InternalListState bufferState; + + @Inject + FlinkStateDelayedMessagesBuffer( + @Label("delayed-messages-buffer-state") + InternalListState bufferState) { + this.bufferState = Objects.requireNonNull(bufferState); + } + + @Override + public void add(Message message, long untilTimestamp) { + bufferState.setCurrentNamespace(untilTimestamp); + try { + bufferState.add(message); + } catch (Exception e) { + throw new RuntimeException("Error adding delayed message to state buffer: " + message, e); + } + } + + @Override + public Iterable getForTimestamp(long timestamp) { + bufferState.setCurrentNamespace(timestamp); + + try { + return bufferState.get(); + } catch (Exception e) { + throw new RuntimeException( + "Error accessing delayed message in state buffer for timestamp: " + timestamp, e); + } + } + + @Override + public void clearForTimestamp(long timestamp) { + bufferState.setCurrentNamespace(timestamp); + bufferState.clear(); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/FlinkTimerServiceFactory.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/FlinkTimerServiceFactory.java new file mode 100644 index 00000000..ca733e08 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/FlinkTimerServiceFactory.java @@ -0,0 +1,49 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.functions; + +import java.util.Objects; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.runtime.state.VoidNamespace; +import org.apache.flink.runtime.state.VoidNamespaceSerializer; +import org.apache.flink.streaming.api.operators.InternalTimeServiceManager; +import org.apache.flink.streaming.api.operators.InternalTimerService; +import org.apache.flink.streaming.api.operators.TimerSerializer; +import org.apache.flink.streaming.api.operators.Triggerable; + +final class FlinkTimerServiceFactory implements TimerServiceFactory { + + private static final String DELAYED_MSG_TIMER_SERVICE_NAME = "delayed-messages-timer-service"; + + private final InternalTimeServiceManager timeServiceManager; + + @SuppressWarnings("unchecked") + FlinkTimerServiceFactory(InternalTimeServiceManager timeServiceManager) { + this.timeServiceManager = + (InternalTimeServiceManager) Objects.requireNonNull(timeServiceManager); + } + + @Override + public InternalTimerService createTimerService( + Triggerable triggerable) { + final TimerSerializer timerSerializer = + new TimerSerializer<>(StringSerializer.INSTANCE, VoidNamespaceSerializer.INSTANCE); + + return timeServiceManager.getInternalTimerService( + DELAYED_MSG_TIMER_SERVICE_NAME, timerSerializer, triggerable); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/FunctionActivation.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/FunctionActivation.java new file mode 100644 index 00000000..477fcb38 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/FunctionActivation.java @@ -0,0 +1,55 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.functions; + +import com.ververica.statefun.flink.core.message.Message; +import com.ververica.statefun.sdk.Address; +import com.ververica.statefun.sdk.StatefulFunction; +import java.util.ArrayDeque; + +/** An {@link StatefulFunction} instance with a mailbox. */ +final class FunctionActivation { + private final ArrayDeque mailbox; + private Address self; + private LiveFunction function; + + FunctionActivation() { + this.mailbox = new ArrayDeque<>(); + } + + void setFunction(Address self, LiveFunction function) { + this.self = self; + this.function = function; + } + + void add(Message message) { + mailbox.addLast(message); + } + + boolean hasPendingEnvelope() { + return !mailbox.isEmpty(); + } + + void applyNextPendingEnvelope(ApplyingContext context) { + Message message = mailbox.pollFirst(); + context.apply(function, message); + } + + Address self() { + return self; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/FunctionGroupOperator.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/FunctionGroupOperator.java new file mode 100644 index 00000000..a5350439 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/FunctionGroupOperator.java @@ -0,0 +1,248 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.functions; + +import static com.ververica.statefun.flink.core.StatefulFunctionsJobConstants.TOTAL_MEMORY_USED_FOR_FEEDBACK_CHECKPOINTING; + +import com.ververica.statefun.flink.core.StatefulFunctionsUniverse; +import com.ververica.statefun.flink.core.StatefulFunctionsUniverses; +import com.ververica.statefun.flink.core.common.StreamTaskMailbox; +import com.ververica.statefun.flink.core.feedback.Feedback; +import com.ververica.statefun.flink.core.feedback.FeedbackConsumer; +import com.ververica.statefun.flink.core.feedback.FeedbackKey; +import com.ververica.statefun.flink.core.feedback.SubtaskFeedbackKey; +import com.ververica.statefun.flink.core.logger.Loggers; +import com.ververica.statefun.flink.core.logger.UnboundedFeedbackLogger; +import com.ververica.statefun.flink.core.message.Message; +import com.ververica.statefun.flink.core.message.MessageFactory; +import com.ververica.statefun.flink.core.message.MessageTypeInformation; +import com.ververica.statefun.sdk.io.EgressIdentifier; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.Executor; +import org.apache.flink.api.common.ExecutionConfig.GlobalJobParameters; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.state.MapState; +import org.apache.flink.api.common.state.MapStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.state.KeyGroupStatePartitionStreamProvider; +import org.apache.flink.runtime.state.KeyedStateBackend; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.runtime.state.internal.InternalListState; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.StreamTask; +import org.apache.flink.util.IOUtils; +import org.apache.flink.util.OutputTag; +import org.apache.flink.util.Preconditions; + +public class FunctionGroupOperator extends AbstractStreamOperator + implements FeedbackConsumer, OneInputStreamOperator { + + private static final long serialVersionUID = 1L; + + // -- configuration + private final Map, OutputTag> sideOutputs; + private final FeedbackKey feedbackKey; + + // -- runtime + private transient Reductions reductions; + private transient UnboundedFeedbackLogger feedbackLogger; + private transient boolean closedOrDisposed; + + public FunctionGroupOperator( + FeedbackKey feedbackKey, Map, OutputTag> sideOutputs) { + + this.feedbackKey = Objects.requireNonNull(feedbackKey); + this.sideOutputs = Objects.requireNonNull(sideOutputs); + } + + // ------------------------------------------------------------------------------------------------------------------ + // Operator API + // ------------------------------------------------------------------------------------------------------------------ + + @Override + public void processElement(StreamRecord record) { + reductions.apply(record.getValue()); + } + + @Override + public void processFeedback(Message message) { + if (closedOrDisposed) { + // since this code executes on a different thread than the operator thread + // (although using the same checkpoint lock), we must check if the operator + // wasn't closed or disposed. + return; + } + if (message.isBarrierMessage()) { + feedbackLogger.commit(); + } else { + reductions.apply(message); + feedbackLogger.append(message); + } + } + + @Override + public void initializeState(StateInitializationContext context) throws Exception { + super.initializeState(context); + + final Configuration configuration = getConfiguration(); + final StatefulFunctionsUniverse statefulFunctionsUniverse = + statefulFunctionsUniverse(configuration); + final TypeSerializer envelopeSerializer = + messageTypeSerializer(statefulFunctionsUniverse); + final Executor mailboxExecutor = StreamTaskMailbox.mailboxExecutor(getContainingTask()); + final Executor checkpointLockExecutor = + new UnderCheckpointLockExecutor( + getContainingTask().getCheckpointLock(), () -> closedOrDisposed, getContainingTask()); + final MapStateDescriptor asyncOperationStateDescriptor = + new MapStateDescriptor<>( + "asyncOperations", LongSerializer.INSTANCE, envelopeSerializer.duplicate()); + final ListStateDescriptor delayedMessageStateDescriptor = + new ListStateDescriptor<>( + FlinkStateDelayedMessagesBuffer.BUFFER_STATE_NAME, envelopeSerializer.duplicate()); + final MapState asyncOperationState = + getRuntimeContext().getMapState(asyncOperationStateDescriptor); + + // + // the core logic of applying messages to functions. + // + this.reductions = + Reductions.create( + configuration, + statefulFunctionsUniverse, + getRuntimeContext(), + getKeyedStateBackend(), + new FlinkTimerServiceFactory(super.timeServiceManager), + delayedMessagesBufferState(delayedMessageStateDescriptor), + sideOutputs, + output, + MessageFactory.forType(statefulFunctionsUniverse.messageFactoryType()), + mailboxExecutor, + getRuntimeContext().getMetricGroup().addGroup("functions"), + asyncOperationState, + checkpointLockExecutor); + // + // setup the write back edge logger + // + final int totalMemoryUsedForFeedbackCheckpointing = + configuration.getInteger(TOTAL_MEMORY_USED_FOR_FEEDBACK_CHECKPOINTING); + IOManager ioManager = getContainingTask().getEnvironment().getIOManager(); + + this.feedbackLogger = + Loggers.unboundedSpillableLogger( + ioManager, + getRuntimeContext().getMaxNumberOfParallelSubtasks(), + totalMemoryUsedForFeedbackCheckpointing, + envelopeSerializer.duplicate()); + + // + // expire all the pending async operations. + // + AsyncOperationFailureNotifier.fireExpiredAsyncOperations( + asyncOperationStateDescriptor, reductions, asyncOperationState, getKeyedStateBackend()); + + // we first must reply previously checkpointed envelopes before we start + // processing any new envelopes. + // + for (KeyGroupStatePartitionStreamProvider keyedStateInput : context.getRawKeyedStateInputs()) { + this.feedbackLogger.replyLoggedEnvelops(keyedStateInput.getStream(), this); + } + registerFeedbackConsumer(mailboxExecutor); + } + + @Override + public void snapshotState(StateSnapshotContext context) throws Exception { + super.snapshotState(context); + this.feedbackLogger.startLogging(context.getRawKeyedOperatorStateOutput()); + } + + @Override + public void close() throws Exception { + closeInternally(); + super.close(); + } + + @Override + public void dispose() throws Exception { + closeInternally(); + super.dispose(); + } + + // ------------------------------------------------------------------------------------------------------------------ + // Helpers + // ------------------------------------------------------------------------------------------------------------------ + + private void closeInternally() { + IOUtils.closeQuietly(feedbackLogger); + feedbackLogger = null; + closedOrDisposed = true; + } + + private void registerFeedbackConsumer(Executor mailboxExecutor) { + final SubtaskFeedbackKey key = + feedbackKey.withSubTaskIndex(getRuntimeContext().getIndexOfThisSubtask()); + final StreamTask containingTask = getContainingTask(); + + Feedback.registerFeedbackConsumer(key, containingTask, mailboxExecutor, this); + } + + private Configuration getConfiguration() { + Configuration merged = new Configuration(); + + merged.addAll(getContainingTask().getJobConfiguration()); + + GlobalJobParameters globalJobParameters = + getRuntimeContext().getExecutionConfig().getGlobalJobParameters(); + + Preconditions.checkState(globalJobParameters instanceof Configuration); + Configuration configuration = (Configuration) globalJobParameters; + + merged.addAll(configuration); + return merged; + } + + private InternalListState delayedMessagesBufferState( + ListStateDescriptor delayedMessageStateDescriptor) { + try { + KeyedStateBackend keyedStateBackend = getKeyedStateBackend(); + return (InternalListState) + keyedStateBackend.getOrCreateKeyedState( + LongSerializer.INSTANCE, delayedMessageStateDescriptor); + } catch (Exception e) { + throw new RuntimeException("Error registered Flink state for delayed messages buffer.", e); + } + } + + private StatefulFunctionsUniverse statefulFunctionsUniverse(Configuration configuration) { + final ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); + return StatefulFunctionsUniverses.get(classLoader, configuration); + } + + private TypeSerializer messageTypeSerializer( + StatefulFunctionsUniverse statefulFunctionsUniverse) { + TypeInformation info = + new MessageTypeInformation(statefulFunctionsUniverse.messageFactoryType()); + return info.createSerializer(getExecutionConfig()); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/FunctionLoader.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/FunctionLoader.java new file mode 100644 index 00000000..f63e6f74 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/FunctionLoader.java @@ -0,0 +1,25 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.functions; + +import com.ververica.statefun.sdk.FunctionType; +import com.ververica.statefun.sdk.StatefulFunction; + +interface FunctionLoader { + + StatefulFunction load(FunctionType type); +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/FunctionRepository.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/FunctionRepository.java new file mode 100644 index 00000000..d92dc663 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/FunctionRepository.java @@ -0,0 +1,24 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.functions; + +import com.ververica.statefun.sdk.FunctionType; + +public interface FunctionRepository { + + LiveFunction get(FunctionType type); +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/LiveFunction.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/LiveFunction.java new file mode 100644 index 00000000..3acabe45 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/LiveFunction.java @@ -0,0 +1,32 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.functions; + +import com.ververica.statefun.flink.core.message.Message; +import com.ververica.statefun.flink.core.metrics.FunctionTypeMetrics; +import com.ververica.statefun.flink.core.state.BoundState; +import com.ververica.statefun.sdk.Context; +import java.util.Optional; + +interface LiveFunction { + + void receive(Context context, Message message); + + FunctionTypeMetrics metrics(); + + Optional state(); +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/LocalFunctionGroup.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/LocalFunctionGroup.java new file mode 100644 index 00000000..61539b81 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/LocalFunctionGroup.java @@ -0,0 +1,77 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.functions; + +import com.ververica.statefun.flink.core.di.Inject; +import com.ververica.statefun.flink.core.di.Label; +import com.ververica.statefun.flink.core.message.Message; +import com.ververica.statefun.flink.core.pool.SimplePool; +import com.ververica.statefun.sdk.Address; +import it.unimi.dsi.fastutil.objects.ObjectOpenHashMap; +import java.util.ArrayDeque; +import java.util.Objects; + +final class LocalFunctionGroup { + private final ObjectOpenHashMap activeFunctions; + private final ArrayDeque pending; + private final SimplePool pool; + private final FunctionRepository repository; + private final ApplyingContext context; + + @Inject + LocalFunctionGroup( + @Label("function-repository") FunctionRepository repository, + @Label("applying-context") ApplyingContext context) { + this.activeFunctions = new ObjectOpenHashMap<>(); + this.pending = new ArrayDeque<>(); + this.pool = new SimplePool<>(FunctionActivation::new, 1024); + this.repository = Objects.requireNonNull(repository); + this.context = Objects.requireNonNull(context); + } + + void enqueue(Message message) { + FunctionActivation activation = activeFunctions.get(message.target()); + if (activation == null) { + activation = newActivation(message.target()); + pending.addLast(activation); + } + activation.add(message); + } + + boolean processNextEnvelope() { + FunctionActivation activation = pending.pollFirst(); + if (activation == null) { + return false; + } + activation.applyNextPendingEnvelope(context); + if (activation.hasPendingEnvelope()) { + pending.addLast(activation); + } else { + activeFunctions.remove(activation.self()); + activation.setFunction(null, null); + pool.release(activation); + } + return true; + } + + private FunctionActivation newActivation(Address self) { + LiveFunction function = repository.get(self.type()); + FunctionActivation activation = pool.get(); + activation.setFunction(self, function); + return activation; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/LocalSink.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/LocalSink.java new file mode 100644 index 00000000..036cb756 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/LocalSink.java @@ -0,0 +1,37 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.functions; + +import com.ververica.statefun.flink.core.di.Inject; +import com.ververica.statefun.flink.core.di.Label; +import com.ververica.statefun.flink.core.di.Lazy; +import com.ververica.statefun.flink.core.message.Message; +import java.util.Objects; + +final class LocalSink { + private final Lazy functionGroup; + + @Inject + LocalSink(@Label("function-group") Lazy functionGroup) { + this.functionGroup = Objects.requireNonNull(functionGroup); + } + + void accept(Message message) { + Objects.requireNonNull(message); + functionGroup.get().enqueue(message); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/Partition.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/Partition.java new file mode 100644 index 00000000..d5dd8453 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/Partition.java @@ -0,0 +1,40 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.functions; + +import static org.apache.flink.runtime.state.KeyGroupRangeAssignment.assignKeyToParallelOperator; + +import com.ververica.statefun.flink.core.common.KeyBy; +import com.ververica.statefun.sdk.Address; + +class Partition { + private final int maxParallelism; + private final int parallelism; + private final int thisOperatorIndex; + + Partition(int maxParallelism, int parallelism, int thisOperatorIndex) { + this.maxParallelism = maxParallelism; + this.parallelism = parallelism; + this.thisOperatorIndex = thisOperatorIndex; + } + + boolean contains(Address address) { + final int destinationOperatorIndex = + assignKeyToParallelOperator(KeyBy.apply(address), maxParallelism, parallelism); + return thisOperatorIndex == destinationOperatorIndex; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/PredefinedFunctionLoader.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/PredefinedFunctionLoader.java new file mode 100644 index 00000000..696018c8 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/PredefinedFunctionLoader.java @@ -0,0 +1,59 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.functions; + +import com.ververica.statefun.flink.core.common.SetContextClassLoader; +import com.ververica.statefun.flink.core.di.Inject; +import com.ververica.statefun.flink.core.di.Label; +import com.ververica.statefun.sdk.FunctionType; +import com.ververica.statefun.sdk.StatefulFunction; +import com.ververica.statefun.sdk.StatefulFunctionProvider; +import java.util.Map; +import java.util.Objects; + +/** An {@link FunctionLoader} that has a predefined set of {@link StatefulFunctionProvider}s. */ +final class PredefinedFunctionLoader implements FunctionLoader { + private final Map functionProviders; + + @Inject + PredefinedFunctionLoader( + @Label("function-providers") Map functionProviders) { + this.functionProviders = Objects.requireNonNull(functionProviders); + } + + @Override + public StatefulFunction load(FunctionType functionType) { + Objects.requireNonNull(functionType); + StatefulFunctionProvider provider = functionProviders.get(functionType); + if (provider == null) { + throw new IllegalArgumentException("Unknown provider for type " + functionType); + } + StatefulFunction statefulFunction = load(provider, functionType); + if (statefulFunction == null) { + throw new IllegalStateException( + "A provider for a type " + functionType + " has produced a NULL function"); + } + return statefulFunction; + } + + private static StatefulFunction load( + StatefulFunctionProvider provider, FunctionType functionType) { + try (SetContextClassLoader ignored = new SetContextClassLoader(provider)) { + return provider.functionOfType(functionType); + } + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/Reductions.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/Reductions.java new file mode 100644 index 00000000..da06ee28 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/Reductions.java @@ -0,0 +1,147 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.functions; + +import com.ververica.statefun.flink.core.StatefulFunctionsJobConstants; +import com.ververica.statefun.flink.core.StatefulFunctionsUniverse; +import com.ververica.statefun.flink.core.di.Inject; +import com.ververica.statefun.flink.core.di.Lazy; +import com.ververica.statefun.flink.core.di.ObjectContainer; +import com.ververica.statefun.flink.core.message.Message; +import com.ververica.statefun.flink.core.message.MessageFactory; +import com.ververica.statefun.flink.core.metrics.FlinkMetricsFactory; +import com.ververica.statefun.flink.core.metrics.MetricsFactory; +import com.ververica.statefun.flink.core.state.FlinkState; +import com.ververica.statefun.flink.core.state.MultiplexedState; +import com.ververica.statefun.flink.core.state.State; +import com.ververica.statefun.flink.core.state.StateBinder; +import com.ververica.statefun.flink.core.types.DynamicallyRegisteredTypes; +import com.ververica.statefun.sdk.io.EgressIdentifier; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.Executor; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.state.MapState; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.runtime.state.KeyedStateBackend; +import org.apache.flink.runtime.state.internal.InternalListState; +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.util.OutputTag; + +final class Reductions { + private final LocalFunctionGroup localFunctionGroup; + + @Inject + Reductions(LocalFunctionGroup functionGroup) { + this.localFunctionGroup = Objects.requireNonNull(functionGroup); + } + + static Reductions create( + Configuration configuration, + StatefulFunctionsUniverse statefulFunctionsUniverse, + RuntimeContext context, + KeyedStateBackend keyedStateBackend, + TimerServiceFactory timerServiceFactory, + InternalListState delayedMessagesBufferState, + Map, OutputTag> sideOutputs, + Output> output, + MessageFactory messageFactory, + Executor mailboxExecutor, + MetricGroup metricGroup, + MapState asyncOperations, + Executor checkpointLockExecutor) { + + ObjectContainer container = new ObjectContainer(); + + container.add("function-providers", Map.class, statefulFunctionsUniverse.functions()); + container.add( + "function-repository", FunctionRepository.class, StatefulFunctionRepository.class); + + // for FlinkState + container.add("runtime-context", RuntimeContext.class, context); + container.add("keyed-state-backend", KeyedStateBackend.class, keyedStateBackend); + container.add(new DynamicallyRegisteredTypes(statefulFunctionsUniverse.types())); + + if (configuration.getBoolean(StatefulFunctionsJobConstants.MULTIPLEX_FLINK_STATE)) { + container.add("state", State.class, MultiplexedState.class); + } else { + container.add("state", State.class, FlinkState.class); + } + + // For reductions + container.add(messageFactory); + + container.add( + new Partition( + context.getMaxNumberOfParallelSubtasks(), + context.getNumberOfParallelSubtasks(), + context.getIndexOfThisSubtask())); + + container.add(new RemoteSink(output)); + container.add(new SideOutputSink(sideOutputs, output)); + + container.add("applying-context", ApplyingContext.class, ReusableContext.class); + container.add(LocalSink.class); + container.add("function-loader", FunctionLoader.class, PredefinedFunctionLoader.class); + container.add(StateBinder.class); + container.add(Reductions.class); + container.add(LocalFunctionGroup.class); + container.add("metrics-factory", MetricsFactory.class, new FlinkMetricsFactory(metricGroup)); + + // for delayed messages + container.add( + "delayed-messages-buffer-state", InternalListState.class, delayedMessagesBufferState); + container.add( + "delayed-messages-buffer", + DelayedMessagesBuffer.class, + FlinkStateDelayedMessagesBuffer.class); + container.add( + "delayed-messages-timer-service-factory", TimerServiceFactory.class, timerServiceFactory); + container.add(DelaySink.class); + + // lazy providers for the sinks + container.add("function-group", new Lazy<>(LocalFunctionGroup.class)); + container.add("reductions", new Lazy<>(Reductions.class)); + + container.add("mailbox-executor", Executor.class, mailboxExecutor); + + // for the async operations + container.add("async-operations", MapState.class, asyncOperations); + container.add("checkpoint-lock-executor", Executor.class, checkpointLockExecutor); + container.add(AsyncSink.class); + + return container.get(Reductions.class); + } + + void apply(Message message) { + enqueue(message); + processEnvelopes(); + } + + void enqueue(Message message) { + localFunctionGroup.enqueue(message); + } + + @SuppressWarnings("StatementWithEmptyBody") + void processEnvelopes() { + while (localFunctionGroup.processNextEnvelope()) { + // TODO: consider preemption if too many local messages. + } + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/RemoteSink.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/RemoteSink.java new file mode 100644 index 00000000..41975ad8 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/RemoteSink.java @@ -0,0 +1,37 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.functions; + +import com.ververica.statefun.flink.core.message.Message; +import java.util.Objects; +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + +final class RemoteSink { + private final Output> output; + private final StreamRecord record; + + RemoteSink(Output> output) { + this.output = Objects.requireNonNull(output); + this.record = new StreamRecord<>(null); + } + + void accept(Message envelope) { + Objects.requireNonNull(envelope); + output.collect(record.replace(envelope)); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/ReusableContext.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/ReusableContext.java new file mode 100644 index 00000000..a6d62675 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/ReusableContext.java @@ -0,0 +1,126 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.functions; + +import com.ververica.statefun.flink.core.di.Inject; +import com.ververica.statefun.flink.core.di.Label; +import com.ververica.statefun.flink.core.message.Message; +import com.ververica.statefun.flink.core.message.MessageFactory; +import com.ververica.statefun.flink.core.state.State; +import com.ververica.statefun.sdk.Address; +import com.ververica.statefun.sdk.io.EgressIdentifier; +import java.time.Duration; +import java.util.Objects; +import java.util.concurrent.CompletableFuture; + +final class ReusableContext implements ApplyingContext { + private final Partition thisPartition; + private final LocalSink localSink; + private final RemoteSink remoteSink; + private final DelaySink delaySink; + private final AsyncSink asyncSink; + private final SideOutputSink sideOutputSink; + private final State state; + private final MessageFactory messageFactory; + + private Message in; + private LiveFunction function; + + @Inject + ReusableContext( + Partition partition, + LocalSink localSink, + RemoteSink remoteSink, + DelaySink delaySink, + AsyncSink asyncSink, + SideOutputSink sideoutputSink, + @Label("state") State state, + MessageFactory messageFactory) { + + this.thisPartition = Objects.requireNonNull(partition); + this.localSink = Objects.requireNonNull(localSink); + this.remoteSink = Objects.requireNonNull(remoteSink); + this.delaySink = Objects.requireNonNull(delaySink); + this.sideOutputSink = Objects.requireNonNull(sideoutputSink); + this.state = Objects.requireNonNull(state); + this.messageFactory = Objects.requireNonNull(messageFactory); + this.asyncSink = Objects.requireNonNull(asyncSink); + } + + @Override + public void apply(LiveFunction function, Message inMessage) { + this.in = inMessage; + this.function = function; + state.setCurrentKey(inMessage.target()); + function.metrics().incomingMessage(); + function.receive(this, in); + in.postApply(); + this.in = null; + } + + @Override + public void send(Address to, Object what) { + Objects.requireNonNull(to); + Objects.requireNonNull(what); + Message envelope = messageFactory.from(self(), to, what); + if (thisPartition.contains(to)) { + localSink.accept(envelope); + function.metrics().outgoingLocalMessage(); + } else { + remoteSink.accept(envelope); + function.metrics().outgoingRemoteMessage(); + } + } + + @Override + public void send(EgressIdentifier egress, T what) { + Objects.requireNonNull(egress); + Objects.requireNonNull(what); + + function.metrics().outgoingEgressMessage(); + sideOutputSink.accept(egress, what); + } + + @Override + public void sendAfter(Duration delay, Address to, Object message) { + Objects.requireNonNull(delay); + Objects.requireNonNull(to); + Objects.requireNonNull(message); + + Message envelope = messageFactory.from(self(), to, message); + delaySink.accept(envelope, delay.toMillis()); + } + + @Override + public void registerAsyncOperation(M metadata, CompletableFuture future) { + Objects.requireNonNull(metadata); + Objects.requireNonNull(future); + + Message message = messageFactory.from(self(), self(), metadata); + asyncSink.accept(message, future); + } + + @Override + public Address caller() { + return in.source(); + } + + @Override + public Address self() { + return in.target(); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/SideOutputSink.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/SideOutputSink.java new file mode 100644 index 00000000..f8a0d665 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/SideOutputSink.java @@ -0,0 +1,48 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.functions; + +import com.ververica.statefun.sdk.io.EgressIdentifier; +import java.util.Map; +import java.util.Objects; +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.util.OutputTag; + +final class SideOutputSink { + private final Map, OutputTag> outputTags; + private final Output output; + private final StreamRecord record; + + SideOutputSink(Map, OutputTag> outputTags, Output output) { + this.outputTags = Objects.requireNonNull(outputTags); + this.output = Objects.requireNonNull(output); + this.record = new StreamRecord<>(null); + } + + void accept(EgressIdentifier id, T message) { + Objects.requireNonNull(id); + Objects.requireNonNull(message); + + OutputTag tag = outputTags.get(id); + if (tag == null) { + throw new IllegalArgumentException("Unknown egress " + id); + } + record.replace(message); + output.collect(tag, record); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/StatefulFunction.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/StatefulFunction.java new file mode 100644 index 00000000..9748c30b --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/StatefulFunction.java @@ -0,0 +1,69 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.functions; + +import com.ververica.statefun.flink.core.message.Message; +import com.ververica.statefun.flink.core.message.MessageFactory; +import com.ververica.statefun.flink.core.metrics.FunctionTypeMetrics; +import com.ververica.statefun.flink.core.state.BoundState; +import com.ververica.statefun.sdk.Context; +import java.util.Objects; +import java.util.Optional; + +final class StatefulFunction implements LiveFunction { + private final com.ververica.statefun.sdk.StatefulFunction statefulFunction; + private final BoundState state; + private final FunctionTypeMetrics metrics; + private final MessageFactory messageFactory; + + StatefulFunction( + com.ververica.statefun.sdk.StatefulFunction statefulFunction, + BoundState state, + FunctionTypeMetrics metrics, + MessageFactory messageFactory) { + + this.statefulFunction = Objects.requireNonNull(statefulFunction); + this.state = Objects.requireNonNull(state); + this.metrics = Objects.requireNonNull(metrics); + this.messageFactory = Objects.requireNonNull(messageFactory); + } + + @Override + public void receive(Context context, Message message) { + final ClassLoader originalClassLoader = Thread.currentThread().getContextClassLoader(); + try { + ClassLoader targetClassLoader = statefulFunction.getClass().getClassLoader(); + Thread.currentThread().setContextClassLoader(targetClassLoader); + Object payload = message.payload(messageFactory, targetClassLoader); + statefulFunction.invoke(context, payload); + } catch (Exception e) { + throw new StatefulFunctionInvocationException(context.self().type(), e); + } finally { + Thread.currentThread().setContextClassLoader(originalClassLoader); + } + } + + @Override + public FunctionTypeMetrics metrics() { + return metrics; + } + + @Override + public Optional state() { + return Optional.of(state); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/StatefulFunctionInvocationException.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/StatefulFunctionInvocationException.java new file mode 100644 index 00000000..07e2924b --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/StatefulFunctionInvocationException.java @@ -0,0 +1,29 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.functions; + +import com.ververica.statefun.sdk.FunctionType; + +/** A Stateful Functions exception that may be thrown when invoking a function. */ +public final class StatefulFunctionInvocationException extends RuntimeException { + + public StatefulFunctionInvocationException(FunctionType functionType, Throwable cause) { + super( + String.format("An error occurred when attempting to invoke function %s.", functionType), + cause); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/StatefulFunctionRepository.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/StatefulFunctionRepository.java new file mode 100644 index 00000000..04df5612 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/StatefulFunctionRepository.java @@ -0,0 +1,69 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.functions; + +import com.ververica.statefun.flink.core.common.SetContextClassLoader; +import com.ververica.statefun.flink.core.di.Inject; +import com.ververica.statefun.flink.core.di.Label; +import com.ververica.statefun.flink.core.message.MessageFactory; +import com.ververica.statefun.flink.core.metrics.FunctionTypeMetrics; +import com.ververica.statefun.flink.core.metrics.MetricsFactory; +import com.ververica.statefun.flink.core.state.BoundState; +import com.ververica.statefun.flink.core.state.StateBinder; +import com.ververica.statefun.sdk.FunctionType; +import it.unimi.dsi.fastutil.objects.ObjectOpenHashMap; +import java.util.Objects; + +final class StatefulFunctionRepository implements FunctionRepository { + private final ObjectOpenHashMap instances; + private final StateBinder stateBinder; + private final FunctionLoader functionLoader; + private final MetricsFactory metricsFactory; + private final MessageFactory messageFactory; + + @Inject + StatefulFunctionRepository( + @Label("function-loader") FunctionLoader functionLoader, + @Label("metrics-factory") MetricsFactory metricsFactory, + MessageFactory messageFactory, + StateBinder stateBinder) { + this.instances = new ObjectOpenHashMap<>(); + this.stateBinder = Objects.requireNonNull(stateBinder); + this.functionLoader = Objects.requireNonNull(functionLoader); + this.metricsFactory = Objects.requireNonNull(metricsFactory); + this.messageFactory = Objects.requireNonNull(messageFactory); + } + + @Override + public LiveFunction get(FunctionType type) { + StatefulFunction function = instances.get(type); + if (function == null) { + instances.put(type, function = load(type)); + } + return function; + } + + private StatefulFunction load(FunctionType functionType) { + com.ververica.statefun.sdk.StatefulFunction statefulFunction = + functionLoader.load(functionType); + try (SetContextClassLoader ignored = new SetContextClassLoader(statefulFunction)) { + BoundState state = stateBinder.bind(functionType, statefulFunction); + FunctionTypeMetrics metrics = metricsFactory.forType(functionType); + return new StatefulFunction(statefulFunction, state, metrics, messageFactory); + } + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/TimerServiceFactory.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/TimerServiceFactory.java new file mode 100644 index 00000000..58fe0fbc --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/TimerServiceFactory.java @@ -0,0 +1,26 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.functions; + +import org.apache.flink.runtime.state.VoidNamespace; +import org.apache.flink.streaming.api.operators.InternalTimerService; +import org.apache.flink.streaming.api.operators.Triggerable; + +interface TimerServiceFactory { + InternalTimerService createTimerService( + Triggerable triggerable); +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/UnderCheckpointLockExecutor.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/UnderCheckpointLockExecutor.java new file mode 100644 index 00000000..1c2734e6 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/functions/UnderCheckpointLockExecutor.java @@ -0,0 +1,61 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.functions; + +import java.util.Objects; +import java.util.concurrent.Executor; +import java.util.function.BooleanSupplier; +import javax.annotation.Nonnull; +import org.apache.flink.streaming.runtime.tasks.AsyncExceptionHandler; + +/** + * An executor that is meant to be used to run commands on background threads but still get the + * effect as if it is executed by the {@link + * org.apache.flink.streaming.api.operators.AbstractStreamOperator}. Tasks executed via this + * executor first obtain the containing {@link org.apache.flink.streaming.runtime.tasks.StreamTask} + * checkpoint lock, then making sure that the operator was not yet canceled. In addition it also + * makes sure to propogate any exceptions thrown to the main thread via {@link + * AsyncExceptionHandler}. + * + *

NOTE: that this executor would execute the given command on the caller's thread. + */ +final class UnderCheckpointLockExecutor implements Executor { + private final Object checkpointLock; + private final BooleanSupplier closed; + private final AsyncExceptionHandler asyncExceptionHandler; + + UnderCheckpointLockExecutor( + Object checkpointLock, BooleanSupplier closed, AsyncExceptionHandler asyncExceptionHandler) { + this.checkpointLock = Objects.requireNonNull(checkpointLock); + this.closed = Objects.requireNonNull(closed); + this.asyncExceptionHandler = Objects.requireNonNull(asyncExceptionHandler); + } + + @Override + public void execute(@Nonnull Runnable command) { + synchronized (checkpointLock) { + if (closed.getAsBoolean()) { + return; + } + try { + command.run(); + } catch (Throwable t) { + asyncExceptionHandler.handleAsyncException("Asynchronous failure", t); + } + } + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/Checkpoint.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/Checkpoint.java new file mode 100644 index 00000000..949bd25b --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/Checkpoint.java @@ -0,0 +1,468 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/stateful-functions.proto + +package com.ververica.statefun.flink.core.generated; + +/** Protobuf type {@code com.ververica.statefun.flink.core.Checkpoint} */ +public final class Checkpoint extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.flink.core.Checkpoint) + CheckpointOrBuilder { + private static final long serialVersionUID = 0L; + // Use Checkpoint.newBuilder() to construct. + private Checkpoint(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private Checkpoint() {} + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private Checkpoint( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 8: + { + checkpointId_ = input.readInt64(); + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.flink.core.generated.StatefulFunctions + .internal_static_com_ververica_statefun_flink_core_Checkpoint_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.flink.core.generated.StatefulFunctions + .internal_static_com_ververica_statefun_flink_core_Checkpoint_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.flink.core.generated.Checkpoint.class, + com.ververica.statefun.flink.core.generated.Checkpoint.Builder.class); + } + + public static final int CHECKPOINT_ID_FIELD_NUMBER = 1; + private long checkpointId_; + /** int64 checkpoint_id = 1; */ + public long getCheckpointId() { + return checkpointId_; + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (checkpointId_ != 0L) { + output.writeInt64(1, checkpointId_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (checkpointId_ != 0L) { + size += com.google.protobuf.CodedOutputStream.computeInt64Size(1, checkpointId_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.ververica.statefun.flink.core.generated.Checkpoint)) { + return super.equals(obj); + } + com.ververica.statefun.flink.core.generated.Checkpoint other = + (com.ververica.statefun.flink.core.generated.Checkpoint) obj; + + if (getCheckpointId() != other.getCheckpointId()) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + CHECKPOINT_ID_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong(getCheckpointId()); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.flink.core.generated.Checkpoint parseFrom( + java.nio.ByteBuffer data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.flink.core.generated.Checkpoint parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.flink.core.generated.Checkpoint parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.flink.core.generated.Checkpoint parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.flink.core.generated.Checkpoint parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.flink.core.generated.Checkpoint parseFrom( + byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.flink.core.generated.Checkpoint parseFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.flink.core.generated.Checkpoint parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.flink.core.generated.Checkpoint parseDelimitedFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.flink.core.generated.Checkpoint parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.flink.core.generated.Checkpoint parseFrom( + com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.flink.core.generated.Checkpoint parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.flink.core.generated.Checkpoint prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code com.ververica.statefun.flink.core.Checkpoint} */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.flink.core.Checkpoint) + com.ververica.statefun.flink.core.generated.CheckpointOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.flink.core.generated.StatefulFunctions + .internal_static_com_ververica_statefun_flink_core_Checkpoint_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.flink.core.generated.StatefulFunctions + .internal_static_com_ververica_statefun_flink_core_Checkpoint_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.flink.core.generated.Checkpoint.class, + com.ververica.statefun.flink.core.generated.Checkpoint.Builder.class); + } + + // Construct using com.ververica.statefun.flink.core.generated.Checkpoint.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + checkpointId_ = 0L; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.flink.core.generated.StatefulFunctions + .internal_static_com_ververica_statefun_flink_core_Checkpoint_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.flink.core.generated.Checkpoint getDefaultInstanceForType() { + return com.ververica.statefun.flink.core.generated.Checkpoint.getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.flink.core.generated.Checkpoint build() { + com.ververica.statefun.flink.core.generated.Checkpoint result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.flink.core.generated.Checkpoint buildPartial() { + com.ververica.statefun.flink.core.generated.Checkpoint result = + new com.ververica.statefun.flink.core.generated.Checkpoint(this); + result.checkpointId_ = checkpointId_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.ververica.statefun.flink.core.generated.Checkpoint) { + return mergeFrom((com.ververica.statefun.flink.core.generated.Checkpoint) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.ververica.statefun.flink.core.generated.Checkpoint other) { + if (other == com.ververica.statefun.flink.core.generated.Checkpoint.getDefaultInstance()) + return this; + if (other.getCheckpointId() != 0L) { + setCheckpointId(other.getCheckpointId()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.flink.core.generated.Checkpoint parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.flink.core.generated.Checkpoint) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private long checkpointId_; + /** int64 checkpoint_id = 1; */ + public long getCheckpointId() { + return checkpointId_; + } + /** int64 checkpoint_id = 1; */ + public Builder setCheckpointId(long value) { + + checkpointId_ = value; + onChanged(); + return this; + } + /** int64 checkpoint_id = 1; */ + public Builder clearCheckpointId() { + + checkpointId_ = 0L; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.flink.core.Checkpoint) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.flink.core.Checkpoint) + private static final com.ververica.statefun.flink.core.generated.Checkpoint DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = new com.ververica.statefun.flink.core.generated.Checkpoint(); + } + + public static com.ververica.statefun.flink.core.generated.Checkpoint getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public Checkpoint parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new Checkpoint(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.flink.core.generated.Checkpoint getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/CheckpointOrBuilder.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/CheckpointOrBuilder.java new file mode 100644 index 00000000..c55bdc45 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/CheckpointOrBuilder.java @@ -0,0 +1,13 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/stateful-functions.proto + +package com.ververica.statefun.flink.core.generated; + +public interface CheckpointOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.flink.core.Checkpoint) + com.google.protobuf.MessageOrBuilder { + + /** int64 checkpoint_id = 1; */ + long getCheckpointId(); +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/Envelope.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/Envelope.java new file mode 100644 index 00000000..aa6d849d --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/Envelope.java @@ -0,0 +1,1231 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/stateful-functions.proto + +package com.ververica.statefun.flink.core.generated; + +/** Protobuf type {@code com.ververica.statefun.flink.core.Envelope} */ +public final class Envelope extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.flink.core.Envelope) + EnvelopeOrBuilder { + private static final long serialVersionUID = 0L; + // Use Envelope.newBuilder() to construct. + private Envelope(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private Envelope() {} + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private Envelope( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + com.ververica.statefun.flink.core.generated.EnvelopeAddress.Builder subBuilder = null; + if (source_ != null) { + subBuilder = source_.toBuilder(); + } + source_ = + input.readMessage( + com.ververica.statefun.flink.core.generated.EnvelopeAddress.parser(), + extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(source_); + source_ = subBuilder.buildPartial(); + } + + break; + } + case 18: + { + com.ververica.statefun.flink.core.generated.EnvelopeAddress.Builder subBuilder = null; + if (target_ != null) { + subBuilder = target_.toBuilder(); + } + target_ = + input.readMessage( + com.ververica.statefun.flink.core.generated.EnvelopeAddress.parser(), + extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(target_); + target_ = subBuilder.buildPartial(); + } + + break; + } + case 26: + { + com.ververica.statefun.flink.core.generated.Payload.Builder subBuilder = null; + if (bodyCase_ == 3) { + subBuilder = + ((com.ververica.statefun.flink.core.generated.Payload) body_).toBuilder(); + } + body_ = + input.readMessage( + com.ververica.statefun.flink.core.generated.Payload.parser(), + extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom((com.ververica.statefun.flink.core.generated.Payload) body_); + body_ = subBuilder.buildPartial(); + } + bodyCase_ = 3; + break; + } + case 34: + { + com.ververica.statefun.flink.core.generated.Checkpoint.Builder subBuilder = null; + if (bodyCase_ == 4) { + subBuilder = + ((com.ververica.statefun.flink.core.generated.Checkpoint) body_).toBuilder(); + } + body_ = + input.readMessage( + com.ververica.statefun.flink.core.generated.Checkpoint.parser(), + extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom( + (com.ververica.statefun.flink.core.generated.Checkpoint) body_); + body_ = subBuilder.buildPartial(); + } + bodyCase_ = 4; + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.flink.core.generated.StatefulFunctions + .internal_static_com_ververica_statefun_flink_core_Envelope_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.flink.core.generated.StatefulFunctions + .internal_static_com_ververica_statefun_flink_core_Envelope_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.flink.core.generated.Envelope.class, + com.ververica.statefun.flink.core.generated.Envelope.Builder.class); + } + + private int bodyCase_ = 0; + private java.lang.Object body_; + + public enum BodyCase implements com.google.protobuf.Internal.EnumLite { + CHECKPOINT(4), + PAYLOAD(3), + BODY_NOT_SET(0); + private final int value; + + private BodyCase(int value) { + this.value = value; + } + /** @deprecated Use {@link #forNumber(int)} instead. */ + @java.lang.Deprecated + public static BodyCase valueOf(int value) { + return forNumber(value); + } + + public static BodyCase forNumber(int value) { + switch (value) { + case 4: + return CHECKPOINT; + case 3: + return PAYLOAD; + case 0: + return BODY_NOT_SET; + default: + return null; + } + } + + public int getNumber() { + return this.value; + } + }; + + public BodyCase getBodyCase() { + return BodyCase.forNumber(bodyCase_); + } + + public static final int SOURCE_FIELD_NUMBER = 1; + private com.ververica.statefun.flink.core.generated.EnvelopeAddress source_; + /** .com.ververica.statefun.flink.core.EnvelopeAddress source = 1; */ + public boolean hasSource() { + return source_ != null; + } + /** .com.ververica.statefun.flink.core.EnvelopeAddress source = 1; */ + public com.ververica.statefun.flink.core.generated.EnvelopeAddress getSource() { + return source_ == null + ? com.ververica.statefun.flink.core.generated.EnvelopeAddress.getDefaultInstance() + : source_; + } + /** .com.ververica.statefun.flink.core.EnvelopeAddress source = 1; */ + public com.ververica.statefun.flink.core.generated.EnvelopeAddressOrBuilder getSourceOrBuilder() { + return getSource(); + } + + public static final int TARGET_FIELD_NUMBER = 2; + private com.ververica.statefun.flink.core.generated.EnvelopeAddress target_; + /** .com.ververica.statefun.flink.core.EnvelopeAddress target = 2; */ + public boolean hasTarget() { + return target_ != null; + } + /** .com.ververica.statefun.flink.core.EnvelopeAddress target = 2; */ + public com.ververica.statefun.flink.core.generated.EnvelopeAddress getTarget() { + return target_ == null + ? com.ververica.statefun.flink.core.generated.EnvelopeAddress.getDefaultInstance() + : target_; + } + /** .com.ververica.statefun.flink.core.EnvelopeAddress target = 2; */ + public com.ververica.statefun.flink.core.generated.EnvelopeAddressOrBuilder getTargetOrBuilder() { + return getTarget(); + } + + public static final int CHECKPOINT_FIELD_NUMBER = 4; + /** .com.ververica.statefun.flink.core.Checkpoint checkpoint = 4; */ + public boolean hasCheckpoint() { + return bodyCase_ == 4; + } + /** .com.ververica.statefun.flink.core.Checkpoint checkpoint = 4; */ + public com.ververica.statefun.flink.core.generated.Checkpoint getCheckpoint() { + if (bodyCase_ == 4) { + return (com.ververica.statefun.flink.core.generated.Checkpoint) body_; + } + return com.ververica.statefun.flink.core.generated.Checkpoint.getDefaultInstance(); + } + /** .com.ververica.statefun.flink.core.Checkpoint checkpoint = 4; */ + public com.ververica.statefun.flink.core.generated.CheckpointOrBuilder getCheckpointOrBuilder() { + if (bodyCase_ == 4) { + return (com.ververica.statefun.flink.core.generated.Checkpoint) body_; + } + return com.ververica.statefun.flink.core.generated.Checkpoint.getDefaultInstance(); + } + + public static final int PAYLOAD_FIELD_NUMBER = 3; + /** .com.ververica.statefun.flink.core.Payload payload = 3; */ + public boolean hasPayload() { + return bodyCase_ == 3; + } + /** .com.ververica.statefun.flink.core.Payload payload = 3; */ + public com.ververica.statefun.flink.core.generated.Payload getPayload() { + if (bodyCase_ == 3) { + return (com.ververica.statefun.flink.core.generated.Payload) body_; + } + return com.ververica.statefun.flink.core.generated.Payload.getDefaultInstance(); + } + /** .com.ververica.statefun.flink.core.Payload payload = 3; */ + public com.ververica.statefun.flink.core.generated.PayloadOrBuilder getPayloadOrBuilder() { + if (bodyCase_ == 3) { + return (com.ververica.statefun.flink.core.generated.Payload) body_; + } + return com.ververica.statefun.flink.core.generated.Payload.getDefaultInstance(); + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (source_ != null) { + output.writeMessage(1, getSource()); + } + if (target_ != null) { + output.writeMessage(2, getTarget()); + } + if (bodyCase_ == 3) { + output.writeMessage(3, (com.ververica.statefun.flink.core.generated.Payload) body_); + } + if (bodyCase_ == 4) { + output.writeMessage(4, (com.ververica.statefun.flink.core.generated.Checkpoint) body_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (source_ != null) { + size += com.google.protobuf.CodedOutputStream.computeMessageSize(1, getSource()); + } + if (target_ != null) { + size += com.google.protobuf.CodedOutputStream.computeMessageSize(2, getTarget()); + } + if (bodyCase_ == 3) { + size += + com.google.protobuf.CodedOutputStream.computeMessageSize( + 3, (com.ververica.statefun.flink.core.generated.Payload) body_); + } + if (bodyCase_ == 4) { + size += + com.google.protobuf.CodedOutputStream.computeMessageSize( + 4, (com.ververica.statefun.flink.core.generated.Checkpoint) body_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.ververica.statefun.flink.core.generated.Envelope)) { + return super.equals(obj); + } + com.ververica.statefun.flink.core.generated.Envelope other = + (com.ververica.statefun.flink.core.generated.Envelope) obj; + + if (hasSource() != other.hasSource()) return false; + if (hasSource()) { + if (!getSource().equals(other.getSource())) return false; + } + if (hasTarget() != other.hasTarget()) return false; + if (hasTarget()) { + if (!getTarget().equals(other.getTarget())) return false; + } + if (!getBodyCase().equals(other.getBodyCase())) return false; + switch (bodyCase_) { + case 4: + if (!getCheckpoint().equals(other.getCheckpoint())) return false; + break; + case 3: + if (!getPayload().equals(other.getPayload())) return false; + break; + case 0: + default: + } + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasSource()) { + hash = (37 * hash) + SOURCE_FIELD_NUMBER; + hash = (53 * hash) + getSource().hashCode(); + } + if (hasTarget()) { + hash = (37 * hash) + TARGET_FIELD_NUMBER; + hash = (53 * hash) + getTarget().hashCode(); + } + switch (bodyCase_) { + case 4: + hash = (37 * hash) + CHECKPOINT_FIELD_NUMBER; + hash = (53 * hash) + getCheckpoint().hashCode(); + break; + case 3: + hash = (37 * hash) + PAYLOAD_FIELD_NUMBER; + hash = (53 * hash) + getPayload().hashCode(); + break; + case 0: + default: + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.flink.core.generated.Envelope parseFrom( + java.nio.ByteBuffer data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.flink.core.generated.Envelope parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.flink.core.generated.Envelope parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.flink.core.generated.Envelope parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.flink.core.generated.Envelope parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.flink.core.generated.Envelope parseFrom( + byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.flink.core.generated.Envelope parseFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.flink.core.generated.Envelope parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.flink.core.generated.Envelope parseDelimitedFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.flink.core.generated.Envelope parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.flink.core.generated.Envelope parseFrom( + com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.flink.core.generated.Envelope parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder(com.ververica.statefun.flink.core.generated.Envelope prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code com.ververica.statefun.flink.core.Envelope} */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.flink.core.Envelope) + com.ververica.statefun.flink.core.generated.EnvelopeOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.flink.core.generated.StatefulFunctions + .internal_static_com_ververica_statefun_flink_core_Envelope_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.flink.core.generated.StatefulFunctions + .internal_static_com_ververica_statefun_flink_core_Envelope_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.flink.core.generated.Envelope.class, + com.ververica.statefun.flink.core.generated.Envelope.Builder.class); + } + + // Construct using com.ververica.statefun.flink.core.generated.Envelope.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + if (sourceBuilder_ == null) { + source_ = null; + } else { + source_ = null; + sourceBuilder_ = null; + } + if (targetBuilder_ == null) { + target_ = null; + } else { + target_ = null; + targetBuilder_ = null; + } + bodyCase_ = 0; + body_ = null; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.flink.core.generated.StatefulFunctions + .internal_static_com_ververica_statefun_flink_core_Envelope_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.flink.core.generated.Envelope getDefaultInstanceForType() { + return com.ververica.statefun.flink.core.generated.Envelope.getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.flink.core.generated.Envelope build() { + com.ververica.statefun.flink.core.generated.Envelope result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.flink.core.generated.Envelope buildPartial() { + com.ververica.statefun.flink.core.generated.Envelope result = + new com.ververica.statefun.flink.core.generated.Envelope(this); + if (sourceBuilder_ == null) { + result.source_ = source_; + } else { + result.source_ = sourceBuilder_.build(); + } + if (targetBuilder_ == null) { + result.target_ = target_; + } else { + result.target_ = targetBuilder_.build(); + } + if (bodyCase_ == 4) { + if (checkpointBuilder_ == null) { + result.body_ = body_; + } else { + result.body_ = checkpointBuilder_.build(); + } + } + if (bodyCase_ == 3) { + if (payloadBuilder_ == null) { + result.body_ = body_; + } else { + result.body_ = payloadBuilder_.build(); + } + } + result.bodyCase_ = bodyCase_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.ververica.statefun.flink.core.generated.Envelope) { + return mergeFrom((com.ververica.statefun.flink.core.generated.Envelope) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.ververica.statefun.flink.core.generated.Envelope other) { + if (other == com.ververica.statefun.flink.core.generated.Envelope.getDefaultInstance()) + return this; + if (other.hasSource()) { + mergeSource(other.getSource()); + } + if (other.hasTarget()) { + mergeTarget(other.getTarget()); + } + switch (other.getBodyCase()) { + case CHECKPOINT: + { + mergeCheckpoint(other.getCheckpoint()); + break; + } + case PAYLOAD: + { + mergePayload(other.getPayload()); + break; + } + case BODY_NOT_SET: + { + break; + } + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.flink.core.generated.Envelope parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.flink.core.generated.Envelope) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int bodyCase_ = 0; + private java.lang.Object body_; + + public BodyCase getBodyCase() { + return BodyCase.forNumber(bodyCase_); + } + + public Builder clearBody() { + bodyCase_ = 0; + body_ = null; + onChanged(); + return this; + } + + private com.ververica.statefun.flink.core.generated.EnvelopeAddress source_; + private com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.flink.core.generated.EnvelopeAddress, + com.ververica.statefun.flink.core.generated.EnvelopeAddress.Builder, + com.ververica.statefun.flink.core.generated.EnvelopeAddressOrBuilder> + sourceBuilder_; + /** .com.ververica.statefun.flink.core.EnvelopeAddress source = 1; */ + public boolean hasSource() { + return sourceBuilder_ != null || source_ != null; + } + /** .com.ververica.statefun.flink.core.EnvelopeAddress source = 1; */ + public com.ververica.statefun.flink.core.generated.EnvelopeAddress getSource() { + if (sourceBuilder_ == null) { + return source_ == null + ? com.ververica.statefun.flink.core.generated.EnvelopeAddress.getDefaultInstance() + : source_; + } else { + return sourceBuilder_.getMessage(); + } + } + /** .com.ververica.statefun.flink.core.EnvelopeAddress source = 1; */ + public Builder setSource(com.ververica.statefun.flink.core.generated.EnvelopeAddress value) { + if (sourceBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + source_ = value; + onChanged(); + } else { + sourceBuilder_.setMessage(value); + } + + return this; + } + /** .com.ververica.statefun.flink.core.EnvelopeAddress source = 1; */ + public Builder setSource( + com.ververica.statefun.flink.core.generated.EnvelopeAddress.Builder builderForValue) { + if (sourceBuilder_ == null) { + source_ = builderForValue.build(); + onChanged(); + } else { + sourceBuilder_.setMessage(builderForValue.build()); + } + + return this; + } + /** .com.ververica.statefun.flink.core.EnvelopeAddress source = 1; */ + public Builder mergeSource(com.ververica.statefun.flink.core.generated.EnvelopeAddress value) { + if (sourceBuilder_ == null) { + if (source_ != null) { + source_ = + com.ververica.statefun.flink.core.generated.EnvelopeAddress.newBuilder(source_) + .mergeFrom(value) + .buildPartial(); + } else { + source_ = value; + } + onChanged(); + } else { + sourceBuilder_.mergeFrom(value); + } + + return this; + } + /** .com.ververica.statefun.flink.core.EnvelopeAddress source = 1; */ + public Builder clearSource() { + if (sourceBuilder_ == null) { + source_ = null; + onChanged(); + } else { + source_ = null; + sourceBuilder_ = null; + } + + return this; + } + /** .com.ververica.statefun.flink.core.EnvelopeAddress source = 1; */ + public com.ververica.statefun.flink.core.generated.EnvelopeAddress.Builder getSourceBuilder() { + + onChanged(); + return getSourceFieldBuilder().getBuilder(); + } + /** .com.ververica.statefun.flink.core.EnvelopeAddress source = 1; */ + public com.ververica.statefun.flink.core.generated.EnvelopeAddressOrBuilder + getSourceOrBuilder() { + if (sourceBuilder_ != null) { + return sourceBuilder_.getMessageOrBuilder(); + } else { + return source_ == null + ? com.ververica.statefun.flink.core.generated.EnvelopeAddress.getDefaultInstance() + : source_; + } + } + /** .com.ververica.statefun.flink.core.EnvelopeAddress source = 1; */ + private com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.flink.core.generated.EnvelopeAddress, + com.ververica.statefun.flink.core.generated.EnvelopeAddress.Builder, + com.ververica.statefun.flink.core.generated.EnvelopeAddressOrBuilder> + getSourceFieldBuilder() { + if (sourceBuilder_ == null) { + sourceBuilder_ = + new com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.flink.core.generated.EnvelopeAddress, + com.ververica.statefun.flink.core.generated.EnvelopeAddress.Builder, + com.ververica.statefun.flink.core.generated.EnvelopeAddressOrBuilder>( + getSource(), getParentForChildren(), isClean()); + source_ = null; + } + return sourceBuilder_; + } + + private com.ververica.statefun.flink.core.generated.EnvelopeAddress target_; + private com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.flink.core.generated.EnvelopeAddress, + com.ververica.statefun.flink.core.generated.EnvelopeAddress.Builder, + com.ververica.statefun.flink.core.generated.EnvelopeAddressOrBuilder> + targetBuilder_; + /** .com.ververica.statefun.flink.core.EnvelopeAddress target = 2; */ + public boolean hasTarget() { + return targetBuilder_ != null || target_ != null; + } + /** .com.ververica.statefun.flink.core.EnvelopeAddress target = 2; */ + public com.ververica.statefun.flink.core.generated.EnvelopeAddress getTarget() { + if (targetBuilder_ == null) { + return target_ == null + ? com.ververica.statefun.flink.core.generated.EnvelopeAddress.getDefaultInstance() + : target_; + } else { + return targetBuilder_.getMessage(); + } + } + /** .com.ververica.statefun.flink.core.EnvelopeAddress target = 2; */ + public Builder setTarget(com.ververica.statefun.flink.core.generated.EnvelopeAddress value) { + if (targetBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + target_ = value; + onChanged(); + } else { + targetBuilder_.setMessage(value); + } + + return this; + } + /** .com.ververica.statefun.flink.core.EnvelopeAddress target = 2; */ + public Builder setTarget( + com.ververica.statefun.flink.core.generated.EnvelopeAddress.Builder builderForValue) { + if (targetBuilder_ == null) { + target_ = builderForValue.build(); + onChanged(); + } else { + targetBuilder_.setMessage(builderForValue.build()); + } + + return this; + } + /** .com.ververica.statefun.flink.core.EnvelopeAddress target = 2; */ + public Builder mergeTarget(com.ververica.statefun.flink.core.generated.EnvelopeAddress value) { + if (targetBuilder_ == null) { + if (target_ != null) { + target_ = + com.ververica.statefun.flink.core.generated.EnvelopeAddress.newBuilder(target_) + .mergeFrom(value) + .buildPartial(); + } else { + target_ = value; + } + onChanged(); + } else { + targetBuilder_.mergeFrom(value); + } + + return this; + } + /** .com.ververica.statefun.flink.core.EnvelopeAddress target = 2; */ + public Builder clearTarget() { + if (targetBuilder_ == null) { + target_ = null; + onChanged(); + } else { + target_ = null; + targetBuilder_ = null; + } + + return this; + } + /** .com.ververica.statefun.flink.core.EnvelopeAddress target = 2; */ + public com.ververica.statefun.flink.core.generated.EnvelopeAddress.Builder getTargetBuilder() { + + onChanged(); + return getTargetFieldBuilder().getBuilder(); + } + /** .com.ververica.statefun.flink.core.EnvelopeAddress target = 2; */ + public com.ververica.statefun.flink.core.generated.EnvelopeAddressOrBuilder + getTargetOrBuilder() { + if (targetBuilder_ != null) { + return targetBuilder_.getMessageOrBuilder(); + } else { + return target_ == null + ? com.ververica.statefun.flink.core.generated.EnvelopeAddress.getDefaultInstance() + : target_; + } + } + /** .com.ververica.statefun.flink.core.EnvelopeAddress target = 2; */ + private com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.flink.core.generated.EnvelopeAddress, + com.ververica.statefun.flink.core.generated.EnvelopeAddress.Builder, + com.ververica.statefun.flink.core.generated.EnvelopeAddressOrBuilder> + getTargetFieldBuilder() { + if (targetBuilder_ == null) { + targetBuilder_ = + new com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.flink.core.generated.EnvelopeAddress, + com.ververica.statefun.flink.core.generated.EnvelopeAddress.Builder, + com.ververica.statefun.flink.core.generated.EnvelopeAddressOrBuilder>( + getTarget(), getParentForChildren(), isClean()); + target_ = null; + } + return targetBuilder_; + } + + private com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.flink.core.generated.Checkpoint, + com.ververica.statefun.flink.core.generated.Checkpoint.Builder, + com.ververica.statefun.flink.core.generated.CheckpointOrBuilder> + checkpointBuilder_; + /** .com.ververica.statefun.flink.core.Checkpoint checkpoint = 4; */ + public boolean hasCheckpoint() { + return bodyCase_ == 4; + } + /** .com.ververica.statefun.flink.core.Checkpoint checkpoint = 4; */ + public com.ververica.statefun.flink.core.generated.Checkpoint getCheckpoint() { + if (checkpointBuilder_ == null) { + if (bodyCase_ == 4) { + return (com.ververica.statefun.flink.core.generated.Checkpoint) body_; + } + return com.ververica.statefun.flink.core.generated.Checkpoint.getDefaultInstance(); + } else { + if (bodyCase_ == 4) { + return checkpointBuilder_.getMessage(); + } + return com.ververica.statefun.flink.core.generated.Checkpoint.getDefaultInstance(); + } + } + /** .com.ververica.statefun.flink.core.Checkpoint checkpoint = 4; */ + public Builder setCheckpoint(com.ververica.statefun.flink.core.generated.Checkpoint value) { + if (checkpointBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + body_ = value; + onChanged(); + } else { + checkpointBuilder_.setMessage(value); + } + bodyCase_ = 4; + return this; + } + /** .com.ververica.statefun.flink.core.Checkpoint checkpoint = 4; */ + public Builder setCheckpoint( + com.ververica.statefun.flink.core.generated.Checkpoint.Builder builderForValue) { + if (checkpointBuilder_ == null) { + body_ = builderForValue.build(); + onChanged(); + } else { + checkpointBuilder_.setMessage(builderForValue.build()); + } + bodyCase_ = 4; + return this; + } + /** .com.ververica.statefun.flink.core.Checkpoint checkpoint = 4; */ + public Builder mergeCheckpoint(com.ververica.statefun.flink.core.generated.Checkpoint value) { + if (checkpointBuilder_ == null) { + if (bodyCase_ == 4 + && body_ + != com.ververica.statefun.flink.core.generated.Checkpoint.getDefaultInstance()) { + body_ = + com.ververica.statefun.flink.core.generated.Checkpoint.newBuilder( + (com.ververica.statefun.flink.core.generated.Checkpoint) body_) + .mergeFrom(value) + .buildPartial(); + } else { + body_ = value; + } + onChanged(); + } else { + if (bodyCase_ == 4) { + checkpointBuilder_.mergeFrom(value); + } + checkpointBuilder_.setMessage(value); + } + bodyCase_ = 4; + return this; + } + /** .com.ververica.statefun.flink.core.Checkpoint checkpoint = 4; */ + public Builder clearCheckpoint() { + if (checkpointBuilder_ == null) { + if (bodyCase_ == 4) { + bodyCase_ = 0; + body_ = null; + onChanged(); + } + } else { + if (bodyCase_ == 4) { + bodyCase_ = 0; + body_ = null; + } + checkpointBuilder_.clear(); + } + return this; + } + /** .com.ververica.statefun.flink.core.Checkpoint checkpoint = 4; */ + public com.ververica.statefun.flink.core.generated.Checkpoint.Builder getCheckpointBuilder() { + return getCheckpointFieldBuilder().getBuilder(); + } + /** .com.ververica.statefun.flink.core.Checkpoint checkpoint = 4; */ + public com.ververica.statefun.flink.core.generated.CheckpointOrBuilder + getCheckpointOrBuilder() { + if ((bodyCase_ == 4) && (checkpointBuilder_ != null)) { + return checkpointBuilder_.getMessageOrBuilder(); + } else { + if (bodyCase_ == 4) { + return (com.ververica.statefun.flink.core.generated.Checkpoint) body_; + } + return com.ververica.statefun.flink.core.generated.Checkpoint.getDefaultInstance(); + } + } + /** .com.ververica.statefun.flink.core.Checkpoint checkpoint = 4; */ + private com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.flink.core.generated.Checkpoint, + com.ververica.statefun.flink.core.generated.Checkpoint.Builder, + com.ververica.statefun.flink.core.generated.CheckpointOrBuilder> + getCheckpointFieldBuilder() { + if (checkpointBuilder_ == null) { + if (!(bodyCase_ == 4)) { + body_ = com.ververica.statefun.flink.core.generated.Checkpoint.getDefaultInstance(); + } + checkpointBuilder_ = + new com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.flink.core.generated.Checkpoint, + com.ververica.statefun.flink.core.generated.Checkpoint.Builder, + com.ververica.statefun.flink.core.generated.CheckpointOrBuilder>( + (com.ververica.statefun.flink.core.generated.Checkpoint) body_, + getParentForChildren(), + isClean()); + body_ = null; + } + bodyCase_ = 4; + onChanged(); + ; + return checkpointBuilder_; + } + + private com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.flink.core.generated.Payload, + com.ververica.statefun.flink.core.generated.Payload.Builder, + com.ververica.statefun.flink.core.generated.PayloadOrBuilder> + payloadBuilder_; + /** .com.ververica.statefun.flink.core.Payload payload = 3; */ + public boolean hasPayload() { + return bodyCase_ == 3; + } + /** .com.ververica.statefun.flink.core.Payload payload = 3; */ + public com.ververica.statefun.flink.core.generated.Payload getPayload() { + if (payloadBuilder_ == null) { + if (bodyCase_ == 3) { + return (com.ververica.statefun.flink.core.generated.Payload) body_; + } + return com.ververica.statefun.flink.core.generated.Payload.getDefaultInstance(); + } else { + if (bodyCase_ == 3) { + return payloadBuilder_.getMessage(); + } + return com.ververica.statefun.flink.core.generated.Payload.getDefaultInstance(); + } + } + /** .com.ververica.statefun.flink.core.Payload payload = 3; */ + public Builder setPayload(com.ververica.statefun.flink.core.generated.Payload value) { + if (payloadBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + body_ = value; + onChanged(); + } else { + payloadBuilder_.setMessage(value); + } + bodyCase_ = 3; + return this; + } + /** .com.ververica.statefun.flink.core.Payload payload = 3; */ + public Builder setPayload( + com.ververica.statefun.flink.core.generated.Payload.Builder builderForValue) { + if (payloadBuilder_ == null) { + body_ = builderForValue.build(); + onChanged(); + } else { + payloadBuilder_.setMessage(builderForValue.build()); + } + bodyCase_ = 3; + return this; + } + /** .com.ververica.statefun.flink.core.Payload payload = 3; */ + public Builder mergePayload(com.ververica.statefun.flink.core.generated.Payload value) { + if (payloadBuilder_ == null) { + if (bodyCase_ == 3 + && body_ != com.ververica.statefun.flink.core.generated.Payload.getDefaultInstance()) { + body_ = + com.ververica.statefun.flink.core.generated.Payload.newBuilder( + (com.ververica.statefun.flink.core.generated.Payload) body_) + .mergeFrom(value) + .buildPartial(); + } else { + body_ = value; + } + onChanged(); + } else { + if (bodyCase_ == 3) { + payloadBuilder_.mergeFrom(value); + } + payloadBuilder_.setMessage(value); + } + bodyCase_ = 3; + return this; + } + /** .com.ververica.statefun.flink.core.Payload payload = 3; */ + public Builder clearPayload() { + if (payloadBuilder_ == null) { + if (bodyCase_ == 3) { + bodyCase_ = 0; + body_ = null; + onChanged(); + } + } else { + if (bodyCase_ == 3) { + bodyCase_ = 0; + body_ = null; + } + payloadBuilder_.clear(); + } + return this; + } + /** .com.ververica.statefun.flink.core.Payload payload = 3; */ + public com.ververica.statefun.flink.core.generated.Payload.Builder getPayloadBuilder() { + return getPayloadFieldBuilder().getBuilder(); + } + /** .com.ververica.statefun.flink.core.Payload payload = 3; */ + public com.ververica.statefun.flink.core.generated.PayloadOrBuilder getPayloadOrBuilder() { + if ((bodyCase_ == 3) && (payloadBuilder_ != null)) { + return payloadBuilder_.getMessageOrBuilder(); + } else { + if (bodyCase_ == 3) { + return (com.ververica.statefun.flink.core.generated.Payload) body_; + } + return com.ververica.statefun.flink.core.generated.Payload.getDefaultInstance(); + } + } + /** .com.ververica.statefun.flink.core.Payload payload = 3; */ + private com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.flink.core.generated.Payload, + com.ververica.statefun.flink.core.generated.Payload.Builder, + com.ververica.statefun.flink.core.generated.PayloadOrBuilder> + getPayloadFieldBuilder() { + if (payloadBuilder_ == null) { + if (!(bodyCase_ == 3)) { + body_ = com.ververica.statefun.flink.core.generated.Payload.getDefaultInstance(); + } + payloadBuilder_ = + new com.google.protobuf.SingleFieldBuilderV3< + com.ververica.statefun.flink.core.generated.Payload, + com.ververica.statefun.flink.core.generated.Payload.Builder, + com.ververica.statefun.flink.core.generated.PayloadOrBuilder>( + (com.ververica.statefun.flink.core.generated.Payload) body_, + getParentForChildren(), + isClean()); + body_ = null; + } + bodyCase_ = 3; + onChanged(); + ; + return payloadBuilder_; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.flink.core.Envelope) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.flink.core.Envelope) + private static final com.ververica.statefun.flink.core.generated.Envelope DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = new com.ververica.statefun.flink.core.generated.Envelope(); + } + + public static com.ververica.statefun.flink.core.generated.Envelope getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public Envelope parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new Envelope(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.flink.core.generated.Envelope getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/EnvelopeAddress.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/EnvelopeAddress.java new file mode 100644 index 00000000..f5f21c8a --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/EnvelopeAddress.java @@ -0,0 +1,737 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/stateful-functions.proto + +package com.ververica.statefun.flink.core.generated; + +/** Protobuf type {@code com.ververica.statefun.flink.core.EnvelopeAddress} */ +public final class EnvelopeAddress extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.flink.core.EnvelopeAddress) + EnvelopeAddressOrBuilder { + private static final long serialVersionUID = 0L; + // Use EnvelopeAddress.newBuilder() to construct. + private EnvelopeAddress(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private EnvelopeAddress() { + namespace_ = ""; + type_ = ""; + id_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private EnvelopeAddress( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + java.lang.String s = input.readStringRequireUtf8(); + + namespace_ = s; + break; + } + case 18: + { + java.lang.String s = input.readStringRequireUtf8(); + + type_ = s; + break; + } + case 26: + { + java.lang.String s = input.readStringRequireUtf8(); + + id_ = s; + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.flink.core.generated.StatefulFunctions + .internal_static_com_ververica_statefun_flink_core_EnvelopeAddress_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.flink.core.generated.StatefulFunctions + .internal_static_com_ververica_statefun_flink_core_EnvelopeAddress_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.flink.core.generated.EnvelopeAddress.class, + com.ververica.statefun.flink.core.generated.EnvelopeAddress.Builder.class); + } + + public static final int NAMESPACE_FIELD_NUMBER = 1; + private volatile java.lang.Object namespace_; + /** string namespace = 1; */ + public java.lang.String getNamespace() { + java.lang.Object ref = namespace_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + namespace_ = s; + return s; + } + } + /** string namespace = 1; */ + public com.google.protobuf.ByteString getNamespaceBytes() { + java.lang.Object ref = namespace_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + namespace_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int TYPE_FIELD_NUMBER = 2; + private volatile java.lang.Object type_; + /** string type = 2; */ + public java.lang.String getType() { + java.lang.Object ref = type_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + type_ = s; + return s; + } + } + /** string type = 2; */ + public com.google.protobuf.ByteString getTypeBytes() { + java.lang.Object ref = type_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + type_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int ID_FIELD_NUMBER = 3; + private volatile java.lang.Object id_; + /** string id = 3; */ + public java.lang.String getId() { + java.lang.Object ref = id_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + id_ = s; + return s; + } + } + /** string id = 3; */ + public com.google.protobuf.ByteString getIdBytes() { + java.lang.Object ref = id_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + id_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!getNamespaceBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, namespace_); + } + if (!getTypeBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 2, type_); + } + if (!getIdBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 3, id_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!getNamespaceBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, namespace_); + } + if (!getTypeBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, type_); + } + if (!getIdBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(3, id_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.ververica.statefun.flink.core.generated.EnvelopeAddress)) { + return super.equals(obj); + } + com.ververica.statefun.flink.core.generated.EnvelopeAddress other = + (com.ververica.statefun.flink.core.generated.EnvelopeAddress) obj; + + if (!getNamespace().equals(other.getNamespace())) return false; + if (!getType().equals(other.getType())) return false; + if (!getId().equals(other.getId())) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + NAMESPACE_FIELD_NUMBER; + hash = (53 * hash) + getNamespace().hashCode(); + hash = (37 * hash) + TYPE_FIELD_NUMBER; + hash = (53 * hash) + getType().hashCode(); + hash = (37 * hash) + ID_FIELD_NUMBER; + hash = (53 * hash) + getId().hashCode(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.flink.core.generated.EnvelopeAddress parseFrom( + java.nio.ByteBuffer data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.flink.core.generated.EnvelopeAddress parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.flink.core.generated.EnvelopeAddress parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.flink.core.generated.EnvelopeAddress parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.flink.core.generated.EnvelopeAddress parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.flink.core.generated.EnvelopeAddress parseFrom( + byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.flink.core.generated.EnvelopeAddress parseFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.flink.core.generated.EnvelopeAddress parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.flink.core.generated.EnvelopeAddress parseDelimitedFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.flink.core.generated.EnvelopeAddress parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.flink.core.generated.EnvelopeAddress parseFrom( + com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.flink.core.generated.EnvelopeAddress parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.flink.core.generated.EnvelopeAddress prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code com.ververica.statefun.flink.core.EnvelopeAddress} */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.flink.core.EnvelopeAddress) + com.ververica.statefun.flink.core.generated.EnvelopeAddressOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.flink.core.generated.StatefulFunctions + .internal_static_com_ververica_statefun_flink_core_EnvelopeAddress_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.flink.core.generated.StatefulFunctions + .internal_static_com_ververica_statefun_flink_core_EnvelopeAddress_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.flink.core.generated.EnvelopeAddress.class, + com.ververica.statefun.flink.core.generated.EnvelopeAddress.Builder.class); + } + + // Construct using com.ververica.statefun.flink.core.generated.EnvelopeAddress.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + namespace_ = ""; + + type_ = ""; + + id_ = ""; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.flink.core.generated.StatefulFunctions + .internal_static_com_ververica_statefun_flink_core_EnvelopeAddress_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.flink.core.generated.EnvelopeAddress getDefaultInstanceForType() { + return com.ververica.statefun.flink.core.generated.EnvelopeAddress.getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.flink.core.generated.EnvelopeAddress build() { + com.ververica.statefun.flink.core.generated.EnvelopeAddress result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.flink.core.generated.EnvelopeAddress buildPartial() { + com.ververica.statefun.flink.core.generated.EnvelopeAddress result = + new com.ververica.statefun.flink.core.generated.EnvelopeAddress(this); + result.namespace_ = namespace_; + result.type_ = type_; + result.id_ = id_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.ververica.statefun.flink.core.generated.EnvelopeAddress) { + return mergeFrom((com.ververica.statefun.flink.core.generated.EnvelopeAddress) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.ververica.statefun.flink.core.generated.EnvelopeAddress other) { + if (other == com.ververica.statefun.flink.core.generated.EnvelopeAddress.getDefaultInstance()) + return this; + if (!other.getNamespace().isEmpty()) { + namespace_ = other.namespace_; + onChanged(); + } + if (!other.getType().isEmpty()) { + type_ = other.type_; + onChanged(); + } + if (!other.getId().isEmpty()) { + id_ = other.id_; + onChanged(); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.flink.core.generated.EnvelopeAddress parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.flink.core.generated.EnvelopeAddress) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private java.lang.Object namespace_ = ""; + /** string namespace = 1; */ + public java.lang.String getNamespace() { + java.lang.Object ref = namespace_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + namespace_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string namespace = 1; */ + public com.google.protobuf.ByteString getNamespaceBytes() { + java.lang.Object ref = namespace_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + namespace_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string namespace = 1; */ + public Builder setNamespace(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + namespace_ = value; + onChanged(); + return this; + } + /** string namespace = 1; */ + public Builder clearNamespace() { + + namespace_ = getDefaultInstance().getNamespace(); + onChanged(); + return this; + } + /** string namespace = 1; */ + public Builder setNamespaceBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + namespace_ = value; + onChanged(); + return this; + } + + private java.lang.Object type_ = ""; + /** string type = 2; */ + public java.lang.String getType() { + java.lang.Object ref = type_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + type_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string type = 2; */ + public com.google.protobuf.ByteString getTypeBytes() { + java.lang.Object ref = type_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + type_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string type = 2; */ + public Builder setType(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + type_ = value; + onChanged(); + return this; + } + /** string type = 2; */ + public Builder clearType() { + + type_ = getDefaultInstance().getType(); + onChanged(); + return this; + } + /** string type = 2; */ + public Builder setTypeBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + type_ = value; + onChanged(); + return this; + } + + private java.lang.Object id_ = ""; + /** string id = 3; */ + public java.lang.String getId() { + java.lang.Object ref = id_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + id_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string id = 3; */ + public com.google.protobuf.ByteString getIdBytes() { + java.lang.Object ref = id_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + id_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string id = 3; */ + public Builder setId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + id_ = value; + onChanged(); + return this; + } + /** string id = 3; */ + public Builder clearId() { + + id_ = getDefaultInstance().getId(); + onChanged(); + return this; + } + /** string id = 3; */ + public Builder setIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + id_ = value; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.flink.core.EnvelopeAddress) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.flink.core.EnvelopeAddress) + private static final com.ververica.statefun.flink.core.generated.EnvelopeAddress DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = new com.ververica.statefun.flink.core.generated.EnvelopeAddress(); + } + + public static com.ververica.statefun.flink.core.generated.EnvelopeAddress getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public EnvelopeAddress parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new EnvelopeAddress(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.flink.core.generated.EnvelopeAddress getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/EnvelopeAddressOrBuilder.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/EnvelopeAddressOrBuilder.java new file mode 100644 index 00000000..e09059e5 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/EnvelopeAddressOrBuilder.java @@ -0,0 +1,25 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/stateful-functions.proto + +package com.ververica.statefun.flink.core.generated; + +public interface EnvelopeAddressOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.flink.core.EnvelopeAddress) + com.google.protobuf.MessageOrBuilder { + + /** string namespace = 1; */ + java.lang.String getNamespace(); + /** string namespace = 1; */ + com.google.protobuf.ByteString getNamespaceBytes(); + + /** string type = 2; */ + java.lang.String getType(); + /** string type = 2; */ + com.google.protobuf.ByteString getTypeBytes(); + + /** string id = 3; */ + java.lang.String getId(); + /** string id = 3; */ + com.google.protobuf.ByteString getIdBytes(); +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/EnvelopeOrBuilder.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/EnvelopeOrBuilder.java new file mode 100644 index 00000000..5cbadf10 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/EnvelopeOrBuilder.java @@ -0,0 +1,40 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/stateful-functions.proto + +package com.ververica.statefun.flink.core.generated; + +public interface EnvelopeOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.flink.core.Envelope) + com.google.protobuf.MessageOrBuilder { + + /** .com.ververica.statefun.flink.core.EnvelopeAddress source = 1; */ + boolean hasSource(); + /** .com.ververica.statefun.flink.core.EnvelopeAddress source = 1; */ + com.ververica.statefun.flink.core.generated.EnvelopeAddress getSource(); + /** .com.ververica.statefun.flink.core.EnvelopeAddress source = 1; */ + com.ververica.statefun.flink.core.generated.EnvelopeAddressOrBuilder getSourceOrBuilder(); + + /** .com.ververica.statefun.flink.core.EnvelopeAddress target = 2; */ + boolean hasTarget(); + /** .com.ververica.statefun.flink.core.EnvelopeAddress target = 2; */ + com.ververica.statefun.flink.core.generated.EnvelopeAddress getTarget(); + /** .com.ververica.statefun.flink.core.EnvelopeAddress target = 2; */ + com.ververica.statefun.flink.core.generated.EnvelopeAddressOrBuilder getTargetOrBuilder(); + + /** .com.ververica.statefun.flink.core.Checkpoint checkpoint = 4; */ + boolean hasCheckpoint(); + /** .com.ververica.statefun.flink.core.Checkpoint checkpoint = 4; */ + com.ververica.statefun.flink.core.generated.Checkpoint getCheckpoint(); + /** .com.ververica.statefun.flink.core.Checkpoint checkpoint = 4; */ + com.ververica.statefun.flink.core.generated.CheckpointOrBuilder getCheckpointOrBuilder(); + + /** .com.ververica.statefun.flink.core.Payload payload = 3; */ + boolean hasPayload(); + /** .com.ververica.statefun.flink.core.Payload payload = 3; */ + com.ververica.statefun.flink.core.generated.Payload getPayload(); + /** .com.ververica.statefun.flink.core.Payload payload = 3; */ + com.ververica.statefun.flink.core.generated.PayloadOrBuilder getPayloadOrBuilder(); + + public com.ververica.statefun.flink.core.generated.Envelope.BodyCase getBodyCase(); +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/Payload.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/Payload.java new file mode 100644 index 00000000..c9a2256a --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/Payload.java @@ -0,0 +1,577 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/stateful-functions.proto + +package com.ververica.statefun.flink.core.generated; + +/** Protobuf type {@code com.ververica.statefun.flink.core.Payload} */ +public final class Payload extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.flink.core.Payload) + PayloadOrBuilder { + private static final long serialVersionUID = 0L; + // Use Payload.newBuilder() to construct. + private Payload(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private Payload() { + className_ = ""; + payloadBytes_ = com.google.protobuf.ByteString.EMPTY; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private Payload( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 18: + { + java.lang.String s = input.readStringRequireUtf8(); + + className_ = s; + break; + } + case 26: + { + payloadBytes_ = input.readBytes(); + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.flink.core.generated.StatefulFunctions + .internal_static_com_ververica_statefun_flink_core_Payload_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.flink.core.generated.StatefulFunctions + .internal_static_com_ververica_statefun_flink_core_Payload_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.flink.core.generated.Payload.class, + com.ververica.statefun.flink.core.generated.Payload.Builder.class); + } + + public static final int CLASS_NAME_FIELD_NUMBER = 2; + private volatile java.lang.Object className_; + /** string class_name = 2; */ + public java.lang.String getClassName() { + java.lang.Object ref = className_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + className_ = s; + return s; + } + } + /** string class_name = 2; */ + public com.google.protobuf.ByteString getClassNameBytes() { + java.lang.Object ref = className_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + className_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int PAYLOAD_BYTES_FIELD_NUMBER = 3; + private com.google.protobuf.ByteString payloadBytes_; + /** bytes payload_bytes = 3; */ + public com.google.protobuf.ByteString getPayloadBytes() { + return payloadBytes_; + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!getClassNameBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 2, className_); + } + if (!payloadBytes_.isEmpty()) { + output.writeBytes(3, payloadBytes_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!getClassNameBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, className_); + } + if (!payloadBytes_.isEmpty()) { + size += com.google.protobuf.CodedOutputStream.computeBytesSize(3, payloadBytes_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.ververica.statefun.flink.core.generated.Payload)) { + return super.equals(obj); + } + com.ververica.statefun.flink.core.generated.Payload other = + (com.ververica.statefun.flink.core.generated.Payload) obj; + + if (!getClassName().equals(other.getClassName())) return false; + if (!getPayloadBytes().equals(other.getPayloadBytes())) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + CLASS_NAME_FIELD_NUMBER; + hash = (53 * hash) + getClassName().hashCode(); + hash = (37 * hash) + PAYLOAD_BYTES_FIELD_NUMBER; + hash = (53 * hash) + getPayloadBytes().hashCode(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.flink.core.generated.Payload parseFrom( + java.nio.ByteBuffer data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.flink.core.generated.Payload parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.flink.core.generated.Payload parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.flink.core.generated.Payload parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.flink.core.generated.Payload parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.flink.core.generated.Payload parseFrom( + byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.flink.core.generated.Payload parseFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.flink.core.generated.Payload parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.flink.core.generated.Payload parseDelimitedFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.flink.core.generated.Payload parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.flink.core.generated.Payload parseFrom( + com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.flink.core.generated.Payload parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder(com.ververica.statefun.flink.core.generated.Payload prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code com.ververica.statefun.flink.core.Payload} */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.flink.core.Payload) + com.ververica.statefun.flink.core.generated.PayloadOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.flink.core.generated.StatefulFunctions + .internal_static_com_ververica_statefun_flink_core_Payload_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.flink.core.generated.StatefulFunctions + .internal_static_com_ververica_statefun_flink_core_Payload_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.flink.core.generated.Payload.class, + com.ververica.statefun.flink.core.generated.Payload.Builder.class); + } + + // Construct using com.ververica.statefun.flink.core.generated.Payload.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + className_ = ""; + + payloadBytes_ = com.google.protobuf.ByteString.EMPTY; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.flink.core.generated.StatefulFunctions + .internal_static_com_ververica_statefun_flink_core_Payload_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.flink.core.generated.Payload getDefaultInstanceForType() { + return com.ververica.statefun.flink.core.generated.Payload.getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.flink.core.generated.Payload build() { + com.ververica.statefun.flink.core.generated.Payload result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.flink.core.generated.Payload buildPartial() { + com.ververica.statefun.flink.core.generated.Payload result = + new com.ververica.statefun.flink.core.generated.Payload(this); + result.className_ = className_; + result.payloadBytes_ = payloadBytes_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.ververica.statefun.flink.core.generated.Payload) { + return mergeFrom((com.ververica.statefun.flink.core.generated.Payload) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.ververica.statefun.flink.core.generated.Payload other) { + if (other == com.ververica.statefun.flink.core.generated.Payload.getDefaultInstance()) + return this; + if (!other.getClassName().isEmpty()) { + className_ = other.className_; + onChanged(); + } + if (other.getPayloadBytes() != com.google.protobuf.ByteString.EMPTY) { + setPayloadBytes(other.getPayloadBytes()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.flink.core.generated.Payload parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.flink.core.generated.Payload) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private java.lang.Object className_ = ""; + /** string class_name = 2; */ + public java.lang.String getClassName() { + java.lang.Object ref = className_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + className_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string class_name = 2; */ + public com.google.protobuf.ByteString getClassNameBytes() { + java.lang.Object ref = className_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + className_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string class_name = 2; */ + public Builder setClassName(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + className_ = value; + onChanged(); + return this; + } + /** string class_name = 2; */ + public Builder clearClassName() { + + className_ = getDefaultInstance().getClassName(); + onChanged(); + return this; + } + /** string class_name = 2; */ + public Builder setClassNameBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + className_ = value; + onChanged(); + return this; + } + + private com.google.protobuf.ByteString payloadBytes_ = com.google.protobuf.ByteString.EMPTY; + /** bytes payload_bytes = 3; */ + public com.google.protobuf.ByteString getPayloadBytes() { + return payloadBytes_; + } + /** bytes payload_bytes = 3; */ + public Builder setPayloadBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + + payloadBytes_ = value; + onChanged(); + return this; + } + /** bytes payload_bytes = 3; */ + public Builder clearPayloadBytes() { + + payloadBytes_ = getDefaultInstance().getPayloadBytes(); + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.flink.core.Payload) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.flink.core.Payload) + private static final com.ververica.statefun.flink.core.generated.Payload DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = new com.ververica.statefun.flink.core.generated.Payload(); + } + + public static com.ververica.statefun.flink.core.generated.Payload getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public Payload parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new Payload(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.flink.core.generated.Payload getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/PayloadOrBuilder.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/PayloadOrBuilder.java new file mode 100644 index 00000000..1056341f --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/PayloadOrBuilder.java @@ -0,0 +1,18 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/stateful-functions.proto + +package com.ververica.statefun.flink.core.generated; + +public interface PayloadOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.flink.core.Payload) + com.google.protobuf.MessageOrBuilder { + + /** string class_name = 2; */ + java.lang.String getClassName(); + /** string class_name = 2; */ + com.google.protobuf.ByteString getClassNameBytes(); + + /** bytes payload_bytes = 3; */ + com.google.protobuf.ByteString getPayloadBytes(); +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/ProtobufSerializerSnapshot.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/ProtobufSerializerSnapshot.java new file mode 100644 index 00000000..7b63b69b --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/ProtobufSerializerSnapshot.java @@ -0,0 +1,831 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/stateful-functions.proto + +package com.ververica.statefun.flink.core.generated; + +/** Protobuf type {@code com.ververica.statefun.flink.core.ProtobufSerializerSnapshot} */ +public final class ProtobufSerializerSnapshot extends com.google.protobuf.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:com.ververica.statefun.flink.core.ProtobufSerializerSnapshot) + ProtobufSerializerSnapshotOrBuilder { + private static final long serialVersionUID = 0L; + // Use ProtobufSerializerSnapshot.newBuilder() to construct. + private ProtobufSerializerSnapshot(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private ProtobufSerializerSnapshot() { + generatedJavaName_ = ""; + messageName_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private ProtobufSerializerSnapshot( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + java.lang.String s = input.readStringRequireUtf8(); + + generatedJavaName_ = s; + break; + } + case 18: + { + java.lang.String s = input.readStringRequireUtf8(); + + messageName_ = s; + break; + } + case 26: + { + com.google.protobuf.DescriptorProtos.FileDescriptorSet.Builder subBuilder = null; + if (descriptorSet_ != null) { + subBuilder = descriptorSet_.toBuilder(); + } + descriptorSet_ = + input.readMessage( + com.google.protobuf.DescriptorProtos.FileDescriptorSet.PARSER, + extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(descriptorSet_); + descriptorSet_ = subBuilder.buildPartial(); + } + + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.flink.core.generated.StatefulFunctions + .internal_static_com_ververica_statefun_flink_core_ProtobufSerializerSnapshot_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.flink.core.generated.StatefulFunctions + .internal_static_com_ververica_statefun_flink_core_ProtobufSerializerSnapshot_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot.class, + com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot.Builder.class); + } + + public static final int GENERATED_JAVA_NAME_FIELD_NUMBER = 1; + private volatile java.lang.Object generatedJavaName_; + /** string generated_java_name = 1; */ + public java.lang.String getGeneratedJavaName() { + java.lang.Object ref = generatedJavaName_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + generatedJavaName_ = s; + return s; + } + } + /** string generated_java_name = 1; */ + public com.google.protobuf.ByteString getGeneratedJavaNameBytes() { + java.lang.Object ref = generatedJavaName_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + generatedJavaName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int MESSAGE_NAME_FIELD_NUMBER = 2; + private volatile java.lang.Object messageName_; + /** string message_name = 2; */ + public java.lang.String getMessageName() { + java.lang.Object ref = messageName_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + messageName_ = s; + return s; + } + } + /** string message_name = 2; */ + public com.google.protobuf.ByteString getMessageNameBytes() { + java.lang.Object ref = messageName_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + messageName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int DESCRIPTOR_SET_FIELD_NUMBER = 3; + private com.google.protobuf.DescriptorProtos.FileDescriptorSet descriptorSet_; + /** .google.protobuf.FileDescriptorSet descriptor_set = 3; */ + public boolean hasDescriptorSet() { + return descriptorSet_ != null; + } + /** .google.protobuf.FileDescriptorSet descriptor_set = 3; */ + public com.google.protobuf.DescriptorProtos.FileDescriptorSet getDescriptorSet() { + return descriptorSet_ == null + ? com.google.protobuf.DescriptorProtos.FileDescriptorSet.getDefaultInstance() + : descriptorSet_; + } + /** .google.protobuf.FileDescriptorSet descriptor_set = 3; */ + public com.google.protobuf.DescriptorProtos.FileDescriptorSetOrBuilder + getDescriptorSetOrBuilder() { + return getDescriptorSet(); + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + if (hasDescriptorSet()) { + if (!getDescriptorSet().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!getGeneratedJavaNameBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, generatedJavaName_); + } + if (!getMessageNameBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 2, messageName_); + } + if (descriptorSet_ != null) { + output.writeMessage(3, getDescriptorSet()); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!getGeneratedJavaNameBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, generatedJavaName_); + } + if (!getMessageNameBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, messageName_); + } + if (descriptorSet_ != null) { + size += com.google.protobuf.CodedOutputStream.computeMessageSize(3, getDescriptorSet()); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot)) { + return super.equals(obj); + } + com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot other = + (com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot) obj; + + if (!getGeneratedJavaName().equals(other.getGeneratedJavaName())) return false; + if (!getMessageName().equals(other.getMessageName())) return false; + if (hasDescriptorSet() != other.hasDescriptorSet()) return false; + if (hasDescriptorSet()) { + if (!getDescriptorSet().equals(other.getDescriptorSet())) return false; + } + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + GENERATED_JAVA_NAME_FIELD_NUMBER; + hash = (53 * hash) + getGeneratedJavaName().hashCode(); + hash = (37 * hash) + MESSAGE_NAME_FIELD_NUMBER; + hash = (53 * hash) + getMessageName().hashCode(); + if (hasDescriptorSet()) { + hash = (37 * hash) + DESCRIPTOR_SET_FIELD_NUMBER; + hash = (53 * hash) + getDescriptorSet().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot parseFrom( + java.nio.ByteBuffer data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot parseFrom( + java.nio.ByteBuffer data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot parseFrom( + byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot parseFrom( + byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot parseFrom( + java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot parseFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot + parseDelimitedFrom( + java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot parseFrom( + com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code com.ververica.statefun.flink.core.ProtobufSerializerSnapshot} */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:com.ververica.statefun.flink.core.ProtobufSerializerSnapshot) + com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshotOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.ververica.statefun.flink.core.generated.StatefulFunctions + .internal_static_com_ververica_statefun_flink_core_ProtobufSerializerSnapshot_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.ververica.statefun.flink.core.generated.StatefulFunctions + .internal_static_com_ververica_statefun_flink_core_ProtobufSerializerSnapshot_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot.class, + com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot.Builder.class); + } + + // Construct using + // com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + generatedJavaName_ = ""; + + messageName_ = ""; + + if (descriptorSetBuilder_ == null) { + descriptorSet_ = null; + } else { + descriptorSet_ = null; + descriptorSetBuilder_ = null; + } + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.ververica.statefun.flink.core.generated.StatefulFunctions + .internal_static_com_ververica_statefun_flink_core_ProtobufSerializerSnapshot_descriptor; + } + + @java.lang.Override + public com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot + getDefaultInstanceForType() { + return com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot + .getDefaultInstance(); + } + + @java.lang.Override + public com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot build() { + com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot result = + buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot buildPartial() { + com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot result = + new com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot(this); + result.generatedJavaName_ = generatedJavaName_; + result.messageName_ = messageName_; + if (descriptorSetBuilder_ == null) { + result.descriptorSet_ = descriptorSet_; + } else { + result.descriptorSet_ = descriptorSetBuilder_.build(); + } + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot) { + return mergeFrom( + (com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot other) { + if (other + == com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot + .getDefaultInstance()) return this; + if (!other.getGeneratedJavaName().isEmpty()) { + generatedJavaName_ = other.generatedJavaName_; + onChanged(); + } + if (!other.getMessageName().isEmpty()) { + messageName_ = other.messageName_; + onChanged(); + } + if (other.hasDescriptorSet()) { + mergeDescriptorSet(other.getDescriptorSet()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + if (hasDescriptorSet()) { + if (!getDescriptorSet().isInitialized()) { + return false; + } + } + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = + (com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private java.lang.Object generatedJavaName_ = ""; + /** string generated_java_name = 1; */ + public java.lang.String getGeneratedJavaName() { + java.lang.Object ref = generatedJavaName_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + generatedJavaName_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string generated_java_name = 1; */ + public com.google.protobuf.ByteString getGeneratedJavaNameBytes() { + java.lang.Object ref = generatedJavaName_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + generatedJavaName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string generated_java_name = 1; */ + public Builder setGeneratedJavaName(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + generatedJavaName_ = value; + onChanged(); + return this; + } + /** string generated_java_name = 1; */ + public Builder clearGeneratedJavaName() { + + generatedJavaName_ = getDefaultInstance().getGeneratedJavaName(); + onChanged(); + return this; + } + /** string generated_java_name = 1; */ + public Builder setGeneratedJavaNameBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + generatedJavaName_ = value; + onChanged(); + return this; + } + + private java.lang.Object messageName_ = ""; + /** string message_name = 2; */ + public java.lang.String getMessageName() { + java.lang.Object ref = messageName_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + messageName_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** string message_name = 2; */ + public com.google.protobuf.ByteString getMessageNameBytes() { + java.lang.Object ref = messageName_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + messageName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** string message_name = 2; */ + public Builder setMessageName(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + messageName_ = value; + onChanged(); + return this; + } + /** string message_name = 2; */ + public Builder clearMessageName() { + + messageName_ = getDefaultInstance().getMessageName(); + onChanged(); + return this; + } + /** string message_name = 2; */ + public Builder setMessageNameBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + messageName_ = value; + onChanged(); + return this; + } + + private com.google.protobuf.DescriptorProtos.FileDescriptorSet descriptorSet_; + private com.google.protobuf.SingleFieldBuilderV3< + com.google.protobuf.DescriptorProtos.FileDescriptorSet, + com.google.protobuf.DescriptorProtos.FileDescriptorSet.Builder, + com.google.protobuf.DescriptorProtos.FileDescriptorSetOrBuilder> + descriptorSetBuilder_; + /** .google.protobuf.FileDescriptorSet descriptor_set = 3; */ + public boolean hasDescriptorSet() { + return descriptorSetBuilder_ != null || descriptorSet_ != null; + } + /** .google.protobuf.FileDescriptorSet descriptor_set = 3; */ + public com.google.protobuf.DescriptorProtos.FileDescriptorSet getDescriptorSet() { + if (descriptorSetBuilder_ == null) { + return descriptorSet_ == null + ? com.google.protobuf.DescriptorProtos.FileDescriptorSet.getDefaultInstance() + : descriptorSet_; + } else { + return descriptorSetBuilder_.getMessage(); + } + } + /** .google.protobuf.FileDescriptorSet descriptor_set = 3; */ + public Builder setDescriptorSet(com.google.protobuf.DescriptorProtos.FileDescriptorSet value) { + if (descriptorSetBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + descriptorSet_ = value; + onChanged(); + } else { + descriptorSetBuilder_.setMessage(value); + } + + return this; + } + /** .google.protobuf.FileDescriptorSet descriptor_set = 3; */ + public Builder setDescriptorSet( + com.google.protobuf.DescriptorProtos.FileDescriptorSet.Builder builderForValue) { + if (descriptorSetBuilder_ == null) { + descriptorSet_ = builderForValue.build(); + onChanged(); + } else { + descriptorSetBuilder_.setMessage(builderForValue.build()); + } + + return this; + } + /** .google.protobuf.FileDescriptorSet descriptor_set = 3; */ + public Builder mergeDescriptorSet( + com.google.protobuf.DescriptorProtos.FileDescriptorSet value) { + if (descriptorSetBuilder_ == null) { + if (descriptorSet_ != null) { + descriptorSet_ = + com.google.protobuf.DescriptorProtos.FileDescriptorSet.newBuilder(descriptorSet_) + .mergeFrom(value) + .buildPartial(); + } else { + descriptorSet_ = value; + } + onChanged(); + } else { + descriptorSetBuilder_.mergeFrom(value); + } + + return this; + } + /** .google.protobuf.FileDescriptorSet descriptor_set = 3; */ + public Builder clearDescriptorSet() { + if (descriptorSetBuilder_ == null) { + descriptorSet_ = null; + onChanged(); + } else { + descriptorSet_ = null; + descriptorSetBuilder_ = null; + } + + return this; + } + /** .google.protobuf.FileDescriptorSet descriptor_set = 3; */ + public com.google.protobuf.DescriptorProtos.FileDescriptorSet.Builder + getDescriptorSetBuilder() { + + onChanged(); + return getDescriptorSetFieldBuilder().getBuilder(); + } + /** .google.protobuf.FileDescriptorSet descriptor_set = 3; */ + public com.google.protobuf.DescriptorProtos.FileDescriptorSetOrBuilder + getDescriptorSetOrBuilder() { + if (descriptorSetBuilder_ != null) { + return descriptorSetBuilder_.getMessageOrBuilder(); + } else { + return descriptorSet_ == null + ? com.google.protobuf.DescriptorProtos.FileDescriptorSet.getDefaultInstance() + : descriptorSet_; + } + } + /** .google.protobuf.FileDescriptorSet descriptor_set = 3; */ + private com.google.protobuf.SingleFieldBuilderV3< + com.google.protobuf.DescriptorProtos.FileDescriptorSet, + com.google.protobuf.DescriptorProtos.FileDescriptorSet.Builder, + com.google.protobuf.DescriptorProtos.FileDescriptorSetOrBuilder> + getDescriptorSetFieldBuilder() { + if (descriptorSetBuilder_ == null) { + descriptorSetBuilder_ = + new com.google.protobuf.SingleFieldBuilderV3< + com.google.protobuf.DescriptorProtos.FileDescriptorSet, + com.google.protobuf.DescriptorProtos.FileDescriptorSet.Builder, + com.google.protobuf.DescriptorProtos.FileDescriptorSetOrBuilder>( + getDescriptorSet(), getParentForChildren(), isClean()); + descriptorSet_ = null; + } + return descriptorSetBuilder_; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:com.ververica.statefun.flink.core.ProtobufSerializerSnapshot) + } + + // @@protoc_insertion_point(class_scope:com.ververica.statefun.flink.core.ProtobufSerializerSnapshot) + private static final com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = new com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot(); + } + + public static com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public ProtobufSerializerSnapshot parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new ProtobufSerializerSnapshot(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/ProtobufSerializerSnapshotOrBuilder.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/ProtobufSerializerSnapshotOrBuilder.java new file mode 100644 index 00000000..791bd89e --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/ProtobufSerializerSnapshotOrBuilder.java @@ -0,0 +1,27 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/stateful-functions.proto + +package com.ververica.statefun.flink.core.generated; + +public interface ProtobufSerializerSnapshotOrBuilder + extends + // @@protoc_insertion_point(interface_extends:com.ververica.statefun.flink.core.ProtobufSerializerSnapshot) + com.google.protobuf.MessageOrBuilder { + + /** string generated_java_name = 1; */ + java.lang.String getGeneratedJavaName(); + /** string generated_java_name = 1; */ + com.google.protobuf.ByteString getGeneratedJavaNameBytes(); + + /** string message_name = 2; */ + java.lang.String getMessageName(); + /** string message_name = 2; */ + com.google.protobuf.ByteString getMessageNameBytes(); + + /** .google.protobuf.FileDescriptorSet descriptor_set = 3; */ + boolean hasDescriptorSet(); + /** .google.protobuf.FileDescriptorSet descriptor_set = 3; */ + com.google.protobuf.DescriptorProtos.FileDescriptorSet getDescriptorSet(); + /** .google.protobuf.FileDescriptorSet descriptor_set = 3; */ + com.google.protobuf.DescriptorProtos.FileDescriptorSetOrBuilder getDescriptorSetOrBuilder(); +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/StatefulFunctions.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/StatefulFunctions.java new file mode 100644 index 00000000..b98e0fe9 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/generated/StatefulFunctions.java @@ -0,0 +1,122 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/protobuf/stateful-functions.proto + +package com.ververica.statefun.flink.core.generated; + +public final class StatefulFunctions { + private StatefulFunctions() {} + + public static void registerAllExtensions(com.google.protobuf.ExtensionRegistryLite registry) {} + + public static void registerAllExtensions(com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions((com.google.protobuf.ExtensionRegistryLite) registry); + } + + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_flink_core_ProtobufSerializerSnapshot_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_flink_core_ProtobufSerializerSnapshot_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_flink_core_EnvelopeAddress_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_flink_core_EnvelopeAddress_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_flink_core_Payload_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_flink_core_Payload_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_flink_core_Checkpoint_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_flink_core_Checkpoint_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_com_ververica_statefun_flink_core_Envelope_descriptor; + static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_com_ververica_statefun_flink_core_Envelope_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { + return descriptor; + } + + private static com.google.protobuf.Descriptors.FileDescriptor descriptor; + + static { + java.lang.String[] descriptorData = { + "\n*src/main/protobuf/stateful-functions.p" + + "roto\022!com.ververica.statefun.flink.core\032" + + " google/protobuf/descriptor.proto\"\213\001\n\032Pr" + + "otobufSerializerSnapshot\022\033\n\023generated_ja" + + "va_name\030\001 \001(\t\022\024\n\014message_name\030\002 \001(\t\022:\n\016d" + + "escriptor_set\030\003 \001(\0132\".google.protobuf.Fi" + + "leDescriptorSet\">\n\017EnvelopeAddress\022\021\n\tna" + + "mespace\030\001 \001(\t\022\014\n\004type\030\002 \001(\t\022\n\n\002id\030\003 \001(\t\"" + + "4\n\007Payload\022\022\n\nclass_name\030\002 \001(\t\022\025\n\rpayloa" + + "d_bytes\030\003 \001(\014\"#\n\nCheckpoint\022\025\n\rcheckpoin" + + "t_id\030\001 \001(\003\"\236\002\n\010Envelope\022B\n\006source\030\001 \001(\0132" + + "2.com.ververica.statefun.flink.core.Enve" + + "lopeAddress\022B\n\006target\030\002 \001(\01322.com.verver" + + "ica.statefun.flink.core.EnvelopeAddress\022" + + "C\n\ncheckpoint\030\004 \001(\0132-.com.ververica.stat" + + "efun.flink.core.CheckpointH\000\022=\n\007payload\030" + + "\003 \001(\0132*.com.ververica.statefun.flink.cor" + + "e.PayloadH\000B\006\n\004bodyB/\n+com.ververica.sta" + + "tefun.flink.core.generatedP\001b\006proto3" + }; + com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = + new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { + public com.google.protobuf.ExtensionRegistry assignDescriptors( + com.google.protobuf.Descriptors.FileDescriptor root) { + descriptor = root; + return null; + } + }; + com.google.protobuf.Descriptors.FileDescriptor.internalBuildGeneratedFileFrom( + descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + com.google.protobuf.DescriptorProtos.getDescriptor(), + }, + assigner); + internal_static_com_ververica_statefun_flink_core_ProtobufSerializerSnapshot_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_com_ververica_statefun_flink_core_ProtobufSerializerSnapshot_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_flink_core_ProtobufSerializerSnapshot_descriptor, + new java.lang.String[] { + "GeneratedJavaName", "MessageName", "DescriptorSet", + }); + internal_static_com_ververica_statefun_flink_core_EnvelopeAddress_descriptor = + getDescriptor().getMessageTypes().get(1); + internal_static_com_ververica_statefun_flink_core_EnvelopeAddress_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_flink_core_EnvelopeAddress_descriptor, + new java.lang.String[] { + "Namespace", "Type", "Id", + }); + internal_static_com_ververica_statefun_flink_core_Payload_descriptor = + getDescriptor().getMessageTypes().get(2); + internal_static_com_ververica_statefun_flink_core_Payload_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_flink_core_Payload_descriptor, + new java.lang.String[] { + "ClassName", "PayloadBytes", + }); + internal_static_com_ververica_statefun_flink_core_Checkpoint_descriptor = + getDescriptor().getMessageTypes().get(3); + internal_static_com_ververica_statefun_flink_core_Checkpoint_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_flink_core_Checkpoint_descriptor, + new java.lang.String[] { + "CheckpointId", + }); + internal_static_com_ververica_statefun_flink_core_Envelope_descriptor = + getDescriptor().getMessageTypes().get(4); + internal_static_com_ververica_statefun_flink_core_Envelope_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_com_ververica_statefun_flink_core_Envelope_descriptor, + new java.lang.String[] { + "Source", "Target", "Checkpoint", "Payload", "Body", + }); + com.google.protobuf.DescriptorProtos.getDescriptor(); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/logger/CheckpointedStreamOperations.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/logger/CheckpointedStreamOperations.java new file mode 100644 index 00000000..5ae10e8d --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/logger/CheckpointedStreamOperations.java @@ -0,0 +1,30 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.logger; + +import java.io.Closeable; +import java.io.IOException; +import java.io.OutputStream; + +public interface CheckpointedStreamOperations { + + void requireKeyedStateCheckpointed(OutputStream keyedStateCheckpointOutputStream); + + void startNewKeyGroup(OutputStream stream, int keyGroup) throws IOException; + + Closeable acquireLease(OutputStream keyedStateCheckpointOutputStream); +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/logger/KeyGroupAssigner.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/logger/KeyGroupAssigner.java new file mode 100644 index 00000000..74d1b4d1 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/logger/KeyGroupAssigner.java @@ -0,0 +1,38 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.logger; + +import com.ververica.statefun.flink.core.common.KeyBy; +import com.ververica.statefun.flink.core.di.Inject; +import com.ververica.statefun.flink.core.di.Label; +import com.ververica.statefun.sdk.Address; +import java.util.function.ToIntFunction; +import org.apache.flink.runtime.state.KeyGroupRangeAssignment; + +final class KeyGroupAssigner implements ToIntFunction

{ + private final int maxParallelism; + + @Inject + KeyGroupAssigner(@Label("max-parallelism") int maxParallelism) { + this.maxParallelism = maxParallelism; + } + + @Override + public int applyAsInt(Address target) { + return KeyGroupRangeAssignment.assignToKeyGroup(KeyBy.apply(target), maxParallelism); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/logger/KeyGroupStream.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/logger/KeyGroupStream.java new file mode 100644 index 00000000..95239d44 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/logger/KeyGroupStream.java @@ -0,0 +1,99 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.logger; + +import com.ververica.statefun.flink.core.feedback.FeedbackConsumer; +import com.ververica.statefun.flink.core.message.Message; +import java.io.IOException; +import java.util.Objects; +import javax.annotation.Nonnull; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.io.disk.SpillingBuffer; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; + +final class KeyGroupStream { + private final TypeSerializer serializer; + private final SpillingBuffer target; + private final MemorySegmentPool memoryPool; + private final DataOutputSerializer output = new DataOutputSerializer(256); + + private long totalSize; + private int elementCount; + + KeyGroupStream( + TypeSerializer serializer, + IOManager ioManager, + MemorySegmentPool memorySegmentPool) { + this.serializer = Objects.requireNonNull(serializer); + this.memoryPool = Objects.requireNonNull(memorySegmentPool); + + // SpillingBuffer requires at least 1 memory segment to be present at construction, otherwise it + // fails + // so we + memorySegmentPool.ensureAtLeastOneSegmentPresent(); + this.target = + new SpillingBuffer(ioManager, memorySegmentPool, memorySegmentPool.getSegmentSize()); + } + + static void readFrom( + DataInputView source, TypeSerializer serializer, FeedbackConsumer consumer) + throws Exception { + final int elementCount = source.readInt(); + + for (int i = 0; i < elementCount; i++) { + Message envelope = serializer.deserialize(source); + consumer.processFeedback(envelope); + } + } + + private static void copy(@Nonnull DataInputView source, @Nonnull DataOutputView target, long size) + throws IOException { + + while (size > 0) { + final int len = (int) Math.min(4 * 1024, size); // read no more then 4k bytes at a time + target.write(source, len); + size -= len; + } + } + + void append(Message envelope) { + elementCount++; + try { + output.clear(); + serializer.serialize(envelope, output); + totalSize += output.length(); + + target.write(output.getSharedBuffer(), 0, output.length()); + } catch (IOException e) { + throw new IllegalStateException(e); + } + } + + void writeTo(DataOutputView target) throws IOException { + target.writeInt(elementCount); + + copy(this.target.flip(), target, totalSize); + + for (MemorySegment segment : this.target.close()) { + memoryPool.release(segment); + } + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/logger/KeyGroupStreamFactory.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/logger/KeyGroupStreamFactory.java new file mode 100644 index 00000000..93615c1a --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/logger/KeyGroupStreamFactory.java @@ -0,0 +1,45 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.logger; + +import com.ververica.statefun.flink.core.di.Inject; +import com.ververica.statefun.flink.core.di.Label; +import com.ververica.statefun.flink.core.message.Message; +import java.util.function.Supplier; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; + +public final class KeyGroupStreamFactory implements Supplier { + private final IOManager ioManager; + private final MemorySegmentPool memorySegmentPool; + private final TypeSerializer serializer; + + @Inject + KeyGroupStreamFactory( + @Label("io-manager") IOManager ioManager, + @Label("in-memory-max-buffer-size") long inMemoryBufferSize, + @Label("envelope-serializer") TypeSerializer serializer) { + this.ioManager = ioManager; + this.serializer = serializer; + this.memorySegmentPool = new MemorySegmentPool(inMemoryBufferSize); + } + + @Override + public KeyGroupStream get() { + return new KeyGroupStream(serializer, ioManager, memorySegmentPool); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/logger/Loggers.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/logger/Loggers.java new file mode 100644 index 00000000..4b1467c0 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/logger/Loggers.java @@ -0,0 +1,104 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.logger; + +import com.ververica.statefun.flink.core.di.ObjectContainer; +import com.ververica.statefun.flink.core.message.Message; +import java.io.Closeable; +import java.io.IOException; +import java.io.OutputStream; +import java.util.function.Supplier; +import java.util.function.ToIntFunction; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.state.KeyedStateCheckpointOutputStream; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.ResourceGuard.Lease; + +public final class Loggers { + private Loggers() {} + + public static UnboundedFeedbackLogger unboundedSpillableLogger( + IOManager ioManager, + int maxParallelism, + long inMemoryMaxBufferSize, + TypeSerializer serializer) { + + ObjectContainer container = + unboundedSpillableLoggerContainer( + ioManager, maxParallelism, inMemoryMaxBufferSize, serializer); + return container.get(UnboundedFeedbackLogger.class); + } + + /** Wires the required dependencies to construct an {@link UnboundedFeedbackLogger}. */ + @VisibleForTesting + static ObjectContainer unboundedSpillableLoggerContainer( + IOManager ioManager, + int maxParallelism, + long inMemoryMaxBufferSize, + TypeSerializer serializer) { + + ObjectContainer container = new ObjectContainer(); + container.add("max-parallelism", int.class, maxParallelism); + container.add("in-memory-max-buffer-size", long.class, inMemoryMaxBufferSize); + container.add("io-manager", IOManager.class, ioManager); + container.add("key-group-supplier", Supplier.class, KeyGroupStreamFactory.class); + container.add("key-group-assigner", ToIntFunction.class, KeyGroupAssigner.class); + container.add("envelope-serializer", TypeSerializer.class, serializer); + container.add( + "checkpoint-stream-ops", + CheckpointedStreamOperations.class, + KeyedStateCheckpointOutputStreamOps.INSTANCE); + container.add(UnboundedFeedbackLogger.class); + return container; + } + + private enum KeyedStateCheckpointOutputStreamOps implements CheckpointedStreamOperations { + INSTANCE; + + @Override + public void requireKeyedStateCheckpointed(OutputStream stream) { + if (stream instanceof KeyedStateCheckpointOutputStream) { + return; + } + throw new IllegalStateException("Not a KeyedStateCheckpointOutputStream"); + } + + @Override + public void startNewKeyGroup(OutputStream stream, int keyGroup) throws IOException { + cast(stream).startNewKeyGroup(keyGroup); + } + + @Override + @SuppressWarnings("resource") + public Closeable acquireLease(OutputStream stream) { + Preconditions.checkState(stream instanceof KeyedStateCheckpointOutputStream); + try { + Lease lease = cast(stream).acquireLease(); + return lease::close; + } catch (IOException e) { + throw new IllegalStateException("Unable to obtain a lease for the input stream.", e); + } + } + + private static KeyedStateCheckpointOutputStream cast(OutputStream stream) { + Preconditions.checkState(stream instanceof KeyedStateCheckpointOutputStream); + return (KeyedStateCheckpointOutputStream) stream; + } + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/logger/MemorySegmentPool.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/logger/MemorySegmentPool.java new file mode 100644 index 00000000..554a5765 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/logger/MemorySegmentPool.java @@ -0,0 +1,98 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.logger; + +import java.util.ArrayDeque; +import javax.annotation.Nullable; +import javax.annotation.concurrent.NotThreadSafe; +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.core.memory.MemorySegmentFactory; +import org.apache.flink.core.memory.MemorySegmentSource; + +@NotThreadSafe +final class MemorySegmentPool implements MemorySegmentSource { + static final int PAGE_SIZE = 64 * 1024; + + private final ArrayDeque pool; + private final long inMemoryBufferSize; + private long totalAllocatedMemory; + + MemorySegmentPool(long inMemoryBufferSize) { + this.pool = new ArrayDeque<>(); + this.inMemoryBufferSize = inMemoryBufferSize; + } + + @Nullable + @Override + public MemorySegment nextSegment() { + MemorySegment segment = pool.pollFirst(); + if (segment != null) { + return segment; + } + // + // no segments in the pool, try to allocate one. + // + if (!hasRemainingCapacity()) { + return null; + } + segment = MemorySegmentFactory.allocateUnpooledSegment(PAGE_SIZE); + totalAllocatedMemory += PAGE_SIZE; + return segment; + } + + void release(MemorySegment segment) { + if (totalAllocatedMemory > inMemoryBufferSize) { + // + // we previously overdraft. + // + segment.free(); + totalAllocatedMemory -= PAGE_SIZE; + return; + } + pool.add(segment); + } + + int getSegmentSize() { + return PAGE_SIZE; + } + + void ensureAtLeastOneSegmentPresent() { + if (!pool.isEmpty()) { + // + // the next allocation would succeeded because the pool is not empty + // + return; + } + if (hasRemainingCapacity()) { + // + // the next allocation would succeeded because the total allocated size is within the allowed + // range + // + return; + } + // + // we overdraft momentarily. + // + MemorySegment segment = MemorySegmentFactory.allocateUnpooledSegment(PAGE_SIZE); + totalAllocatedMemory += PAGE_SIZE; + pool.add(segment); + } + + private boolean hasRemainingCapacity() { + return totalAllocatedMemory + PAGE_SIZE <= inMemoryBufferSize; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/logger/UnboundedFeedbackLogger.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/logger/UnboundedFeedbackLogger.java new file mode 100644 index 00000000..ae2b807f --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/logger/UnboundedFeedbackLogger.java @@ -0,0 +1,135 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.logger; + +import static org.apache.flink.util.Preconditions.checkState; + +import com.ververica.statefun.flink.core.di.Inject; +import com.ververica.statefun.flink.core.di.Label; +import com.ververica.statefun.flink.core.feedback.FeedbackConsumer; +import com.ververica.statefun.flink.core.message.Message; +import com.ververica.statefun.sdk.Address; +import java.io.Closeable; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Objects; +import java.util.TreeMap; +import java.util.function.Supplier; +import java.util.function.ToIntFunction; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.util.IOUtils; + +public final class UnboundedFeedbackLogger implements Closeable { + private final Supplier supplier; + private final ToIntFunction
keyGroupAssigner; + private final Map keyGroupStreams; + private final CheckpointedStreamOperations checkpointedStreamOperations; + + @Nullable private OutputStream keyedStateOutputStream; + private TypeSerializer serializer; + private Closeable snapshotLease; + + @Inject + public UnboundedFeedbackLogger( + @Label("key-group-supplier") Supplier supplier, + @Label("key-group-assigner") ToIntFunction
keyGroupAssigner, + @Label("checkpoint-stream-ops") CheckpointedStreamOperations ops, + @Label("envelope-serializer") TypeSerializer serializer) { + this.supplier = Objects.requireNonNull(supplier); + this.keyGroupAssigner = Objects.requireNonNull(keyGroupAssigner); + this.serializer = Objects.requireNonNull(serializer); + this.keyGroupStreams = new TreeMap<>(); + this.checkpointedStreamOperations = Objects.requireNonNull(ops); + } + + public void startLogging(OutputStream keyedStateCheckpointOutputStream) { + this.checkpointedStreamOperations.requireKeyedStateCheckpointed( + keyedStateCheckpointOutputStream); + this.keyedStateOutputStream = Objects.requireNonNull(keyedStateCheckpointOutputStream); + this.snapshotLease = + checkpointedStreamOperations.acquireLease(keyedStateCheckpointOutputStream); + } + + public void append(Message message) { + if (keyedStateOutputStream == null) { + // + // we are not currently logging. + // + return; + } + KeyGroupStream keyGroup = keyGroupStreamFor(message.target()); + keyGroup.append(message); + } + + public void commit() { + try { + flushToKeyedStateOutputStream(); + } catch (IOException e) { + throw new RuntimeException(e); + } finally { + keyGroupStreams.clear(); + IOUtils.closeQuietly(snapshotLease); + snapshotLease = null; + keyedStateOutputStream = null; + } + } + + private void flushToKeyedStateOutputStream() throws IOException { + checkState(keyedStateOutputStream != null, "Trying to flush envelopes not in a logging state"); + + final DataOutputView target = new DataOutputViewStreamWrapper(keyedStateOutputStream); + for (Entry entry : keyGroupStreams.entrySet()) { + checkpointedStreamOperations.startNewKeyGroup(keyedStateOutputStream, entry.getKey()); + + KeyGroupStream stream = entry.getValue(); + stream.writeTo(target); + } + } + + public void replyLoggedEnvelops( + InputStream rawKeyedStateInputs, FeedbackConsumer consumer) throws Exception { + + DataInputViewStreamWrapper in = new DataInputViewStreamWrapper(rawKeyedStateInputs); + KeyGroupStream.readFrom(in, serializer, consumer); + } + + @Nonnull + private KeyGroupStream keyGroupStreamFor(Address target) { + final int keyGroupId = keyGroupAssigner.applyAsInt(target); + KeyGroupStream keyGroup = keyGroupStreams.get(keyGroupId); + if (keyGroup == null) { + keyGroupStreams.put(keyGroupId, keyGroup = supplier.get()); + } + return keyGroup; + } + + @Override + public void close() { + IOUtils.closeQuietly(snapshotLease); + snapshotLease = null; + keyedStateOutputStream = null; + keyGroupStreams.clear(); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/Message.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/Message.java new file mode 100644 index 00000000..245bdfc4 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/Message.java @@ -0,0 +1,40 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.message; + +import com.ververica.statefun.sdk.Address; +import java.io.IOException; +import javax.annotation.Nullable; +import org.apache.flink.core.memory.DataOutputView; + +public interface Message { + + @Nullable + Address source(); + + Address target(); + + Object payload(MessageFactory context, ClassLoader targetClassLoader); + + boolean isBarrierMessage(); + + Message copy(MessageFactory context); + + void writeTo(MessageFactory context, DataOutputView target) throws IOException; + + default void postApply() {} +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/MessageFactory.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/MessageFactory.java new file mode 100644 index 00000000..7b61bc02 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/MessageFactory.java @@ -0,0 +1,107 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.message; + +import com.ververica.statefun.flink.core.generated.Checkpoint; +import com.ververica.statefun.flink.core.generated.Envelope; +import com.ververica.statefun.flink.core.generated.Payload; +import com.ververica.statefun.flink.core.types.protobuf.ProtobufSerializer; +import com.ververica.statefun.sdk.Address; +import java.io.IOException; +import java.util.Objects; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +public final class MessageFactory { + + public static MessageFactory forType(MessageFactoryType type) { + return new MessageFactory(forPayloadType(type)); + } + + private final ProtobufSerializer envelopeSerializer; + private final MessagePayloadSerializer userMessagePayloadSerializer; + + private MessageFactory(MessagePayloadSerializer userMessagePayloadSerializer) { + this.envelopeSerializer = ProtobufSerializer.forMessageGeneratedClass(Envelope.class); + this.userMessagePayloadSerializer = Objects.requireNonNull(userMessagePayloadSerializer); + } + + public Message from(long checkpointId) { + return from(envelopeWithCheckpointId(checkpointId)); + } + + public Message from(DataInputView input) throws IOException { + return from(deserializeEnvelope(input)); + } + + public Message from(Address from, Address to, Object payload) { + return new SdkMessage(from, to, payload); + } + + // ------------------------------------------------------------------------------------------------------- + + void copy(DataInputView source, DataOutputView target) throws IOException { + copyEnvelope(source, target); + } + + private Message from(Envelope envelope) { + return new ProtobufMessage(envelope); + } + + Payload serializeUserMessagePayload(Object payloadObject) { + return userMessagePayloadSerializer.serialize(payloadObject); + } + + Object deserializeUserMessagePayload(ClassLoader targetClassLoader, Payload payload) { + return userMessagePayloadSerializer.deserialize(targetClassLoader, payload); + } + + Object copyUserMessagePayload(ClassLoader targetClassLoader, Object payload) { + return userMessagePayloadSerializer.copy(targetClassLoader, payload); + } + + void serializeEnvelope(Envelope envelope, DataOutputView target) throws IOException { + envelopeSerializer.serialize(envelope, target); + } + + private Envelope deserializeEnvelope(DataInputView source) throws IOException { + return envelopeSerializer.deserialize(source); + } + + private void copyEnvelope(DataInputView source, DataOutputView target) throws IOException { + envelopeSerializer.copy(source, target); + } + + private static Envelope envelopeWithCheckpointId(long checkpointId) { + Checkpoint checkpoint = Checkpoint.newBuilder().setCheckpointId(checkpointId).build(); + + return Envelope.newBuilder().setCheckpoint(checkpoint).build(); + } + + private static MessagePayloadSerializer forPayloadType(MessageFactoryType type) { + switch (type) { + case WITH_KRYO_PAYLOADS: + return new MessagePayloadSerializerKryo(); + case WITH_PROTOBUF_PAYLOADS: + return new MessagePayloadSerializerPb(); + case WITH_RAW_PAYLOADS: + return new MessagePayloadSerializerRaw(); + default: + throw new IllegalArgumentException("unknown serialization method " + type); + } + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/MessageFactoryType.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/MessageFactoryType.java new file mode 100644 index 00000000..9915d974 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/MessageFactoryType.java @@ -0,0 +1,23 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.message; + +public enum MessageFactoryType { + WITH_KRYO_PAYLOADS, + WITH_PROTOBUF_PAYLOADS, + WITH_RAW_PAYLOADS +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/MessageKeySelector.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/MessageKeySelector.java new file mode 100644 index 00000000..e2ab460f --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/MessageKeySelector.java @@ -0,0 +1,30 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.message; + +import com.ververica.statefun.flink.core.common.KeyBy; +import org.apache.flink.api.java.functions.KeySelector; + +public final class MessageKeySelector implements KeySelector { + + private static final long serialVersionUID = 1; + + @Override + public String getKey(Message value) { + return KeyBy.apply(value.target()); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/MessagePayloadSerializer.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/MessagePayloadSerializer.java new file mode 100644 index 00000000..d96a19e8 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/MessagePayloadSerializer.java @@ -0,0 +1,29 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.message; + +import com.ververica.statefun.flink.core.generated.Payload; +import javax.annotation.Nonnull; + +public interface MessagePayloadSerializer { + + Payload serialize(@Nonnull Object payloadObject); + + Object deserialize(@Nonnull ClassLoader targetClassLoader, @Nonnull Payload payload); + + Object copy(@Nonnull ClassLoader targetClassLoader, @Nonnull Object what); +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/MessagePayloadSerializerKryo.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/MessagePayloadSerializerKryo.java new file mode 100644 index 00000000..8e39d30f --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/MessagePayloadSerializerKryo.java @@ -0,0 +1,84 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.message; + +import com.google.protobuf.ByteString; +import com.ververica.statefun.flink.core.generated.Payload; +import java.io.IOException; +import javax.annotation.Nonnull; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; + +public final class MessagePayloadSerializerKryo implements MessagePayloadSerializer { + + private KryoSerializer kryo = new KryoSerializer<>(Object.class, new ExecutionConfig()); + private DataInputDeserializer source = new DataInputDeserializer(); + private DataOutputSerializer target = new DataOutputSerializer(4096); + + @Override + public Payload serialize(@Nonnull Object payloadObject) { + target.clear(); + try { + kryo.serialize(payloadObject, target); + } catch (IOException e) { + throw new IllegalStateException(e); + } + // TODO: avoid copying, consider adding a zero-copy ByteString. + ByteString serializedBytes = ByteString.copyFrom(target.getSharedBuffer(), 0, target.length()); + return Payload.newBuilder() + .setClassName(payloadObject.getClass().getName()) + .setPayloadBytes(serializedBytes) + .build(); + } + + @Override + public Object deserialize(@Nonnull ClassLoader targetClassLoader, @Nonnull Payload payload) { + source.setBuffer(payload.getPayloadBytes().asReadOnlyByteBuffer()); + try { + return kryo.deserialize(source); + } catch (IOException e) { + throw new IllegalStateException(e); + } + } + + @Override + public Object copy(@Nonnull ClassLoader targetClassLoader, @Nonnull Object what) { + target.clear(); + try { + kryo.serialize(what, target); + source.setBuffer(target.getSharedBuffer(), 0, target.length()); + + final ClassLoader currentClassLoader = Thread.currentThread().getContextClassLoader(); + Thread.currentThread().setContextClassLoader(targetClassLoader); + try { + final ClassLoader originalKryoCl = kryo.getKryo().getClassLoader(); + kryo.getKryo().setClassLoader(targetClassLoader); + try { + return kryo.deserialize(source); + } finally { + kryo.getKryo().setClassLoader(originalKryoCl); + } + } finally { + Thread.currentThread().setContextClassLoader(currentClassLoader); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/MessagePayloadSerializerPb.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/MessagePayloadSerializerPb.java new file mode 100644 index 00000000..06a7e55f --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/MessagePayloadSerializerPb.java @@ -0,0 +1,96 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.message; + +import com.google.protobuf.ByteString; +import com.google.protobuf.InvalidProtocolBufferException; +import com.google.protobuf.Message; +import com.google.protobuf.Parser; +import com.ververica.statefun.flink.core.generated.Payload; +import com.ververica.statefun.flink.core.types.protobuf.ProtobufReflectionUtil; +import it.unimi.dsi.fastutil.objects.ObjectOpenHashMap; +import java.util.Objects; +import javax.annotation.Nonnull; +import javax.annotation.concurrent.NotThreadSafe; + +@NotThreadSafe +public class MessagePayloadSerializerPb implements MessagePayloadSerializer { + + private final ObjectOpenHashMap>> + PARSER_CACHE = new ObjectOpenHashMap<>(); + + @Override + public Object deserialize(@Nonnull ClassLoader targetClassLoader, @Nonnull Payload payload) { + try { + Parser parser = + parserForClassName(targetClassLoader, payload.getClassName()); + return parser.parseFrom(payload.getPayloadBytes()); + } catch (InvalidProtocolBufferException | ClassNotFoundException e) { + throw new IllegalStateException(e); + } + } + + @Override + public Payload serialize(@Nonnull Object what) { + final Message message = (Message) what; + final String className = what.getClass().getName(); + final ByteString body = message.toByteString(); + + return Payload.newBuilder().setClassName(className).setPayloadBytes(body).build(); + } + + @Override + public Object copy(@Nonnull ClassLoader targetClassLoader, @Nonnull Object what) { + Objects.requireNonNull(targetClassLoader); + if (!(what instanceof Message)) { + throw new IllegalStateException(); + } + Message message = (Message) what; + ByteString messageBytes = message.toByteString(); + try { + Parser parser = + parserForClassName(targetClassLoader, what.getClass().getName()); + return parser.parseFrom(messageBytes); + } catch (InvalidProtocolBufferException | ClassNotFoundException e) { + throw new IllegalStateException(e); + } + } + + private Parser parserForClassName( + ClassLoader userCodeClassLoader, String messageClassName) throws ClassNotFoundException { + + ObjectOpenHashMap> classLoaders = + PARSER_CACHE.get(messageClassName); + if (classLoaders == null) { + PARSER_CACHE.put(messageClassName, classLoaders = new ObjectOpenHashMap<>()); + } + Parser parser = classLoaders.get(userCodeClassLoader); + if (parser == null) { + classLoaders.put( + userCodeClassLoader, parser = findParser(userCodeClassLoader, messageClassName)); + } + return parser; + } + + private Parser findParser( + ClassLoader userCodeClassLoader, String messageClassName) throws ClassNotFoundException { + Class messageType = + Class.forName(messageClassName, true, userCodeClassLoader).asSubclass(Message.class); + + return ProtobufReflectionUtil.protobufParser(messageType); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/MessagePayloadSerializerRaw.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/MessagePayloadSerializerRaw.java new file mode 100644 index 00000000..a3a7fd93 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/MessagePayloadSerializerRaw.java @@ -0,0 +1,43 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.message; + +import com.google.protobuf.ByteString; +import com.ververica.statefun.flink.core.generated.Payload; +import javax.annotation.Nonnull; +import javax.annotation.concurrent.NotThreadSafe; + +@NotThreadSafe +public class MessagePayloadSerializerRaw implements MessagePayloadSerializer { + + @Override + public Object deserialize(@Nonnull ClassLoader targetClassLoader, @Nonnull Payload payload) { + return payload.getPayloadBytes().toByteArray(); + } + + @Override + public Payload serialize(@Nonnull Object what) { + byte[] bytes = (byte[]) what; + ByteString bs = ByteString.copyFrom(bytes); + return Payload.newBuilder().setPayloadBytes(bs).build(); + } + + @Override + public Object copy(@Nonnull ClassLoader targetClassLoader, @Nonnull Object what) { + return what; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/MessageTypeInformation.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/MessageTypeInformation.java new file mode 100644 index 00000000..42581108 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/MessageTypeInformation.java @@ -0,0 +1,88 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.message; + +import java.util.Objects; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; + +public class MessageTypeInformation extends TypeInformation { + + private static final long serialVersionUID = 1L; + + private final MessageFactoryType messageFactoryType; + + public MessageTypeInformation(MessageFactoryType messageFactoryType) { + this.messageFactoryType = Objects.requireNonNull(messageFactoryType); + } + + @Override + public boolean isBasicType() { + return false; + } + + @Override + public boolean isTupleType() { + return false; + } + + @Override + public int getArity() { + return 0; + } + + @Override + public int getTotalFields() { + return 0; + } + + @Override + public Class getTypeClass() { + return Message.class; + } + + @Override + public boolean isKeyType() { + return false; + } + + @Override + public TypeSerializer createSerializer(ExecutionConfig executionConfig) { + return new MessageTypeSerializer(messageFactoryType); + } + + @Override + public String toString() { + return "MessageTypeInformation(" + messageFactoryType + ")"; + } + + @Override + public boolean equals(Object o) { + return o instanceof MessageTypeInformation; + } + + @Override + public int hashCode() { + return getClass().hashCode(); + } + + @Override + public boolean canEqual(Object o) { + return o instanceof MessageTypeInformation; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/MessageTypeSerializer.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/MessageTypeSerializer.java new file mode 100644 index 00000000..f7f3fcf7 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/MessageTypeSerializer.java @@ -0,0 +1,157 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.message; + +import java.io.IOException; +import java.util.Objects; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +public final class MessageTypeSerializer extends TypeSerializer { + + private static final long serialVersionUID = 1L; + + // -- configuration -- + private final MessageFactoryType messageFactoryType; + + // -- runtime -- + private transient MessageFactory factory; + + MessageTypeSerializer(MessageFactoryType messageFactoryType) { + this.messageFactoryType = Objects.requireNonNull(messageFactoryType); + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer duplicate() { + return new MessageTypeSerializer(messageFactoryType); + } + + @Override + public Message createInstance() { + return null; + } + + @Override + public Message copy(Message message) { + return message.copy(factory()); + } + + @Override + public Message copy(Message message, Message reuse) { + return message.copy(factory()); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(Message message, DataOutputView dataOutputView) throws IOException { + message.writeTo(factory(), dataOutputView); + } + + @Override + public Message deserialize(DataInputView dataInputView) throws IOException { + return factory().from(dataInputView); + } + + @Override + public Message deserialize(Message message, DataInputView dataInputView) throws IOException { + return deserialize(dataInputView); + } + + @Override + public void copy(DataInputView dataInputView, DataOutputView dataOutputView) throws IOException { + factory().copy(dataInputView, dataOutputView); + } + + @Override + public boolean equals(Object o) { + return o instanceof MessageTypeSerializer; + } + + @Override + public int hashCode() { + return getClass().hashCode(); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new Snapshot(messageFactoryType); + } + + private MessageFactory factory() { + if (factory == null) { + factory = MessageFactory.forType(messageFactoryType); + } + return factory; + } + + public static final class Snapshot implements TypeSerializerSnapshot { + private MessageFactoryType messageFactoryType; + + @SuppressWarnings("unused") + public Snapshot() {} + + Snapshot(MessageFactoryType messageFactoryType) { + this.messageFactoryType = messageFactoryType; + } + + @Override + public int getCurrentVersion() { + return 1; + } + + @Override + public void writeSnapshot(DataOutputView dataOutputView) throws IOException { + dataOutputView.writeUTF(messageFactoryType.name()); + } + + @Override + public void readSnapshot(int version, DataInputView dataInputView, ClassLoader classLoader) + throws IOException { + messageFactoryType = MessageFactoryType.valueOf(dataInputView.readUTF()); + } + + @Override + public TypeSerializer restoreSerializer() { + return new MessageTypeSerializer(messageFactoryType); + } + + @Override + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializer typeSerializer) { + if (!(typeSerializer instanceof MessageTypeSerializer)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + MessageTypeSerializer casted = (MessageTypeSerializer) typeSerializer; + if (casted.messageFactoryType == messageFactoryType) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } + return TypeSerializerSchemaCompatibility.incompatible(); + } + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/ProtobufMessage.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/ProtobufMessage.java new file mode 100644 index 00000000..7c1a1846 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/ProtobufMessage.java @@ -0,0 +1,104 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.message; + +import com.ververica.statefun.flink.core.generated.Envelope; +import com.ververica.statefun.flink.core.generated.EnvelopeAddress; +import com.ververica.statefun.sdk.Address; +import com.ververica.statefun.sdk.FunctionType; +import java.io.IOException; +import java.util.Objects; +import javax.annotation.Nullable; +import org.apache.flink.core.memory.DataOutputView; + +final class ProtobufMessage implements Message { + + private final Envelope envelope; + + private Address source; + private Address target; + private Object payload; + + ProtobufMessage(Envelope envelope) { + this.envelope = Objects.requireNonNull(envelope); + } + + @Override + @Nullable + public Address source() { + if (source != null) { + return source; + } + if ((source = protobufAddressToSdkAddress(envelope.getSource())) == null) { + return null; + } + return source; + } + + @Override + public Address target() { + if (target != null) { + return target; + } + if ((target = protobufAddressToSdkAddress(envelope.getTarget())) == null) { + throw new IllegalStateException("A mandatory target address is missing"); + } + return target; + } + + @Override + public Object payload(MessageFactory factory, ClassLoader targetClassLoader) { + if (payload == null) { + payload = factory.deserializeUserMessagePayload(targetClassLoader, envelope.getPayload()); + } else if (!sameClassLoader(targetClassLoader, payload)) { + payload = factory.copyUserMessagePayload(targetClassLoader, payload); + } + return payload; + } + + @Override + public boolean isBarrierMessage() { + return envelope.hasCheckpoint(); + } + + @Override + public Message copy(MessageFactory unused) { + return new ProtobufMessage(envelope); + } + + @Override + public void writeTo(MessageFactory factory, DataOutputView target) throws IOException { + Objects.requireNonNull(target); + factory.serializeEnvelope(envelope, target); + } + + private static boolean sameClassLoader(ClassLoader targetClassLoader, Object payload) { + return payload.getClass().getClassLoader() == targetClassLoader; + } + + @Nullable + private static Address protobufAddressToSdkAddress(EnvelopeAddress address) { + if (address == null + || (address.getId().isEmpty() + && address.getNamespace().isEmpty() + && address.getType().isEmpty())) { + return null; + } + FunctionType functionType = new FunctionType(address.getNamespace(), address.getType()); + return new Address(functionType, address.getId()); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/SdkMessage.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/SdkMessage.java new file mode 100644 index 00000000..a7a37e5c --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/message/SdkMessage.java @@ -0,0 +1,103 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.message; + +import com.ververica.statefun.flink.core.generated.Envelope; +import com.ververica.statefun.flink.core.generated.Envelope.Builder; +import com.ververica.statefun.flink.core.generated.EnvelopeAddress; +import com.ververica.statefun.sdk.Address; +import java.io.IOException; +import java.util.Objects; +import javax.annotation.Nullable; +import org.apache.flink.core.memory.DataOutputView; + +final class SdkMessage implements Message { + + @Nullable private final Address source; + + private final Address target; + + private Object payload; + + @Nullable private Envelope cachedEnvelope; + + SdkMessage(@Nullable Address source, Address target, Object payload) { + this.source = source; + this.target = Objects.requireNonNull(target); + this.payload = Objects.requireNonNull(payload); + } + + @Override + @Nullable + public Address source() { + return source; + } + + @Override + public Address target() { + return target; + } + + @Override + public Object payload(MessageFactory factory, ClassLoader targetClassLoader) { + if (!sameClassLoader(targetClassLoader, payload)) { + payload = factory.copyUserMessagePayload(targetClassLoader, payload); + } + return payload; + } + + @Override + public boolean isBarrierMessage() { + return false; + } + + @Override + public Message copy(MessageFactory factory) { + return new SdkMessage(source, target, payload); + } + + @Override + public void writeTo(MessageFactory factory, DataOutputView target) throws IOException { + Envelope envelope = envelope(factory); + factory.serializeEnvelope(envelope, target); + } + + private Envelope envelope(MessageFactory factory) { + if (cachedEnvelope == null) { + Builder builder = Envelope.newBuilder(); + if (source != null) { + builder.setSource(sdkAddressToProtobufAddress(source)); + } + builder.setTarget(sdkAddressToProtobufAddress(target)); + builder.setPayload(factory.serializeUserMessagePayload(payload)); + cachedEnvelope = builder.build(); + } + return cachedEnvelope; + } + + private static boolean sameClassLoader(ClassLoader targetClassLoader, Object payload) { + return payload.getClass().getClassLoader() == targetClassLoader; + } + + private static EnvelopeAddress sdkAddressToProtobufAddress(Address source) { + return EnvelopeAddress.newBuilder() + .setNamespace(source.type().namespace()) + .setType(source.type().name()) + .setId(source.id()) + .build(); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/metrics/FlinkFunctionTypeMetrics.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/metrics/FlinkFunctionTypeMetrics.java new file mode 100644 index 00000000..dfa4ceb4 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/metrics/FlinkFunctionTypeMetrics.java @@ -0,0 +1,62 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.metrics; + +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.MeterView; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.SimpleCounter; + +final class FlinkFunctionTypeMetrics implements FunctionTypeMetrics { + private final Counter incoming; + private final Counter outgoingLocalMessage; + private final Counter outgoingRemoteMessage; + private final Counter outgoingEgress; + + FlinkFunctionTypeMetrics(MetricGroup typeGroup) { + this.incoming = metered(typeGroup, "in"); + this.outgoingLocalMessage = metered(typeGroup, "out-local"); + this.outgoingRemoteMessage = metered(typeGroup, "out-remote"); + this.outgoingEgress = metered(typeGroup, "out-egress"); + } + + @Override + public void incomingMessage() { + incoming.inc(); + } + + @Override + public void outgoingLocalMessage() { + this.outgoingLocalMessage.inc(); + } + + @Override + public void outgoingRemoteMessage() { + this.outgoingRemoteMessage.inc(); + } + + @Override + public void outgoingEgressMessage() { + this.outgoingEgress.inc(); + } + + private static SimpleCounter metered(MetricGroup metrics, String name) { + SimpleCounter counter = metrics.counter(name, new SimpleCounter()); + metrics.meter(name + "Rate", new MeterView(counter, 60)); + return counter; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/metrics/FlinkMetricsFactory.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/metrics/FlinkMetricsFactory.java new file mode 100644 index 00000000..2fdb3794 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/metrics/FlinkMetricsFactory.java @@ -0,0 +1,37 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.metrics; + +import com.ververica.statefun.sdk.FunctionType; +import java.util.Objects; +import org.apache.flink.metrics.MetricGroup; + +public class FlinkMetricsFactory implements MetricsFactory { + + private final MetricGroup metricGroup; + + public FlinkMetricsFactory(MetricGroup metricGroup) { + this.metricGroup = Objects.requireNonNull(metricGroup); + } + + @Override + public FunctionTypeMetrics forType(FunctionType functionType) { + MetricGroup namespace = metricGroup.addGroup(functionType.namespace()); + MetricGroup typeGroup = namespace.addGroup(functionType.name()); + return new FlinkFunctionTypeMetrics(typeGroup); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/metrics/FunctionTypeMetrics.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/metrics/FunctionTypeMetrics.java new file mode 100644 index 00000000..bc277f9f --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/metrics/FunctionTypeMetrics.java @@ -0,0 +1,28 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.metrics; + +public interface FunctionTypeMetrics { + + void incomingMessage(); + + void outgoingLocalMessage(); + + void outgoingRemoteMessage(); + + void outgoingEgressMessage(); +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/metrics/MetricsFactory.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/metrics/MetricsFactory.java new file mode 100644 index 00000000..1f5e805b --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/metrics/MetricsFactory.java @@ -0,0 +1,24 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.metrics; + +import com.ververica.statefun.sdk.FunctionType; + +public interface MetricsFactory { + + FunctionTypeMetrics forType(FunctionType functionType); +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/pool/SimplePool.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/pool/SimplePool.java new file mode 100644 index 00000000..224a8c99 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/pool/SimplePool.java @@ -0,0 +1,53 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.pool; + +import java.util.ArrayDeque; +import java.util.Objects; +import java.util.function.Supplier; +import javax.annotation.concurrent.NotThreadSafe; + +/** + * Simple element pool. + * + * @param type of elements being pooled. + */ +@NotThreadSafe +public final class SimplePool { + private final ArrayDeque elements = new ArrayDeque<>(); + private final Supplier supplier; + private final int maxCapacity; + + public SimplePool(Supplier supplier, int maxCapacity) { + this.supplier = Objects.requireNonNull(supplier); + this.maxCapacity = maxCapacity; + } + + public ElementT get() { + ElementT element = elements.pollFirst(); + if (element != null) { + return element; + } + return supplier.get(); + } + + public void release(ElementT item) { + if (elements.size() < maxCapacity) { + elements.addFirst(item); + } + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/queue/Lock.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/queue/Lock.java new file mode 100644 index 00000000..044fe085 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/queue/Lock.java @@ -0,0 +1,24 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.queue; + +public interface Lock { + + void lockUninterruptibly(); + + void unlock(); +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/queue/Locks.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/queue/Locks.java new file mode 100644 index 00000000..d45782e3 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/queue/Locks.java @@ -0,0 +1,87 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.queue; + +import java.util.concurrent.atomic.AtomicLongFieldUpdater; +import java.util.concurrent.locks.ReentrantLock; + +/** Provides few implementations of {@link Lock} interface to be used with {@link MpscQueue}. */ +public final class Locks { + private Locks() {} + + public static Lock spinLock() { + return new YieldingSpinLock(); + } + + public static Lock jdkReentrantLock() { + return new JdkLock(); + } + + // -------------------------------------------------------------------------------------------------------- + // JdkLock + // -------------------------------------------------------------------------------------------------------- + + private static final class JdkLock implements Lock { + private final ReentrantLock lock = new ReentrantLock(true); + + @Override + public void lockUninterruptibly() { + lock.lock(); + } + + @Override + public void unlock() { + lock.unlock(); + } + } + + // -------------------------------------------------------------------------------------------------------- + // YieldingSpinLock + // -------------------------------------------------------------------------------------------------------- + + @SuppressWarnings("unused") + private static class LhsPadding { + protected long p1, p2, p3, p4, p5, p6, p7; + } + + private static class Value extends LhsPadding { + protected volatile long state; + } + + @SuppressWarnings("unused") + private static class RhsPadding extends Value { + protected long p9, p10, p11, p12, p13, p14, p15; + } + + private static final class YieldingSpinLock extends RhsPadding implements Lock { + + private static final AtomicLongFieldUpdater UPDATER = + AtomicLongFieldUpdater.newUpdater(Value.class, "state"); + + @Override + public void lockUninterruptibly() { + while (!UPDATER.compareAndSet(this, 0, 1)) { + Thread.yield(); + } + } + + @Override + public void unlock() { + UPDATER.lazySet(this, 0); + } + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/queue/MpscQueue.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/queue/MpscQueue.java new file mode 100644 index 00000000..ffc325f1 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/queue/MpscQueue.java @@ -0,0 +1,101 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.queue; + +import java.util.ArrayDeque; +import java.util.Deque; +import java.util.Objects; +import org.apache.flink.annotation.Internal; + +/** + * Multi producers single consumer fifo queue. + * + *

This queue supports two operations: + * + *

    + *
  • {@link #add(Object)} atomically adds an element to this queue and returns the number of + * elements in the queue after the addition. + *
  • {@link #drainAll()} atomically obtains a snapshot of the queue and simultaneously empties + * the queue, i.e. drains it. + *
+ * + * @param element type + */ +@Internal +public final class MpscQueue { + + private static final Deque EMPTY = new ArrayDeque<>(0); + + // -- configuration + private final Lock lock; + + // -- runtime + private ArrayDeque active; + private ArrayDeque standby; + + public MpscQueue(int initialBufferSize, Lock lock) { + this.lock = Objects.requireNonNull(lock); + this.active = new ArrayDeque<>(initialBufferSize); + this.standby = new ArrayDeque<>(initialBufferSize); + } + + /** + * Adds an element to this (unbound) queue. + * + * @param element the element to add. + * @return the number of elements in the queue after the addition. + */ + public int add(T element) { + Objects.requireNonNull(element); + final Lock lock = this.lock; + lock.lockUninterruptibly(); + try { + ArrayDeque active = this.active; + active.addLast(element); + return active.size(); + } finally { + lock.unlock(); + } + } + + /** + * Atomically drains the queue. + * + * @return a batch of elements that obtained atomically from that queue. + */ + public Deque drainAll() { + final Lock lock = this.lock; + lock.lockUninterruptibly(); + try { + final ArrayDeque ready = this.active; + if (ready.isEmpty()) { + return empty(); + } + // swap active with standby + this.active = this.standby; + this.standby = ready; + return ready; + } finally { + lock.unlock(); + } + } + + @SuppressWarnings("unchecked") + private static Deque empty() { + return (Deque) EMPTY; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/spi/Modules.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/spi/Modules.java new file mode 100644 index 00000000..bbb5a4e9 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/spi/Modules.java @@ -0,0 +1,78 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.spi; + +import com.ververica.statefun.flink.core.StatefulFunctionsJobConstants; +import com.ververica.statefun.flink.core.StatefulFunctionsUniverse; +import com.ververica.statefun.flink.core.common.SetContextClassLoader; +import com.ververica.statefun.flink.core.message.MessageFactoryType; +import com.ververica.statefun.flink.io.spi.FlinkIoModule; +import com.ververica.statefun.sdk.spi.StatefulFunctionModule; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.ServiceLoader; +import org.apache.flink.configuration.Configuration; + +public final class Modules { + private final List ioModules; + private final List statefulFunctionModules; + + private Modules( + List ioModules, List statefulFunctionModules) { + this.ioModules = ioModules; + this.statefulFunctionModules = statefulFunctionModules; + } + + public static Modules loadFromClassPath() { + List statefulFunctionModules = new ArrayList<>(); + List ioModules = new ArrayList<>(); + + for (StatefulFunctionModule provider : ServiceLoader.load(StatefulFunctionModule.class)) { + statefulFunctionModules.add(provider); + } + for (FlinkIoModule provider : ServiceLoader.load(FlinkIoModule.class)) { + ioModules.add(provider); + } + return new Modules(ioModules, statefulFunctionModules); + } + + public StatefulFunctionsUniverse createStatefulFunctionsUniverse(Configuration configuration) { + MessageFactoryType factoryType = + configuration.getEnum( + MessageFactoryType.class, StatefulFunctionsJobConstants.USER_MESSAGE_SERIALIZER); + + StatefulFunctionsUniverse universe = new StatefulFunctionsUniverse(factoryType); + + final Map globalConfiguration = + Collections.unmodifiableMap(configuration.toMap()); + + for (FlinkIoModule module : ioModules) { + try (SetContextClassLoader ignored = new SetContextClassLoader(module)) { + module.configure(globalConfiguration, universe); + } + } + for (StatefulFunctionModule module : statefulFunctionModules) { + try (SetContextClassLoader ignored = new SetContextClassLoader(module)) { + module.configure(globalConfiguration, universe); + } + } + + return universe; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/state/BoundState.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/state/BoundState.java new file mode 100644 index 00000000..ed4e6295 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/state/BoundState.java @@ -0,0 +1,35 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.state; + +import com.ververica.statefun.sdk.state.PersistedValue; +import java.util.ArrayList; +import java.util.List; + +public class BoundState { + + private final List> persistedValues; + + BoundState(List> persistedValues) { + this.persistedValues = new ArrayList<>(persistedValues); + } + + @SuppressWarnings("unused") + public List> persistedValues() { + return persistedValues; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/state/FlinkState.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/state/FlinkState.java new file mode 100644 index 00000000..1568f434 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/state/FlinkState.java @@ -0,0 +1,69 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.state; + +import com.ververica.statefun.flink.core.common.KeyBy; +import com.ververica.statefun.flink.core.di.Inject; +import com.ververica.statefun.flink.core.di.Label; +import com.ververica.statefun.flink.core.types.DynamicallyRegisteredTypes; +import com.ververica.statefun.sdk.Address; +import com.ververica.statefun.sdk.FunctionType; +import com.ververica.statefun.sdk.state.Accessor; +import com.ververica.statefun.sdk.state.PersistedValue; +import java.util.Objects; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.runtime.state.KeyedStateBackend; + +public final class FlinkState implements State { + + private final RuntimeContext runtimeContext; + private final KeyedStateBackend keyedStateBackend; + private final DynamicallyRegisteredTypes types; + + @Inject + public FlinkState( + @Label("runtime-context") RuntimeContext runtimeContext, + @Label("keyed-state-backend") KeyedStateBackend keyedStateBackend, + DynamicallyRegisteredTypes types) { + + this.runtimeContext = Objects.requireNonNull(runtimeContext); + this.keyedStateBackend = Objects.requireNonNull(keyedStateBackend); + this.types = Objects.requireNonNull(types); + } + + @Override + public Accessor createFlinkStateAccessor( + FunctionType functionType, PersistedValue persistedValue) { + TypeInformation typeInfo = types.registerType(persistedValue.type()); + String stateName = flinkStateName(functionType, persistedValue.name()); + ValueStateDescriptor descriptor = new ValueStateDescriptor<>(stateName, typeInfo); + ValueState handle = runtimeContext.getState(descriptor); + return new FlinkValueAccessor<>(handle); + } + + @Override + public void setCurrentKey(Address address) { + keyedStateBackend.setCurrentKey(KeyBy.apply(address)); + } + + private static String flinkStateName(FunctionType functionType, String name) { + return String.format("%s.%s.%s", functionType.namespace(), functionType.name(), name); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/state/FlinkValueAccessor.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/state/FlinkValueAccessor.java new file mode 100644 index 00000000..7717d43f --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/state/FlinkValueAccessor.java @@ -0,0 +1,58 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.state; + +import com.ververica.statefun.sdk.state.Accessor; +import java.io.IOException; +import java.util.Objects; +import org.apache.flink.api.common.state.ValueState; + +final class FlinkValueAccessor implements Accessor { + + private final ValueState handle; + + FlinkValueAccessor(ValueState handle) { + this.handle = Objects.requireNonNull(handle); + } + + @Override + public void set(T value) { + try { + if (value == null) { + handle.clear(); + } else { + handle.update(value); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public T get() { + try { + return handle.value(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public void clear() { + handle.clear(); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/state/MultiplexedMapStateAccessor.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/state/MultiplexedMapStateAccessor.java new file mode 100644 index 00000000..0779b229 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/state/MultiplexedMapStateAccessor.java @@ -0,0 +1,101 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.state; + +import com.ververica.statefun.sdk.state.Accessor; +import java.io.IOException; +import java.util.Objects; +import org.apache.flink.api.common.state.MapState; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; + +final class MultiplexedMapStateAccessor implements Accessor { + private final MapState mapStateHandle; + private final byte[] accessorMapKey; + private final RawSerializer serializer; + + MultiplexedMapStateAccessor( + MapState handle, + byte[] accessorMapKey, + TypeSerializer subValueSerializer) { + this.mapStateHandle = Objects.requireNonNull(handle); + this.accessorMapKey = Objects.requireNonNull(accessorMapKey); + this.serializer = new RawSerializer<>(subValueSerializer); + } + + @Override + public void set(T value) { + try { + if (value == null) { + mapStateHandle.remove(accessorMapKey); + } else { + byte[] bytes = serializer.serialize(value); + mapStateHandle.put(accessorMapKey, bytes); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + public T get() { + try { + final byte[] bytes = mapStateHandle.get(accessorMapKey); + if (bytes == null) { + return null; + } + return serializer.deserialize(bytes); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + public void clear() { + try { + mapStateHandle.remove(accessorMapKey); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + private static final class RawSerializer { + private final TypeSerializer delegate; + private final DataOutputSerializer output; + private final DataInputDeserializer input; + + RawSerializer(TypeSerializer delegate) { + this.delegate = Objects.requireNonNull(delegate); + this.output = new DataOutputSerializer(32); + this.input = new DataInputDeserializer(); + } + + byte[] serialize(T value) throws IOException { + output.clear(); + delegate.serialize(value, output); + return output.getCopyOfBuffer(); // TODO: consider avoiding buffer copying + } + + T deserialize(byte[] bytes) throws IOException { + input.setBuffer(bytes); + final T value = delegate.deserialize(input); + input.releaseArrays(); + return value; + } + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/state/MultiplexedState.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/state/MultiplexedState.java new file mode 100644 index 00000000..2002a610 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/state/MultiplexedState.java @@ -0,0 +1,91 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.state; + +import com.ververica.statefun.flink.core.common.KeyBy; +import com.ververica.statefun.flink.core.di.Inject; +import com.ververica.statefun.flink.core.di.Label; +import com.ververica.statefun.flink.core.types.DynamicallyRegisteredTypes; +import com.ververica.statefun.sdk.Address; +import com.ververica.statefun.sdk.FunctionType; +import com.ververica.statefun.sdk.state.Accessor; +import com.ververica.statefun.sdk.state.PersistedValue; +import java.util.Objects; +import org.apache.commons.io.Charsets; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.state.MapState; +import org.apache.flink.api.common.state.MapStateDescriptor; +import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.state.KeyedStateBackend; + +public final class MultiplexedState implements State { + + private final KeyedStateBackend keyedStateBackend; + private final DynamicallyRegisteredTypes types; + private final MapState sharedMapStateHandle; + private final ExecutionConfig executionConfiguration; + + @Inject + public MultiplexedState( + @Label("runtime-context") RuntimeContext runtimeContext, + @Label("keyed-state-backend") KeyedStateBackend keyedStateBackend, + DynamicallyRegisteredTypes types) { + + this.keyedStateBackend = Objects.requireNonNull(keyedStateBackend); + this.types = Objects.requireNonNull(types); + this.sharedMapStateHandle = createSharedMapState(runtimeContext); + this.executionConfiguration = Objects.requireNonNull(runtimeContext.getExecutionConfig()); + } + + @Override + public Accessor createFlinkStateAccessor( + FunctionType functionType, PersistedValue persistedValue) { + final byte[] uniqueSubKey = multiplexedSubstateKey(functionType, persistedValue.name()); + final TypeSerializer valueSerializer = multiplexedSubstateValueSerializer(persistedValue); + return new MultiplexedMapStateAccessor<>(sharedMapStateHandle, uniqueSubKey, valueSerializer); + } + + @Override + public void setCurrentKey(Address address) { + keyedStateBackend.setCurrentKey(KeyBy.apply(address)); + } + + private TypeSerializer multiplexedSubstateValueSerializer( + PersistedValue persistedValue) { + TypeInformation typeInfo = types.registerType(persistedValue.type()); + return typeInfo.createSerializer(executionConfiguration); + } + + private static MapState createSharedMapState(RuntimeContext runtimeContext) { + MapStateDescriptor descriptor = + new MapStateDescriptor<>( + "state", + PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO, + PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO); + + return runtimeContext.getMapState(descriptor); + } + + private static byte[] multiplexedSubstateKey(FunctionType functionType, String name) { + String stateKey = + String.format("%s.%s.%s", functionType.namespace(), functionType.name(), name); + return stateKey.getBytes(Charsets.UTF_8); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/state/PersistedValues.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/state/PersistedValues.java new file mode 100644 index 00000000..91194b04 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/state/PersistedValues.java @@ -0,0 +1,81 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.state; + +import com.ververica.statefun.sdk.annotations.Persisted; +import com.ververica.statefun.sdk.state.PersistedValue; +import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.List; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +final class PersistedValues { + + static List> findReflectively(@Nullable Object instance) { + PersistedValues visitor = new PersistedValues(); + visitor.visit(instance); + return visitor.getPersistedValues(); + } + + private final List> persistedValues = new ArrayList<>(); + + private void visit(@Nullable Object instance) { + if (instance == null) { + return; + } + for (Field field : instance.getClass().getDeclaredFields()) { + visitField(instance, field); + } + } + + private List> getPersistedValues() { + return persistedValues; + } + + private void visitField(@Nonnull Object instance, @Nonnull Field field) { + @Nonnull Persisted[] annotationsByType = field.getAnnotationsByType(Persisted.class); + if (annotationsByType.length == 0) { + return; + } + if (field.getType() != PersistedValue.class) { + throw new IllegalArgumentException( + "Unknown persisted value type " + + field.getType() + + " on " + + instance.getClass().getName()); + } + PersistedValue persistedValue = getPersistedValueReflectively(instance, field); + if (persistedValue == null) { + throw new IllegalStateException( + "The field " + field + " of a " + instance.getClass().getName() + " was not initialized"); + } + persistedValues.add(persistedValue); + } + + @SuppressWarnings("unchecked") + private static PersistedValue getPersistedValueReflectively( + Object instance, Field persistedValueField) { + try { + persistedValueField.setAccessible(true); + return (PersistedValue) persistedValueField.get(instance); + } catch (IllegalAccessException e) { + throw new RuntimeException( + "Unable access field " + persistedValueField.getName() + " of " + instance.getClass()); + } + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/state/State.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/state/State.java new file mode 100644 index 00000000..3865849f --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/state/State.java @@ -0,0 +1,30 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.state; + +import com.ververica.statefun.sdk.Address; +import com.ververica.statefun.sdk.FunctionType; +import com.ververica.statefun.sdk.state.Accessor; +import com.ververica.statefun.sdk.state.PersistedValue; + +public interface State { + + Accessor createFlinkStateAccessor( + FunctionType functionType, PersistedValue persistedValue); + + void setCurrentKey(Address address); +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/state/StateBinder.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/state/StateBinder.java new file mode 100644 index 00000000..6106c519 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/state/StateBinder.java @@ -0,0 +1,47 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.state; + +import com.ververica.statefun.flink.core.di.Inject; +import com.ververica.statefun.flink.core.di.Label; +import com.ververica.statefun.sdk.FunctionType; +import com.ververica.statefun.sdk.state.Accessor; +import com.ververica.statefun.sdk.state.ApiExtension; +import com.ververica.statefun.sdk.state.PersistedValue; +import java.util.List; +import java.util.Objects; +import javax.annotation.Nullable; + +public final class StateBinder { + private final State state; + + @Inject + StateBinder(@Label("state") State state) { + this.state = Objects.requireNonNull(state); + } + + public BoundState bind(FunctionType functionType, @Nullable Object instance) { + List> values = PersistedValues.findReflectively(instance); + + for (PersistedValue persistedValue : values) { + Accessor accessor = state.createFlinkStateAccessor(functionType, persistedValue); + ApiExtension.setPersistedValueAccessor(persistedValue, accessor); + } + + return new BoundState(values); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/CheckpointToMessage.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/CheckpointToMessage.java new file mode 100644 index 00000000..34759dc5 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/CheckpointToMessage.java @@ -0,0 +1,47 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.translation; + +import com.ververica.statefun.flink.core.message.Message; +import com.ververica.statefun.flink.core.message.MessageFactory; +import com.ververica.statefun.flink.core.message.MessageFactoryType; +import java.io.Serializable; +import java.util.function.LongFunction; + +final class CheckpointToMessage implements Serializable, LongFunction { + + private static final long serialVersionUID = 1L; + + private final MessageFactoryType messageFactoryType; + private transient MessageFactory factory; + + CheckpointToMessage(MessageFactoryType messageFactoryType) { + this.messageFactoryType = messageFactoryType; + } + + @Override + public Message apply(long checkpointId) { + return factory().from(checkpointId); + } + + private MessageFactory factory() { + if (factory == null) { + factory = MessageFactory.forType(messageFactoryType); + } + return factory; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/DecoratedSink.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/DecoratedSink.java new file mode 100644 index 00000000..289209a9 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/DecoratedSink.java @@ -0,0 +1,46 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.translation; + +import com.ververica.statefun.sdk.io.EgressIdentifier; +import com.ververica.statefun.sdk.io.EgressSpec; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; + +final class DecoratedSink { + final String name; + + final String uid; + + final SinkFunction sink; + + private DecoratedSink(String name, String uid, SinkFunction sink) { + this.name = name; + this.uid = uid; + this.sink = sink; + } + + public static DecoratedSink of(EgressSpec spec, SinkFunction sink) { + EgressIdentifier identifier = spec.id(); + String name = String.format("%s-%s-egress", identifier.namespace(), identifier.name()); + String uid = + String.format( + "%s-%s-%s-%s-egress", + spec.type().namespace(), spec.type().type(), identifier.namespace(), identifier.name()); + + return new DecoratedSink(name, uid, sink); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/DecoratedSource.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/DecoratedSource.java new file mode 100644 index 00000000..ed9a745a --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/DecoratedSource.java @@ -0,0 +1,46 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.translation; + +import com.ververica.statefun.sdk.io.IngressIdentifier; +import com.ververica.statefun.sdk.io.IngressSpec; +import org.apache.flink.streaming.api.functions.source.SourceFunction; + +final class DecoratedSource { + final String name; + + final String uid; + + final SourceFunction source; + + private DecoratedSource(String name, String uid, SourceFunction source) { + this.name = name; + this.uid = uid; + this.source = source; + } + + public static DecoratedSource of(IngressSpec spec, SourceFunction source) { + IngressIdentifier identifier = spec.id(); + String name = String.format("%s-%s-ingress", identifier.namespace(), identifier.name()); + String uid = + String.format( + "%s-%s-%s-%s-ingress", + spec.type().namespace(), spec.type().type(), identifier.namespace(), identifier.name()); + + return new DecoratedSource(name, uid, source); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/EgressToSinkTranslator.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/EgressToSinkTranslator.java new file mode 100644 index 00000000..1319b331 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/EgressToSinkTranslator.java @@ -0,0 +1,44 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.translation; + +import com.ververica.statefun.flink.core.StatefulFunctionsUniverse; +import com.ververica.statefun.flink.io.spi.SinkProvider; +import com.ververica.statefun.sdk.io.EgressIdentifier; +import com.ververica.statefun.sdk.io.EgressSpec; +import java.util.Map; +import java.util.Objects; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; + +final class EgressToSinkTranslator { + private final StatefulFunctionsUniverse universe; + + EgressToSinkTranslator(StatefulFunctionsUniverse universe) { + this.universe = Objects.requireNonNull(universe); + } + + Map, DecoratedSink> translate() { + return Maps.transformValues(universe.egress(), this::sinkFromSpec); + } + + private DecoratedSink sinkFromSpec(EgressSpec spec) { + SinkProvider provider = universe.sinks().get(spec.type()); + SinkFunction sink = provider.forSpec(spec); + + return DecoratedSink.of(spec, sink); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/FlinkUniverse.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/FlinkUniverse.java new file mode 100644 index 00000000..b84ef547 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/FlinkUniverse.java @@ -0,0 +1,94 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.translation; + +import com.ververica.statefun.flink.core.StatefulFunctionsJobConstants; +import com.ververica.statefun.flink.core.StatefulFunctionsUniverse; +import com.ververica.statefun.flink.core.feedback.FeedbackKey; +import com.ververica.statefun.flink.core.feedback.FeedbackSinkOperator; +import com.ververica.statefun.flink.core.functions.FunctionGroupOperator; +import com.ververica.statefun.flink.core.message.Message; +import com.ververica.statefun.flink.core.message.MessageKeySelector; +import com.ververica.statefun.sdk.io.EgressIdentifier; +import java.util.Map; +import java.util.Objects; +import java.util.function.LongFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.util.OutputTag; + +public final class FlinkUniverse { + private static final FeedbackKey FEEDBACK_KEY = + new FeedbackKey<>("stateful-functions-pipeline", 1); + private final StatefulFunctionsUniverse universe; + + public FlinkUniverse(StatefulFunctionsUniverse universe) { + this.universe = Objects.requireNonNull(universe); + } + + public void configure(StreamExecutionEnvironment env) { + Sources sources = Sources.create(env, universe); + Sinks sinks = Sinks.create(universe); + + SingleOutputStreamOperator functionOutputStream = + functionOperator(sources.unionStream(), sinks.sideOutputTags()); + SingleOutputStreamOperator writeBackOut = feedbackOperator(functionOutputStream); + + coLocate(functionOutputStream, writeBackOut); + + sinks.consumeFrom(functionOutputStream); + } + + private SingleOutputStreamOperator feedbackOperator( + SingleOutputStreamOperator functionOut) { + + LongFunction toMessage = new CheckpointToMessage(universe.messageFactoryType()); + + FeedbackSinkOperator sinkOperator = + new FeedbackSinkOperator<>(FEEDBACK_KEY, toMessage); + + return functionOut + .keyBy(new MessageKeySelector()) + .transform( + StatefulFunctionsJobConstants.WRITE_BACK_OPERATOR_NAME, + TypeInformation.of(Void.class), + sinkOperator) + .uid(StatefulFunctionsJobConstants.WRITE_BACK_OPERATOR_UID); + } + + private SingleOutputStreamOperator functionOperator( + DataStream input, Map, OutputTag> sideOutputs) { + + TypeInformation typeInfo = universe.types().registerType(Message.class); + + FunctionGroupOperator operator = new FunctionGroupOperator(FEEDBACK_KEY, sideOutputs); + + return input + .keyBy(new MessageKeySelector()) + .transform(StatefulFunctionsJobConstants.FUNCTION_OPERATOR_NAME, typeInfo, operator) + .uid(StatefulFunctionsJobConstants.FUNCTION_OPERATOR_UID); + } + + private static void coLocate(DataStream a, DataStream b) { + String stringKey = FEEDBACK_KEY.asColocationKey(); + a.getTransformation().setCoLocationGroupKey(stringKey); + b.getTransformation().setCoLocationGroupKey(stringKey); + a.getTransformation().setParallelism(b.getParallelism()); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/IngressRouterFlatMap.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/IngressRouterFlatMap.java new file mode 100644 index 00000000..be4f316c --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/IngressRouterFlatMap.java @@ -0,0 +1,118 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.translation; + +import com.ververica.statefun.flink.core.StatefulFunctionsUniverse; +import com.ververica.statefun.flink.core.StatefulFunctionsUniverses; +import com.ververica.statefun.flink.core.message.Message; +import com.ververica.statefun.flink.core.message.MessageFactory; +import com.ververica.statefun.sdk.Address; +import com.ververica.statefun.sdk.io.IngressIdentifier; +import com.ververica.statefun.sdk.io.Router; +import com.ververica.statefun.sdk.io.Router.Downstream; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import org.apache.flink.api.common.ExecutionConfig.GlobalJobParameters; +import org.apache.flink.api.common.functions.RichFlatMapFunction; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.util.Collector; +import org.apache.flink.util.Preconditions; + +public final class IngressRouterFlatMap extends RichFlatMapFunction { + + private static final long serialVersionUID = 1; + + private final IngressIdentifier id; + private transient List> routers; + private transient DownstreamCollector downstream; + + IngressRouterFlatMap(IngressIdentifier id) { + this.id = Objects.requireNonNull(id); + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + + Configuration configuration = combineWithGlobalJobConfiguration(parameters); + StatefulFunctionsUniverse universe = + StatefulFunctionsUniverses.get( + Thread.currentThread().getContextClassLoader(), configuration); + + this.downstream = + new DownstreamCollector<>(MessageFactory.forType(universe.messageFactoryType())); + this.routers = loadRoutersAttachedToIngress(id, universe.routers()); + } + + @Override + public void flatMap(T in, Collector collector) { + downstream.collector = collector; + for (Router router : routers) { + router.route(in, downstream); + } + } + + private Configuration combineWithGlobalJobConfiguration(Configuration parameters) { + Configuration combined = new Configuration(); + combined.addAll(parameters); + + GlobalJobParameters globalJobParameters = + getRuntimeContext().getExecutionConfig().getGlobalJobParameters(); + + Preconditions.checkState(globalJobParameters instanceof Configuration); + Configuration configuration = (Configuration) globalJobParameters; + + combined.addAll(configuration); + return combined; + } + + @SuppressWarnings("unchecked") + private static List> loadRoutersAttachedToIngress( + IngressIdentifier id, Map, List>> definedRouters) { + + List> routerList = definedRouters.get(id); + Preconditions.checkState(routerList != null, "unable to find a router for ingress " + id); + return (List>) (List) routerList; + } + + private static final class DownstreamCollector implements Downstream { + + private final MessageFactory factory; + + Collector collector; + + DownstreamCollector(MessageFactory factory) { + this.factory = Objects.requireNonNull(factory); + } + + @Override + public void forward(Address to, Object message) { + if (to == null) { + throw new NullPointerException("Unable to send a message downstream without an address."); + } + if (message == null) { + throw new NullPointerException("message is mandatory parameter and can not be NULL."); + } + // + // set the envelope + // + Message message1 = factory.from(null, to, message); + collector.collect(message1); + } + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/IngressToSourceFunctionTranslator.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/IngressToSourceFunctionTranslator.java new file mode 100644 index 00000000..15eca25e --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/IngressToSourceFunctionTranslator.java @@ -0,0 +1,54 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.translation; + +import com.ververica.statefun.flink.core.StatefulFunctionsUniverse; +import com.ververica.statefun.flink.io.spi.SourceProvider; +import com.ververica.statefun.sdk.io.IngressIdentifier; +import com.ververica.statefun.sdk.io.IngressSpec; +import java.util.Map; +import java.util.Objects; +import org.apache.flink.streaming.api.functions.source.SourceFunction; + +final class IngressToSourceFunctionTranslator { + private final StatefulFunctionsUniverse universe; + + IngressToSourceFunctionTranslator(StatefulFunctionsUniverse universe) { + this.universe = Objects.requireNonNull(universe); + } + + Map, DecoratedSource> translate() { + return Maps.transformValues(universe.ingress(), this::sourceFromSpec); + } + + private DecoratedSource sourceFromSpec(IngressIdentifier key, IngressSpec spec) { + SourceProvider provider = universe.sources().get(spec.type()); + if (provider == null) { + throw new IllegalStateException( + "Unable to find a source translation for ingress of type " + + spec.type() + + ", which is bound for key " + + key); + } + SourceFunction source = provider.forSpec(spec); + if (source == null) { + throw new NullPointerException( + "A source provider for type " + spec.type() + ", has produced a NULL source."); + } + return DecoratedSource.of(spec, source); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/Maps.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/Maps.java new file mode 100644 index 00000000..c1b73cd5 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/Maps.java @@ -0,0 +1,48 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.translation; + +import java.util.HashMap; +import java.util.Map; +import java.util.function.BiFunction; +import java.util.function.Function; + +final class Maps { + private Maps() {} + + static Map transformValues(Map map, Function fn) { + Map result = new HashMap<>(); + + for (Map.Entry entry : map.entrySet()) { + U u = fn.apply(entry.getValue()); + result.put(entry.getKey(), u); + } + + return result; + } + + static Map transformValues(Map map, BiFunction fn) { + Map result = new HashMap<>(); + + for (Map.Entry entry : map.entrySet()) { + U u = fn.apply(entry.getKey(), entry.getValue()); + result.put(entry.getKey(), u); + } + + return result; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/RouterTranslator.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/RouterTranslator.java new file mode 100644 index 00000000..6b61cfa4 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/RouterTranslator.java @@ -0,0 +1,65 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.translation; + +import com.ververica.statefun.flink.core.StatefulFunctionsJobConstants; +import com.ververica.statefun.flink.core.StatefulFunctionsUniverse; +import com.ververica.statefun.flink.core.message.Message; +import com.ververica.statefun.sdk.io.IngressIdentifier; +import com.ververica.statefun.sdk.io.IngressSpec; +import java.util.Map; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.api.datastream.DataStream; + +final class RouterTranslator { + private final StatefulFunctionsUniverse universe; + + RouterTranslator(StatefulFunctionsUniverse universe) { + this.universe = universe; + } + + Map, DataStream> translate( + Map, DataStream> sources) { + return Maps.transformValues( + universe.routers(), (id, unused) -> createRoutersForSource(id, sources.get(id))); + } + + /** + * For each input {@linkplain DataStream} (created as a result of {@linkplain IngressSpec} + * translation) we attach a single FlatMap function that would invoke all the defined routers for + * that spec. Please note that the FlatMap function must have the same parallelism as the + * {@linkplain DataStream} it is attached to, so that we keep per key ordering. + */ + @SuppressWarnings("unchecked") + private DataStream createRoutersForSource( + IngressIdentifier id, DataStream sourceStream) { + IngressIdentifier castedId = (IngressIdentifier) id; + DataStream castedSource = (DataStream) sourceStream; + + IngressRouterFlatMap router = new IngressRouterFlatMap<>(castedId); + + TypeInformation typeInfo = universe.types().registerType(Message.class); + + int sourceParallelism = castedSource.getParallelism(); + + return castedSource + .flatMap(router) + .name(StatefulFunctionsJobConstants.ROUTER_NAME + " (" + castedId.name() + ")") + .returns(typeInfo) + .setParallelism(sourceParallelism); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/SideOutputTranslator.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/SideOutputTranslator.java new file mode 100644 index 00000000..e9c74573 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/SideOutputTranslator.java @@ -0,0 +1,46 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.translation; + +import com.ververica.statefun.flink.core.StatefulFunctionsUniverse; +import com.ververica.statefun.flink.core.types.StaticallyRegisteredTypes; +import com.ververica.statefun.sdk.io.EgressIdentifier; +import java.util.Map; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.util.OutputTag; + +final class SideOutputTranslator { + private final StatefulFunctionsUniverse universe; + + SideOutputTranslator(StatefulFunctionsUniverse universe) { + this.universe = universe; + } + + private static OutputTag outputTagFromId( + EgressIdentifier id, StaticallyRegisteredTypes types) { + @SuppressWarnings("unchecked") + EgressIdentifier casted = (EgressIdentifier) id; + String name = String.format("%s.%s", id.namespace(), id.name()); + TypeInformation typeInformation = types.registerType(casted.consumedType()); + return new OutputTag<>(name, typeInformation); + } + + Map, OutputTag> translate() { + return Maps.transformValues( + universe.egress(), (id, unused) -> outputTagFromId(id, universe.types())); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/Sinks.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/Sinks.java new file mode 100644 index 00000000..89db9e9a --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/Sinks.java @@ -0,0 +1,75 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.translation; + +import com.ververica.statefun.flink.core.StatefulFunctionsUniverse; +import com.ververica.statefun.sdk.io.EgressIdentifier; +import java.util.Map; +import java.util.Objects; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.util.OutputTag; + +final class Sinks { + private final Map, OutputTag> sideOutputs; + private final Map, DecoratedSink> sinks; + + private Sinks( + Map, OutputTag> sideOutputs, + Map, DecoratedSink> sinks) { + + this.sideOutputs = Objects.requireNonNull(sideOutputs); + this.sinks = Objects.requireNonNull(sinks); + } + + static Sinks create(StatefulFunctionsUniverse universe) { + return new Sinks(sideOutputs(universe), sinkFunctions(universe)); + } + + private static Map, DecoratedSink> sinkFunctions( + StatefulFunctionsUniverse universe) { + EgressToSinkTranslator egressTranslator = new EgressToSinkTranslator(universe); + return egressTranslator.translate(); + } + + private static Map, OutputTag> sideOutputs( + StatefulFunctionsUniverse universe) { + SideOutputTranslator sideOutputTranslator = new SideOutputTranslator(universe); + return sideOutputTranslator.translate(); + } + + Map, OutputTag> sideOutputTags() { + return sideOutputs; + } + + void consumeFrom(SingleOutputStreamOperator mainOutput) { + sideOutputs.forEach( + (id, tag) -> { + final DataStream sideOutputStream = mainOutput.getSideOutput(tag); + + DecoratedSink decoratedSink = sinks.get(id); + @SuppressWarnings("unchecked") + SinkFunction sink = (SinkFunction) decoratedSink.sink; + + DataStreamSink streamSink = sideOutputStream.addSink(sink); + streamSink.name(decoratedSink.name); + streamSink.uid(decoratedSink.uid); + }); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/Sources.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/Sources.java new file mode 100644 index 00000000..7233318b --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/translation/Sources.java @@ -0,0 +1,120 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.translation; + +import com.ververica.statefun.flink.core.StatefulFunctionsUniverse; +import com.ververica.statefun.flink.core.message.Message; +import com.ververica.statefun.flink.core.types.StaticallyRegisteredTypes; +import com.ververica.statefun.sdk.io.IngressIdentifier; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + +final class Sources { + + private final DataStream sourceUnion; + + private Sources(DataStream union) { + this.sourceUnion = union; + } + + static Sources create(StreamExecutionEnvironment env, StatefulFunctionsUniverse universe) { + final Map, DecoratedSource> sourceFunctions = + ingressToSourceFunction(universe); + + final Map, DataStream> sourceStreams = + sourceFunctionToDataStream(env, universe, sourceFunctions); + + final Map, DataStream> envelopeSources = + dataStreamToEnvelopStream(universe, sourceStreams); + + return new Sources(union(envelopeSources.values())); + } + + private static Map, DataStream> dataStreamToEnvelopStream( + StatefulFunctionsUniverse universe, Map, DataStream> sourceStreams) { + + RouterTranslator routerTranslator = new RouterTranslator(universe); + return routerTranslator.translate(sourceStreams); + } + + private static Map, DataStream> sourceFunctionToDataStream( + StreamExecutionEnvironment env, + StatefulFunctionsUniverse universe, + Map, DecoratedSource> sourceFunctions) { + + Map, DataStream> sourceStreams = new HashMap<>(); + sourceFunctions.forEach( + (id, sourceFunction) -> { + DataStreamSource stream = env.addSource(sourceFunction.source); + + stream.name(sourceFunction.name); + stream.uid(sourceFunction.uid); + + // we have to overwrite the TypeInformation of the produced type + // because @sourceFunction might be of type ResultTypeQueryable, + // but we need to use the type information that is derived from + // the ingress identifier type(). + setOutputType(universe.types(), id.producedType(), stream.getTransformation()); + + sourceStreams.put(id, stream); + }); + return sourceStreams; + } + + @SuppressWarnings({"unchecked", "raw"}) + private static void setOutputType( + StaticallyRegisteredTypes types, Class type, Transformation transformation) { + + TypeInformation typeInfo = types.registerType(type); + transformation.setOutputType(typeInfo); + } + + private static Map, DecoratedSource> ingressToSourceFunction( + StatefulFunctionsUniverse universe) { + IngressToSourceFunctionTranslator translator = new IngressToSourceFunctionTranslator(universe); + return translator.translate(); + } + + private static DataStream union(Collection> sources) { + if (sources.isEmpty()) { + throw new IllegalStateException("There are no routers defined."); + } + final int sourceCount = sources.size(); + final Iterator> iterator = sources.iterator(); + if (sourceCount == 1) { + return iterator.next(); + } + DataStream first = iterator.next(); + @SuppressWarnings("unchecked") + DataStream[] rest = new DataStream[sourceCount - 1]; + for (int i = 0; i < sourceCount - 1; i++) { + rest[i] = iterator.next(); + } + return first.union(rest); + } + + DataStream unionStream() { + return sourceUnion; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/types/DynamicallyRegisteredTypes.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/types/DynamicallyRegisteredTypes.java new file mode 100644 index 00000000..59df3108 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/types/DynamicallyRegisteredTypes.java @@ -0,0 +1,61 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.types; + +import com.google.protobuf.Message; +import com.ververica.statefun.flink.core.types.protobuf.ProtobufTypeInformation; +import com.ververica.statefun.sdk.state.PersistedValue; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import javax.annotation.concurrent.NotThreadSafe; +import org.apache.flink.api.common.typeinfo.TypeInformation; + +/** + * DynamicallyRegisteredTypes are types that are types that were discovered during runtime, for + * example registered {@linkplain PersistedValue}s. + */ +@NotThreadSafe +public final class DynamicallyRegisteredTypes { + + private final StaticallyRegisteredTypes staticallyKnownTypes; + private final Map, TypeInformation> registeredTypes = new HashMap<>(); + + public DynamicallyRegisteredTypes(StaticallyRegisteredTypes staticallyKnownTypes) { + this.staticallyKnownTypes = Objects.requireNonNull(staticallyKnownTypes); + } + + @SuppressWarnings("unchecked") + public TypeInformation registerType(Class type) { + TypeInformation typeInfo = staticallyKnownTypes.getType(type); + if (typeInfo != null) { + return typeInfo; + } + return (TypeInformation) registeredTypes.computeIfAbsent(type, this::typeInformation); + } + + @SuppressWarnings("unchecked") + private TypeInformation typeInformation(Class valueType) { + if (Message.class.isAssignableFrom(valueType)) { + Class message = (Class) valueType; + return new ProtobufTypeInformation<>(message); + } + // TODO: we may want to restrict the allowed typeInfo here to theses that respect schema + // evaluation. + return TypeInformation.of(valueType); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/types/StaticallyRegisteredTypes.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/types/StaticallyRegisteredTypes.java new file mode 100644 index 00000000..c92b167c --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/types/StaticallyRegisteredTypes.java @@ -0,0 +1,70 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.types; + +import com.google.protobuf.Message; +import com.ververica.statefun.flink.core.message.MessageFactoryType; +import com.ververica.statefun.flink.core.message.MessageTypeInformation; +import com.ververica.statefun.flink.core.types.protobuf.ProtobufTypeInformation; +import java.util.HashMap; +import java.util.Map; +import javax.annotation.Nullable; +import javax.annotation.concurrent.NotThreadSafe; +import org.apache.flink.api.common.typeinfo.TypeInformation; + +/** + * StaticallyRegisteredTypes are types that were registered during the creation of the Stateful + * Functions universe. + */ +@NotThreadSafe +@SuppressWarnings("unchecked") +public final class StaticallyRegisteredTypes { + + private final Map, TypeInformation> registeredTypes = new HashMap<>(); + + public StaticallyRegisteredTypes(MessageFactoryType messageFactoryType) { + this.messageFactoryType = messageFactoryType; + } + + private final MessageFactoryType messageFactoryType; + + public TypeInformation registerType(Class type) { + return (TypeInformation) registeredTypes.computeIfAbsent(type, this::typeInformation); + } + + /** + * Retrieves the previously registered type. This is safe to access concurrently, after the + * translation phase is over. + */ + @Nullable + TypeInformation getType(Class valueType) { + return (TypeInformation) registeredTypes.get(valueType); + } + + private TypeInformation typeInformation(Class valueType) { + if (Message.class.isAssignableFrom(valueType)) { + Class message = (Class) valueType; + return new ProtobufTypeInformation<>(message); + } + if (com.ververica.statefun.flink.core.message.Message.class.isAssignableFrom(valueType)) { + return new MessageTypeInformation(messageFactoryType); + } + // TODO: we may want to restrict the allowed typeInfo here to theses that respect shcema + // evaluation. + return TypeInformation.of(valueType); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/types/protobuf/InputStreamView.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/types/protobuf/InputStreamView.java new file mode 100644 index 00000000..597dbfe2 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/types/protobuf/InputStreamView.java @@ -0,0 +1,81 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.types.protobuf; + +import java.io.IOException; +import java.io.InputStream; +import javax.annotation.Nonnull; +import javax.annotation.concurrent.NotThreadSafe; +import org.apache.flink.core.memory.DataInputView; + +@NotThreadSafe +final class InputStreamView extends InputStream { + private int limit; + private DataInputView source; + + void set(DataInputView source, int serializedSize) { + this.source = source; + this.limit = serializedSize; + } + + void done() { + this.source = null; + this.limit = 0; + } + + @Override + public int read() throws IOException { + if (limit <= 0) { + return -1; + } + --limit; + return (int) source.readByte(); + } + + @Override + public int read(@Nonnull byte[] b, final int off, int len) throws IOException { + if (limit <= 0) { + return -1; + } + len = Math.min(len, limit); + final int result = source.read(b, off, len); + if (result >= 0) { + limit -= result; + } + return result; + } + + @Override + public long skip(final long n) throws IOException { + final int min = (int) Math.min(n, limit); + final long result = source.skipBytes(min); + if (result >= 0) { + limit -= result; + } + return result; + } + + @Override + public synchronized void mark(int unused) { + throw new UnsupportedOperationException(); + } + + @Override + public synchronized void reset() { + throw new UnsupportedOperationException(); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/types/protobuf/OutputStreamView.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/types/protobuf/OutputStreamView.java new file mode 100644 index 00000000..a9ade32c --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/types/protobuf/OutputStreamView.java @@ -0,0 +1,57 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.types.protobuf; + +import java.io.IOException; +import java.io.OutputStream; +import javax.annotation.Nonnull; +import javax.annotation.concurrent.NotThreadSafe; +import org.apache.flink.core.memory.DataOutputView; + +@NotThreadSafe +final class OutputStreamView extends OutputStream { + private DataOutputView target; + + void set(DataOutputView target) { + this.target = target; + } + + void done() { + target = null; + } + + @Override + public void write(@Nonnull byte[] b) throws IOException { + target.write(b); + } + + @Override + public void write(@Nonnull byte[] b, int off, int len) throws IOException { + target.write(b, off, len); + } + + @Override + public void write(int b) throws IOException { + target.write(b); + } + + @Override + public void flush() {} + + @Override + public void close() {} +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/types/protobuf/ProtobufReflectionUtil.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/types/protobuf/ProtobufReflectionUtil.java new file mode 100644 index 00000000..435ecc77 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/types/protobuf/ProtobufReflectionUtil.java @@ -0,0 +1,90 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.types.protobuf; + +import com.google.protobuf.DescriptorProtos.FileDescriptorSet; +import com.google.protobuf.DescriptorProtos.FileDescriptorSet.Builder; +import com.google.protobuf.Descriptors.Descriptor; +import com.google.protobuf.Descriptors.FileDescriptor; +import com.google.protobuf.Message; +import com.google.protobuf.Parser; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.HashSet; +import java.util.Set; + +public final class ProtobufReflectionUtil { + private ProtobufReflectionUtil() {} + + @SuppressWarnings("unchecked") + public static Parser protobufParser(Class messageClass) { + Object parser = getParserFromGeneratedMessage(messageClass); + if (!(parser instanceof Parser)) { + throw new IllegalStateException( + "was expecting a protobuf parser to be return from the static parser() method on the type " + + messageClass + + " but instead got " + + parser); + } + return (Parser) parser; + } + + static FileDescriptorSet protoFileDescriptorSet(Descriptor descriptor) { + Set descriptors = new HashSet<>(); + descriptors.add(descriptor.getFile()); + addDependenciesRecursively(descriptors, descriptor.getFile()); + + Builder fileDescriptorSet = FileDescriptorSet.newBuilder(); + for (FileDescriptor d : descriptors) { + fileDescriptorSet.addFile(d.toProto()); + } + return fileDescriptorSet.build(); + } + + /** extract the {@linkplain Descriptor} for the generated message type. */ + static Descriptor protobufDescriptor(Class type) { + try { + Method getDescriptor = type.getDeclaredMethod("getDescriptor"); + return (Descriptor) getDescriptor.invoke(type); + } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) { + throw new IllegalStateException( + "unable to obtain protobuf type fileDescriptorSet for " + type, e); + } + } + + /** + * extracts the {@linkplain Parser} implementation for that type. see: + * https://developers.google.com/protocol-buffers/docs/reference/java-generated + */ + private static Object getParserFromGeneratedMessage(Class messageClass) { + try { + Method parserMethod = messageClass.getDeclaredMethod("parser"); + return parserMethod.invoke(parserMethod); + } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) { + throw new RuntimeException(e); + } + } + + private static void addDependenciesRecursively( + Set visited, FileDescriptor descriptor) { + for (FileDescriptor dependency : descriptor.getDependencies()) { + if (visited.add(dependency)) { + addDependenciesRecursively(visited, dependency.getFile()); + } + } + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/types/protobuf/ProtobufSerializer.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/types/protobuf/ProtobufSerializer.java new file mode 100644 index 00000000..ea74e06a --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/types/protobuf/ProtobufSerializer.java @@ -0,0 +1,103 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.types.protobuf; + +import com.google.protobuf.DescriptorProtos.FileDescriptorSet; +import com.google.protobuf.Descriptors.Descriptor; +import com.google.protobuf.Message; +import com.google.protobuf.Parser; +import com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot; +import java.io.IOException; +import java.util.Objects; +import javax.annotation.concurrent.NotThreadSafe; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +@NotThreadSafe +public final class ProtobufSerializer { + + private final OutputStreamView output; + private final InputStreamView input; + private final Parser parser; + private final ProtobufSerializerSnapshot snapshot; + + public static ProtobufSerializer forMessageGeneratedClass(Class type) { + Objects.requireNonNull(type); + Parser parser = ProtobufReflectionUtil.protobufParser(type); + ProtobufSerializerSnapshot snapshot = createSnapshot(type); + return new ProtobufSerializer<>(parser, snapshot); + } + + private ProtobufSerializer(Parser parser, ProtobufSerializerSnapshot snapshot) { + this.parser = Objects.requireNonNull(parser); + this.snapshot = Objects.requireNonNull(snapshot); + this.input = new InputStreamView(); + this.output = new OutputStreamView(); + } + + public void serialize(M record, DataOutputView target) throws IOException { + final int size = record.getSerializedSize(); + target.writeInt(size); + + output.set(target); + try { + record.writeTo(output); + } finally { + output.done(); + } + } + + public M deserialize(DataInputView source) throws IOException { + final int serializedSize = source.readInt(); + input.set(source, serializedSize); + try { + return parser.parseFrom(input); + } finally { + input.done(); + } + } + + public void copy(DataInputView source, DataOutputView target) throws IOException { + int serializedSize = source.readInt(); + target.writeInt(serializedSize); + target.write(source, serializedSize); + } + + ProtobufSerializerSnapshot snapshot() { + return snapshot; + } + + public ProtobufSerializer duplicate() { + return new ProtobufSerializer<>(parser, snapshot); + } + + // --------------------------------------------------------------------------------------------------------------- + // Internal helpers + // --------------------------------------------------------------------------------------------------------------- + + private static ProtobufSerializerSnapshot createSnapshot(Class type) { + Descriptor messageDescriptor = ProtobufReflectionUtil.protobufDescriptor(type); + FileDescriptorSet dependencies = + ProtobufReflectionUtil.protoFileDescriptorSet(messageDescriptor); + + return ProtobufSerializerSnapshot.newBuilder() + .setMessageName(messageDescriptor.getFullName()) + .setGeneratedJavaName(type.getName()) + .setDescriptorSet(dependencies) + .build(); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/types/protobuf/ProtobufTypeInformation.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/types/protobuf/ProtobufTypeInformation.java new file mode 100644 index 00000000..474db1b7 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/types/protobuf/ProtobufTypeInformation.java @@ -0,0 +1,96 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.types.protobuf; + +import com.google.protobuf.Message; +import java.util.Objects; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; + +public class ProtobufTypeInformation extends TypeInformation { + + private static final long serialVersionUID = 1L; + + private final Class messageTypeClass; + + public ProtobufTypeInformation(Class messageTypeClass) { + this.messageTypeClass = Objects.requireNonNull(messageTypeClass); + } + + @Override + public boolean isBasicType() { + return false; + } + + @Override + public boolean isTupleType() { + return false; + } + + @Override + public int getArity() { + return 0; + } + + @Override + public int getTotalFields() { + return 0; + } + + @Override + public Class getTypeClass() { + return messageTypeClass; + } + + @Override + public boolean isKeyType() { + return false; + } + + @Override + public TypeSerializer createSerializer(ExecutionConfig config) { + return new ProtobufTypeSerializer<>(messageTypeClass); + } + + @Override + public String toString() { + return "ProtobufTypeInformation(" + messageTypeClass + ")"; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ProtobufTypeInformation that = (ProtobufTypeInformation) o; + return messageTypeClass.equals(that.messageTypeClass); + } + + @Override + public int hashCode() { + return Objects.hashCode(messageTypeClass); + } + + @Override + public boolean canEqual(Object obj) { + return obj instanceof ProtobufTypeInformation; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/types/protobuf/ProtobufTypeSerializer.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/types/protobuf/ProtobufTypeSerializer.java new file mode 100644 index 00000000..5a6c0911 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/types/protobuf/ProtobufTypeSerializer.java @@ -0,0 +1,135 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.types.protobuf; + +import com.google.protobuf.Message; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.util.Objects; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +public final class ProtobufTypeSerializer extends TypeSerializer { + + private static final long serialVersionUID = 1; + + private final Class typeClass; + private transient ProtobufSerializer underlyingSerializer; + + /** this is a lazy computed snapshot */ + @SuppressWarnings("InstanceVariableMayNotBeInitializedByReadObject") + private transient ProtobufTypeSerializerSnapshot snapshot; + + // -------------------------------------------------------------------------------------------------- + // Constructors + // -------------------------------------------------------------------------------------------------- + + ProtobufTypeSerializer(Class typeClass) { + this(typeClass, ProtobufSerializer.forMessageGeneratedClass(typeClass)); + } + + private ProtobufTypeSerializer(Class typeClass, ProtobufSerializer protobufSerializer) { + this.typeClass = Objects.requireNonNull(typeClass); + this.underlyingSerializer = Objects.requireNonNull(protobufSerializer); + } + + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + this.underlyingSerializer = ProtobufSerializer.forMessageGeneratedClass(typeClass); + } + + @Override + public TypeSerializer duplicate() { + return new ProtobufTypeSerializer<>(typeClass, underlyingSerializer.duplicate()); + } + + @Override + public boolean isImmutableType() { + return true; + } + + @Override + public M createInstance() { + return null; + } + + @SuppressWarnings("unchecked") + @Override + public M copy(M from) { + return (M) from.toBuilder().build(); + } + + @Override + public M copy(M from, M reuse) { + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(M record, DataOutputView target) throws IOException { + underlyingSerializer.serialize(record, target); + } + + @Override + public M deserialize(DataInputView source) throws IOException { + return underlyingSerializer.deserialize(source); + } + + @Override + public M deserialize(M reuse, DataInputView source) throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + underlyingSerializer.copy(source, target); + } + + @Override + public boolean equals(Object obj) { + if (obj == null) { + return false; + } + Class aClass = obj.getClass(); + return getClass().equals(aClass); + } + + @Override + public int hashCode() { + return getClass().hashCode(); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + ProtobufTypeSerializerSnapshot snapshot = this.snapshot; + if (snapshot == null) { + snapshot = new ProtobufTypeSerializerSnapshot<>(typeClass, underlyingSerializer.snapshot()); + this.snapshot = snapshot; + } + return snapshot; + } + + Class getTypeClass() { + return typeClass; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/types/protobuf/ProtobufTypeSerializerSnapshot.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/types/protobuf/ProtobufTypeSerializerSnapshot.java new file mode 100644 index 00000000..e524631f --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/flink/core/types/protobuf/ProtobufTypeSerializerSnapshot.java @@ -0,0 +1,123 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.types.protobuf; + +import com.google.protobuf.Message; +import com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot; +import java.io.IOException; +import java.util.Objects; +import javax.annotation.Nullable; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +public final class ProtobufTypeSerializerSnapshot + implements TypeSerializerSnapshot { + + private static final int VERSION = 1; + + @Nullable private ProtobufSerializerSnapshot snapshotData; + + @Nullable private Class typeClass; + + @SuppressWarnings("unused") + public ProtobufTypeSerializerSnapshot() { + // used for reflective instantiation. + } + + ProtobufTypeSerializerSnapshot(Class messageType, ProtobufSerializerSnapshot snapshotData) { + this.typeClass = Objects.requireNonNull(messageType); + this.snapshotData = Objects.requireNonNull(snapshotData); + } + + @SuppressWarnings("unchecked") + private static Class classForName( + ClassLoader userCodeClassLoader, ProtobufSerializerSnapshot snapshotData) { + try { + return (Class) + Class.forName(snapshotData.getGeneratedJavaName(), false, userCodeClassLoader); + } catch (ClassNotFoundException e) { + throw new IllegalStateException( + "Unable to restore the protobuf serializer since the generated java class is not found. " + + "previously the generated java class was at " + + snapshotData.getGeneratedJavaName() + + " with " + + snapshotData.getMessageName(), + e); + } + } + + @Override + public int getCurrentVersion() { + return VERSION; + } + + @Override + public void writeSnapshot(DataOutputView out) throws IOException { + if (!(snapshotData != null)) { + throw new IllegalStateException(); + } + out.writeInt(snapshotData.getSerializedSize()); + out.write(snapshotData.toByteArray()); + } + + @Override + public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) + throws IOException { + final int snapshotSize = in.readInt(); + final byte[] snapshotBytes = new byte[snapshotSize]; + in.readFully(snapshotBytes); + + this.snapshotData = ProtobufSerializerSnapshot.parseFrom(snapshotBytes); + this.typeClass = classForName(userCodeClassLoader, snapshotData); + } + + @Override + public TypeSerializer restoreSerializer() { + Objects.requireNonNull(typeClass); + return new ProtobufTypeSerializer<>(typeClass); + } + + @Override + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializer newSerializer) { + if (!(newSerializer instanceof ProtobufTypeSerializer)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + ProtobufTypeSerializer casted = (ProtobufTypeSerializer) newSerializer; + return resolveSchemaCompatibility(casted); + } + + /** + * Check schema compatibility with the new serializer. + * + *

This check is very simplistic, that just compares the two typeClasses, but the {@link + * ProtobufSerializerSnapshot} has much more information to be used for compatibility resolution. + * We make sure to store this information first, and implement a more robust schema resolution + * logic in the future. + */ + private TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + ProtobufTypeSerializer newSerializer) { + Class otherTypeClass = newSerializer.getTypeClass(); + if (otherTypeClass == typeClass) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } + return TypeSerializerSchemaCompatibility.incompatible(); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/sdk/state/ApiExtension.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/sdk/state/ApiExtension.java new file mode 100644 index 00000000..0f197d80 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/com/ververica/statefun/sdk/state/ApiExtension.java @@ -0,0 +1,24 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.sdk.state; + +public class ApiExtension { + public static void setPersistedValueAccessor( + PersistedValue persistedValue, Accessor accessor) { + persistedValue.setAccessor(accessor); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/it/unimi/dsi/fastutil/HashCommon.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/it/unimi/dsi/fastutil/HashCommon.java new file mode 100644 index 00000000..33f5249e --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/it/unimi/dsi/fastutil/HashCommon.java @@ -0,0 +1,255 @@ +package it.unimi.dsi.fastutil; + +/* + * Copyright (C) 2002-2017 Sebastiano Vigna + * + * 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. + */ + +/** Common code for all hash-based classes. */ +public class HashCommon { + + /** 232 · φ, φ = (√5 − 1)/2. */ + private static final int INT_PHI = 0x9E3779B9; + /** The reciprocal of {@link #INT_PHI} modulo 232. */ + private static final int INV_INT_PHI = 0x144cbc89; + /** 264 · φ, φ = (√5 − 1)/2. */ + private static final long LONG_PHI = 0x9E3779B97F4A7C15L; + /** The reciprocal of {@link #LONG_PHI} modulo 264. */ + private static final long INV_LONG_PHI = 0xf1de83e19937733dL; + + protected HashCommon() {} + + /** + * Avalanches the bits of an integer by applying the finalisation step of MurmurHash3. + * + *

This method implements the finalisation step of Austin Appleby's MurmurHash3. Its purpose is to avalanche the bits + * of the argument to within 0.25% bias. + * + * @param x an integer. + * @return a hash value with good avalanching properties. + */ + public static int murmurHash3(int x) { + x ^= x >>> 16; + x *= 0x85ebca6b; + x ^= x >>> 13; + x *= 0xc2b2ae35; + x ^= x >>> 16; + return x; + } + + /** + * Avalanches the bits of a long integer by applying the finalisation step of MurmurHash3. + * + *

This method implements the finalisation step of Austin Appleby's MurmurHash3. Its purpose is to avalanche the bits + * of the argument to within 0.25% bias. + * + * @param x a long integer. + * @return a hash value with good avalanching properties. + */ + public static long murmurHash3(long x) { + x ^= x >>> 33; + x *= 0xff51afd7ed558ccdL; + x ^= x >>> 33; + x *= 0xc4ceb9fe1a85ec53L; + x ^= x >>> 33; + return x; + } + + /** + * Quickly mixes the bits of an integer. + * + *

This method mixes the bits of the argument by multiplying by the golden ratio and + * xorshifting the result. It is borrowed from Koloboke, and it has slightly worse behaviour + * than {@link #murmurHash3(int)} (in open-addressing hash tables the average number of probes is + * slightly larger), but it's much faster. + * + * @param x an integer. + * @return a hash value obtained by mixing the bits of {@code x}. + * @see #invMix(int) + */ + public static int mix(final int x) { + final int h = x * INT_PHI; + return h ^ (h >>> 16); + } + + /** + * The inverse of {@link #mix(int)}. This method is mainly useful to create unit tests. + * + * @param x an integer. + * @return a value that passed through {@link #mix(int)} would give {@code x}. + */ + public static int invMix(final int x) { + return (x ^ x >>> 16) * INV_INT_PHI; + } + + /** + * Quickly mixes the bits of a long integer. + * + *

This method mixes the bits of the argument by multiplying by the golden ratio and + * xorshifting twice the result. It is borrowed from Koloboke, and it has slightly worse behaviour + * than {@link #murmurHash3(long)} (in open-addressing hash tables the average number of probes is + * slightly larger), but it's much faster. + * + * @param x a long integer. + * @return a hash value obtained by mixing the bits of {@code x}. + */ + public static long mix(final long x) { + long h = x * LONG_PHI; + h ^= h >>> 32; + return h ^ (h >>> 16); + } + + /** + * The inverse of {@link #mix(long)}. This method is mainly useful to create unit tests. + * + * @param x a long integer. + * @return a value that passed through {@link #mix(long)} would give {@code x}. + */ + public static long invMix(long x) { + x ^= x >>> 32; + x ^= x >>> 16; + return (x ^ x >>> 32) * INV_LONG_PHI; + } + + /** + * Returns the hash code that would be returned by {@link Float#hashCode()}. + * + * @param f a float. + * @return the same code as {@link Float#hashCode() new Float(f).hashCode()}. + */ + public static int float2int(final float f) { + return Float.floatToRawIntBits(f); + } + + /** + * Returns the hash code that would be returned by {@link Double#hashCode()}. + * + * @param d a double. + * @return the same code as {@link Double#hashCode() new Double(f).hashCode()}. + */ + public static int double2int(final double d) { + final long l = Double.doubleToRawLongBits(d); + return (int) (l ^ (l >>> 32)); + } + + /** + * Returns the hash code that would be returned by {@link Long#hashCode()}. + * + * @param l a long. + * @return the same code as {@link Long#hashCode() new Long(f).hashCode()}. + */ + public static int long2int(final long l) { + return (int) (l ^ (l >>> 32)); + } + + /** + * Returns the least power of two greater than or equal to the specified value. + * + *

Note that this function will return 1 when the argument is 0. + * + * @param x an integer smaller than or equal to 230. + * @return the least power of two greater than or equal to the specified value. + */ + public static int nextPowerOfTwo(int x) { + if (x == 0) { + return 1; + } + x--; + x |= x >> 1; + x |= x >> 2; + x |= x >> 4; + x |= x >> 8; + return (x | x >> 16) + 1; + } + + /** + * Returns the least power of two greater than or equal to the specified value. + * + *

Note that this function will return 1 when the argument is 0. + * + * @param x a long integer smaller than or equal to 262. + * @return the least power of two greater than or equal to the specified value. + */ + public static long nextPowerOfTwo(long x) { + if (x == 0) { + return 1; + } + x--; + x |= x >> 1; + x |= x >> 2; + x |= x >> 4; + x |= x >> 8; + x |= x >> 16; + return (x | x >> 32) + 1; + } + + /** + * Returns the maximum number of entries that can be filled before rehashing. + * + * @param n the size of the backing array. + * @param f the load factor. + * @return the maximum number of entries before rehashing. + */ + public static int maxFill(final int n, final float f) { + /* We must guarantee that there is always at least + * one free entry (even with pathological load ffunctions). */ + return Math.min((int) Math.ceil(n * f), n - 1); + } + + /** + * Returns the maximum number of entries that can be filled before rehashing. + * + * @param n the size of the backing array. + * @param f the load factor. + * @return the maximum number of entries before rehashing. + */ + public static long maxFill(final long n, final float f) { + /* We must guarantee that there is always at least + * one free entry (even with pathological load ffunctions). */ + return Math.min((long) Math.ceil(n * f), n - 1); + } + + /** + * Returns the least power of two smaller than or equal to 230 and larger than or equal + * to {@code Math.ceil(expected / f)}. + * + * @param expected the expected number of elements in a hash table. + * @param f the load factor. + * @return the minimum possible size for a backing array. + * @throws IllegalArgumentException if the necessary size is larger than 230. + */ + public static int arraySize(final int expected, final float f) { + final long s = Math.max(2, nextPowerOfTwo((long) Math.ceil(expected / f))); + if (s > (1 << 30)) { + throw new IllegalArgumentException( + "Too large (" + expected + " expected elements with load factor " + f + ")"); + } + return (int) s; + } + + /** + * Returns the least power of two larger than or equal to {@code Math.ceil(expected / f)}. + * + * @param expected the expected number of elements in a hash table. + * @param f the load factor. + * @return the minimum possible size for a backing big array. + */ + public static long bigArraySize(final long expected, final float f) { + return nextPowerOfTwo((long) Math.ceil(expected / f)); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/java/it/unimi/dsi/fastutil/objects/ObjectOpenHashMap.java b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/it/unimi/dsi/fastutil/objects/ObjectOpenHashMap.java new file mode 100644 index 00000000..1a1bea49 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/java/it/unimi/dsi/fastutil/objects/ObjectOpenHashMap.java @@ -0,0 +1,386 @@ +/* + * Copyright (C) 2002-2017 Sebastiano Vigna + * + * 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. + */ + +package it.unimi.dsi.fastutil.objects; + +import static it.unimi.dsi.fastutil.HashCommon.arraySize; +import static it.unimi.dsi.fastutil.HashCommon.maxFill; + +import it.unimi.dsi.fastutil.HashCommon; +import java.util.Arrays; + +/** + * A type-specific hash map with a fast, small-footprint implementation. + * + *

Instances of this class use a hash table to represent a map. The table is filled up to a + * specified load factor, and then doubled in size to accommodate new entries. If the table + * is emptied below one fourth of the load factor, it is halved in size; however, the table + * is never reduced to a size smaller than that at creation time: this approach makes it possible to + * create maps with a large capacity in which insertions and deletions do not cause immediately + * rehashing. Moreover, halving is not performed when deleting entries from an iterator, as it would + * interfere with the iteration process. + * + * @see HashCommon + */ +public final class ObjectOpenHashMap { + + /** The initial default size of a hash table. */ + private static final int DEFAULT_INITIAL_SIZE = 16; + /** The default load factor of a hash table. */ + private static final float DEFAULT_LOAD_FACTOR = .75f; + /** We never resize below this threshold, which is the construction-time {#n}. */ + private final transient int minN; + /** The acceptable load factor. */ + private final float f; + /** The array of keys. */ + private transient K[] key; + /** The array of values. */ + private transient V[] value; + /** The mask for wrapping a position counter. */ + private transient int mask; + /** Whether this map contains the key zero. */ + private transient boolean containsNullKey; + /** The current table size. */ + private transient int n; + /** Threshold after which we rehash. It must be the table size times {@link #f}. */ + private transient int maxFill; + /** Number of entries in the set (including the key zero, if present). */ + private int size; + + /** + * Creates a new hash map. + * + *

The actual table size will be the least power of two greater than {@code expected}/{@code + * f}. + * + * @param expected the expected number of elements in the hash map. + * @param f the load factor. + */ + @SuppressWarnings({"unchecked", "WeakerAccess"}) + public ObjectOpenHashMap(final int expected, final float f) { + if (f <= 0 || f > 1) { + throw new IllegalArgumentException( + "Load factor must be greater than 0 and smaller than or equal to 1"); + } + if (expected < 0) { + throw new IllegalArgumentException("The expected number of elements must be nonnegative"); + } + this.f = f; + minN = n = arraySize(expected, f); + mask = n - 1; + maxFill = maxFill(n, f); + key = (K[]) new Object[n + 1]; + value = (V[]) new Object[n + 1]; + } + + /** + * Creates a new hash map with {@link #DEFAULT_LOAD_FACTOR} as load factor. + * + * @param expected the expected number of elements in the hash map. + */ + @SuppressWarnings({"WeakerAccess", "unused"}) + public ObjectOpenHashMap(final int expected) { + this(expected, DEFAULT_LOAD_FACTOR); + } + + /** + * Creates a new hash map with initial expected {@link #DEFAULT_INITIAL_SIZE} entries and {@link + * #DEFAULT_LOAD_FACTOR} as load factor. + */ + @SuppressWarnings({"WeakerAccess", "unused"}) + public ObjectOpenHashMap() { + this(DEFAULT_INITIAL_SIZE, DEFAULT_LOAD_FACTOR); + } + + @SuppressWarnings({"unused"}) + public V put(final K k, final V v) { + final int pos = find(k); + if (pos < 0) { + insert(-pos - 1, k, v); + return null; + } + final V oldValue = value[pos]; + value[pos] = v; + return oldValue; + } + + @SuppressWarnings({"unchecked", "unused"}) + public V get(final Object k) { + if (k == null) { + return containsNullKey ? value[n] : null; + } + K curr; + final K[] key = this.key; + int pos; + // The starting point. + if (((curr = key[pos = (it.unimi.dsi.fastutil.HashCommon.mix((k).hashCode())) & mask]) + == null)) { + return null; + } + if (((k).equals(curr))) { + return value[pos]; + } + // There's always an unused entry. + while (true) { + if (((curr = key[pos = (pos + 1) & mask]) == null)) { + return null; + } + if (((k).equals(curr))) { + return value[pos]; + } + } + } + + @SuppressWarnings({"unchecked", "unused"}) + public boolean containsKey(final Object k) { + if (k == null) { + return containsNullKey; + } + K curr; + final K[] key = this.key; + int pos; + // The starting point. + if (((curr = key[pos = (it.unimi.dsi.fastutil.HashCommon.mix((k).hashCode())) & mask]) + == null)) { + return false; + } + if (((k).equals(curr))) { + return true; + } + // There's always an unused entry. + while (true) { + if (((curr = key[pos = (pos + 1) & mask]) == null)) { + return false; + } + if (((k).equals(curr))) { + return true; + } + } + } + + @SuppressWarnings({"unchecked", "unused"}) + public V remove(final Object k) { + if (k == null) { + if (containsNullKey) { + return removeNullEntry(); + } + return null; + } + K curr; + final K[] key = this.key; + int pos; + // The starting point. + if (((curr = key[pos = (it.unimi.dsi.fastutil.HashCommon.mix((k).hashCode())) & mask]) + == null)) { + return null; + } + if (((k).equals(curr))) { + return removeEntry(pos); + } + while (true) { + if (((curr = key[pos = (pos + 1) & mask]) == null)) { + return null; + } + if (((k).equals(curr))) { + return removeEntry(pos); + } + } + } + + @SuppressWarnings({"unused"}) + public void clear() { + if (size == 0) { + return; + } + size = 0; + containsNullKey = false; + Arrays.fill(key, (null)); + Arrays.fill(value, null); + } + + @SuppressWarnings({"unused"}) + public int size() { + return size; + } + + @SuppressWarnings({"unused"}) + public boolean isEmpty() { + return size == 0; + } + + // ------------------------------------------------------------------------------------------------------------- + + private int realSize() { + return containsNullKey ? size - 1 : size; + } + + private V removeEntry(final int pos) { + final V oldValue = value[pos]; + value[pos] = null; + size--; + shiftKeys(pos); + if (n > minN && size < maxFill / 4 && n > DEFAULT_INITIAL_SIZE) { + rehash(n / 2); + } + return oldValue; + } + + private V removeNullEntry() { + containsNullKey = false; + key[n] = null; + final V oldValue = value[n]; + value[n] = null; + size--; + if (n > minN && size < maxFill / 4 && n > DEFAULT_INITIAL_SIZE) { + rehash(n / 2); + } + return oldValue; + } + + @SuppressWarnings("unchecked") + private int find(final K k) { + if (((k) == null)) { + return containsNullKey ? n : -(n + 1); + } + K curr; + final K[] key = this.key; + int pos; + // The starting point. + if (((curr = key[pos = (it.unimi.dsi.fastutil.HashCommon.mix((k).hashCode())) & mask]) + == null)) { + return -(pos + 1); + } + if (((k).equals(curr))) { + return pos; + } + // There's always an unused entry. + while (true) { + if (((curr = key[pos = (pos + 1) & mask]) == null)) { + return -(pos + 1); + } + if (((k).equals(curr))) { + return pos; + } + } + } + + private void insert(final int pos, final K k, final V v) { + if (pos == n) { + containsNullKey = true; + } + key[pos] = k; + value[pos] = v; + if (size++ >= maxFill) { + rehash(arraySize(size + 1, f)); + } + } + + /** + * Shifts left entries with the specified hash code, starting at the specified position, and + * empties the resulting free entry. + * + * @param pos a starting position. + */ + private void shiftKeys(int pos) { + // Shift entries with the same hash. + int last, slot; + K curr; + final K[] key = this.key; + for (; ; ) { + pos = ((last = pos) + 1) & mask; + for (; ; ) { + if (((curr = key[pos]) == null)) { + key[last] = (null); + value[last] = null; + return; + } + slot = (it.unimi.dsi.fastutil.HashCommon.mix((curr).hashCode())) & mask; + if (last <= pos ? last >= slot || slot > pos : last >= slot && slot > pos) { + break; + } + pos = (pos + 1) & mask; + } + key[last] = curr; + value[last] = value[pos]; + } + } + + /** + * Rehashes the map. + * + *

This method implements the basic rehashing strategy, and may be overridden by subclasses + * implementing different rehashing strategies (e.g., disk-based rehashing). However, you should + * not override this method unless you understand the internal workings of this class. + * + * @param newN the new size + */ + @SuppressWarnings({"unchecked", "StatementWithEmptyBody"}) + private void rehash(final int newN) { + final K key[] = this.key; + final V value[] = this.value; + final int mask = newN - 1; // Note that this is used by the hashing macro + final K newKey[] = (K[]) new Object[newN + 1]; + final V newValue[] = (V[]) new Object[newN + 1]; + int i = n, pos; + for (int j = realSize(); j-- != 0; ) { + while (((key[--i]) == null)) {} + if (!((newKey[pos = (it.unimi.dsi.fastutil.HashCommon.mix((key[i]).hashCode())) & mask]) + == null)) { + while (!((newKey[pos = (pos + 1) & mask]) == null)) {} + } + newKey[pos] = key[i]; + newValue[pos] = value[i]; + } + newValue[newN] = value[n]; + n = newN; + this.mask = mask; + maxFill = maxFill(n, f); + this.key = newKey; + this.value = newValue; + } + + /** + * Returns a hash code for this map. + * + *

This method overrides the generic method provided by the superclass. Since {@code equals()} + * is not overriden, it is important that the value returned by this method is the same value as + * the one returned by the overriden method. + * + * @return a hash code for this map. + */ + @Override + public int hashCode() { + int h = 0; + for (int j = realSize(), i = 0, t = 0; j-- != 0; ) { + while (((key[i]) == null)) { + i++; + } + if (this != key[i]) { + t = ((key[i]).hashCode()); + } + if (this != value[i]) { + t ^= ((value[i]) == null ? 0 : (value[i]).hashCode()); + } + h += t; + i++; + } + // Zero / null keys have hash zero. + if (containsNullKey) { + h += ((value[n]) == null ? 0 : (value[n]).hashCode()); + } + return h; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/main/protobuf/stateful-functions.proto b/stateful-functions-flink/stateful-functions-flink-core/src/main/protobuf/stateful-functions.proto new file mode 100644 index 00000000..e2c1d1c6 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/main/protobuf/stateful-functions.proto @@ -0,0 +1,55 @@ +// +// Copyright 2019 Ververica GmbH. +// +// 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"; + +package com.ververica.statefun.flink.core; +option java_package = "com.ververica.statefun.flink.core.generated"; +option java_multiple_files = true; + +import "google/protobuf/descriptor.proto"; + +message ProtobufSerializerSnapshot { + string generated_java_name = 1; + string message_name = 2; + + google.protobuf.FileDescriptorSet descriptor_set = 3; +} + +message EnvelopeAddress { + string namespace = 1; + string type = 2; + string id = 3; +} + +message Payload { + string class_name = 2; + bytes payload_bytes = 3; +} + +message Checkpoint { + int64 checkpoint_id = 1; +} + +message Envelope { + EnvelopeAddress source = 1; + EnvelopeAddress target = 2; + + oneof body { + Checkpoint checkpoint = 4; + Payload payload = 3; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/TestUtils.java b/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/TestUtils.java new file mode 100644 index 00000000..68d36ab1 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/TestUtils.java @@ -0,0 +1,61 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core; + +import com.ververica.statefun.flink.core.generated.Envelope; +import com.ververica.statefun.flink.core.generated.EnvelopeAddress; +import com.ververica.statefun.flink.core.generated.Payload; +import com.ververica.statefun.flink.core.message.MessageFactory; +import com.ververica.statefun.flink.core.message.MessageFactoryType; +import com.ververica.statefun.sdk.Address; +import com.ververica.statefun.sdk.FunctionType; + +@SuppressWarnings("WeakerAccess") +public class TestUtils { + + public static final MessageFactory ENVELOPE_FACTORY = + MessageFactory.forType(MessageFactoryType.WITH_KRYO_PAYLOADS); + + public static final FunctionType FUNCTION_TYPE = new FunctionType("test", "a"); + public static final Address FUNCTION_1_ADDR = new Address(FUNCTION_TYPE, "a-1"); + public static final Address FUNCTION_2_ADDR = new Address(FUNCTION_TYPE, "a-2"); + public static final EnvelopeAddress DUMMY_PAYLOAD = EnvelopeAddress.getDefaultInstance(); + + public static final EnvelopeAddress ADDRESS = + EnvelopeAddress.newBuilder().setNamespace("namespace").setType("type").setId("key-1").build(); + + public static Address integerAddress(int i) { + return new Address(new FunctionType("foo", "bar"), "bar-" + i); + } + + public static Envelope[] envelopesOfVariousSizes() { + Envelope[] envelopes = new Envelope[10]; + for (int i = 0; i < envelopes.length; i++) { + envelopes[i] = + Envelope.newBuilder() + .setSource(ADDRESS) + .setTarget(ADDRESS) + .setPayload( + Payload.newBuilder() + .setClassName(EnvelopeAddress.class.getName()) + .setPayloadBytes(ADDRESS.toByteString()) + .build()) + .build(); + } + return envelopes; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/feedback/FeedbackChannelTest.java b/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/feedback/FeedbackChannelTest.java new file mode 100644 index 00000000..21b4c807 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/feedback/FeedbackChannelTest.java @@ -0,0 +1,124 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.feedback; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; + +import java.util.ArrayList; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import org.junit.Ignore; +import org.junit.Test; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.infra.Blackhole; +import org.openjdk.jmh.runner.Runner; +import org.openjdk.jmh.runner.options.Options; +import org.openjdk.jmh.runner.options.OptionsBuilder; + +@SuppressWarnings({ + "SameParameterValue", + "IOResourceOpenedButNotSafelyClosed", + "resource", + "IOResourceOpenedButNotSafelyClosed", + "unused" +}) +public class FeedbackChannelTest { + private static final SubtaskFeedbackKey KEY = + new FeedbackKey("foo", 1).withSubTaskIndex(0); + + @Test + public void exampleUsage() { + FeedbackChannel channel = + new FeedbackChannel<>(KEY, new LockFreeBatchFeedbackQueue<>()); + channel.put("hello"); + channel.put("world"); + channel.close(); + + ArrayList results = new ArrayList<>(); + + // the consumer draining would execute on this thread,to avoid test race condition. + channel.registerConsumer(results::add, new Object(), Runnable::run); + + assertThat(results, contains("hello", "world")); + } + + @Ignore("benchmarks are not run as part of a regular test suite.") + @Test + public void launchBenchmark() throws Exception { + // The following is the result of comparing 3 different queue implementations for a spsc + // this is the result of concurrently producing and consuming 1 million items. + // The other implementations are not checkpointed into the code base to avoid accidental + // megamorphic call sites. + // + // Benchmark Mode Cnt Score Error Units + // HandOffFeedbackChannelTest.lockBasedHandOffQueue avgt 4 55.284 ± 43.094 ms/op + // HandOffFeedbackChannelTest.lockFreeBatchHandOffQueue avgt 4 17.411 ± 3.968 ms/op + // HandOffFeedbackChannelTest.lockFreeStackBasedQueue avgt 4 42.569 ± 8.616 ms/op + // + Options opt = + new OptionsBuilder() + .include(this.getClass().getName() + ".*") + .timeUnit(TimeUnit.MILLISECONDS) + .warmupIterations(4) + .measurementIterations(4) + .threads(2) + .forks(1) + .shouldFailOnError(true) + .shouldDoGC(true) + .build(); + + new Runner(opt).run(); + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + public void lockFreeBatchHandOffQueue(Blackhole blackhole) { + int bh = benchmark(new LockFreeBatchFeedbackQueue<>(), 1_000_000); + blackhole.consume(bh); + } + + private static int benchmark(FeedbackQueue queue, int items) { + FeedbackChannel channel = new FeedbackChannel<>(KEY, queue); + // + // consumer + // + int[] consumed = new int[1]; + Object lock = new Object(); + ExecutorService executor = + Executors.newSingleThreadExecutor( + runnable -> { + Thread t = new Thread(runnable); + t.setDaemon(true); + return t; + }); + + channel.registerConsumer(unused -> consumed[0]++, lock, executor); + // + // producer + // + for (int i = 0; i < items; i++) { + channel.put("hello"); + } + channel.close(); + + return consumed[0]; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/functions/LocalStatefulFunctionGroupTest.java b/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/functions/LocalStatefulFunctionGroupTest.java new file mode 100644 index 00000000..9363c4d2 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/functions/LocalStatefulFunctionGroupTest.java @@ -0,0 +1,150 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.functions; + +import static com.ververica.statefun.flink.core.TestUtils.ENVELOPE_FACTORY; +import static org.hamcrest.CoreMatchers.is; +import static org.junit.Assert.assertThat; + +import com.ververica.statefun.flink.core.generated.EnvelopeAddress; +import com.ververica.statefun.flink.core.message.Message; +import com.ververica.statefun.flink.core.metrics.FunctionTypeMetrics; +import com.ververica.statefun.flink.core.state.BoundState; +import com.ververica.statefun.sdk.Address; +import com.ververica.statefun.sdk.Context; +import com.ververica.statefun.sdk.FunctionType; +import com.ververica.statefun.sdk.io.EgressIdentifier; +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import org.junit.Test; + +public class LocalStatefulFunctionGroupTest { + // test constants + private static final FunctionType FUNCTION_TYPE = new FunctionType("test", "a"); + private static final Address FUNCTION_1_ADDR = new Address(FUNCTION_TYPE, "a-1"); + private static final Address FUNCTION_2_ADDR = new Address(FUNCTION_TYPE, "a-2"); + private static final EnvelopeAddress DUMMY_PAYLOAD = EnvelopeAddress.getDefaultInstance(); + + // test collaborators + private final FakeContext context = new FakeContext(); + private final FakeFunction function = new FakeFunction(); + private final FakeFunctionRepository fakeRepository = new FakeFunctionRepository(function); + + // object under test + private final LocalFunctionGroup functionGroupUnderTest = + new LocalFunctionGroup(fakeRepository, context); + + @Test + public void sanity() { + boolean processed = functionGroupUnderTest.processNextEnvelope(); + + assertThat(processed, is(false)); + } + + @Test + public void addingMessageWouldBeProcessedLater() { + Message message = ENVELOPE_FACTORY.from(FUNCTION_1_ADDR, FUNCTION_2_ADDR, DUMMY_PAYLOAD); + + functionGroupUnderTest.enqueue(message); + + assertThat(functionGroupUnderTest.processNextEnvelope(), is(true)); + assertThat(functionGroupUnderTest.processNextEnvelope(), is(false)); + } + + @Test + public void aFunctionWouldReceiveAMessageAddressedToIt() { + Message message = ENVELOPE_FACTORY.from(FUNCTION_1_ADDR, FUNCTION_2_ADDR, DUMMY_PAYLOAD); + + functionGroupUnderTest.enqueue(message); + functionGroupUnderTest.processNextEnvelope(); + + Message m = function.receivedMessages.get(0); + + assertThat(m.target(), is(message.target())); + } + + // --------------------------------------------------------------------------- + // test helpers + // --------------------------------------------------------------------------- + + static final class FakeFunction implements LiveFunction { + List receivedMessages = new ArrayList<>(); + + @Override + public void receive(Context context, Message message) { + receivedMessages.add(message); + } + + @Override + public FunctionTypeMetrics metrics() { + throw new UnsupportedOperationException(); + } + + @Override + public Optional state() { + return Optional.empty(); + } + } + + static final class FakeFunctionRepository implements FunctionRepository { + private LiveFunction function; + + FakeFunctionRepository(FakeFunction function) { + this.function = function; + } + + @Override + public LiveFunction get(FunctionType type) { + return function; + } + } + + static final class FakeContext implements ApplyingContext { + Message in; + + @Override + public Address self() { + return in.target(); + } + + @Override + public Address caller() { + return in.source(); + } + + @Override + public void send(Address to, Object message) {} + + @Override + public void send(EgressIdentifier egress, T what) {} + + @Override + public void sendAfter(Duration duration, Address to, Object message) {} + + @Override + public void registerAsyncOperation(M metadata, CompletableFuture future) {} + + @Override + public void apply(LiveFunction function, Message inMessage) { + in = inMessage; + function.receive(this, inMessage); + } + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/functions/ReductionsTest.java b/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/functions/ReductionsTest.java new file mode 100644 index 00000000..6d04607a --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/functions/ReductionsTest.java @@ -0,0 +1,653 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.functions; + +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.junit.Assert.assertThat; + +import com.ververica.statefun.flink.core.StatefulFunctionsUniverse; +import com.ververica.statefun.flink.core.TestUtils; +import com.ververica.statefun.flink.core.message.Message; +import com.ververica.statefun.flink.core.message.MessageFactoryType; +import java.io.Serializable; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.stream.Stream; +import javax.annotation.Nonnull; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.api.common.accumulators.DoubleCounter; +import org.apache.flink.api.common.accumulators.Histogram; +import org.apache.flink.api.common.accumulators.IntCounter; +import org.apache.flink.api.common.accumulators.LongCounter; +import org.apache.flink.api.common.cache.DistributedCache; +import org.apache.flink.api.common.functions.BroadcastVariableInitializer; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.state.AggregatingState; +import org.apache.flink.api.common.state.AggregatingStateDescriptor; +import org.apache.flink.api.common.state.FoldingState; +import org.apache.flink.api.common.state.FoldingStateDescriptor; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.state.MapState; +import org.apache.flink.api.common.state.MapStateDescriptor; +import org.apache.flink.api.common.state.ReducingState; +import org.apache.flink.api.common.state.ReducingStateDescriptor; +import org.apache.flink.api.common.state.State; +import org.apache.flink.api.common.state.StateDescriptor; +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.CharacterFilter; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.runtime.state.KeyGroupedInternalPriorityQueue; +import org.apache.flink.runtime.state.Keyed; +import org.apache.flink.runtime.state.KeyedStateBackend; +import org.apache.flink.runtime.state.KeyedStateFunction; +import org.apache.flink.runtime.state.PriorityComparable; +import org.apache.flink.runtime.state.StateSnapshotTransformer.StateSnapshotTransformFactory; +import org.apache.flink.runtime.state.VoidNamespace; +import org.apache.flink.runtime.state.heap.HeapPriorityQueueElement; +import org.apache.flink.runtime.state.internal.InternalListState; +import org.apache.flink.shaded.guava18.com.google.common.util.concurrent.MoreExecutors; +import org.apache.flink.streaming.api.operators.InternalTimerService; +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.operators.Triggerable; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.util.OutputTag; +import org.apache.flink.util.function.BiConsumerWithException; +import org.junit.Test; + +public class ReductionsTest { + + @Test + public void testFactory() { + + Configuration configuration = new Configuration(); + + Reductions reductions = + Reductions.create( + configuration, + new StatefulFunctionsUniverse(MessageFactoryType.WITH_KRYO_PAYLOADS), + new FakeRuntimeContext(), + new FakeKeyedStateBackend(), + new FakeTimerServiceFactory(), + new FakeInternalListState(), + new HashMap<>(), + new FakeOutput(), + TestUtils.ENVELOPE_FACTORY, + MoreExecutors.directExecutor(), + new FakeMetricGroup(), + new FakeMapState(), + MoreExecutors.directExecutor()); + + assertThat(reductions, notNullValue()); + } + + @SuppressWarnings("deprecation") + private static final class FakeRuntimeContext implements RuntimeContext { + + @Override + public ValueState getState(ValueStateDescriptor stateProperties) { + return new ValueState() { + @Override + public T value() { + return null; + } + + @Override + public void update(T value) {} + + @Override + public void clear() {} + }; + } + + @Override + public MapState getMapState(MapStateDescriptor stateProperties) { + return new MapState() { + @Override + public UV get(UK key) { + return null; + } + + @Override + public void put(UK key, UV value) {} + + @Override + public void putAll(Map map) {} + + @Override + public void remove(UK key) {} + + @Override + public boolean contains(UK key) { + return false; + } + + @Override + public Iterable> entries() { + return null; + } + + @Override + public Iterable keys() { + return null; + } + + @Override + public Iterable values() { + return null; + } + + @Override + public Iterator> iterator() { + return null; + } + + @Override + public void clear() {} + }; + } + + @Override + public ExecutionConfig getExecutionConfig() { + return new ExecutionConfig(); + } + + // everything below this line would throw UnspportedOperationException() + + @Override + public String getTaskName() { + throw new UnsupportedOperationException(); + } + + @Override + public MetricGroup getMetricGroup() { + throw new UnsupportedOperationException(); + } + + @Override + public int getNumberOfParallelSubtasks() { + return 0; + } + + @Override + public int getMaxNumberOfParallelSubtasks() { + return 0; + } + + @Override + public int getIndexOfThisSubtask() { + return 0; + } + + @Override + public int getAttemptNumber() { + return 0; + } + + @Override + public String getTaskNameWithSubtasks() { + throw new UnsupportedOperationException(); + } + + @Override + public ClassLoader getUserCodeClassLoader() { + throw new UnsupportedOperationException(); + } + + @Override + public void addAccumulator( + String name, Accumulator accumulator) {} + + @Override + public Accumulator getAccumulator(String name) { + throw new UnsupportedOperationException(); + } + + @Override + public Map> getAllAccumulators() { + throw new UnsupportedOperationException(); + } + + @Override + public IntCounter getIntCounter(String name) { + throw new UnsupportedOperationException(); + } + + @Override + public LongCounter getLongCounter(String name) { + throw new UnsupportedOperationException(); + } + + @Override + public DoubleCounter getDoubleCounter(String name) { + throw new UnsupportedOperationException(); + } + + @Override + public Histogram getHistogram(String name) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean hasBroadcastVariable(String name) { + return false; + } + + @Override + public List getBroadcastVariable(String name) { + throw new UnsupportedOperationException(); + } + + @Override + public C getBroadcastVariableWithInitializer( + String name, BroadcastVariableInitializer initializer) { + throw new UnsupportedOperationException(); + } + + @Override + public DistributedCache getDistributedCache() { + throw new UnsupportedOperationException(); + } + + @Override + public ListState getListState(ListStateDescriptor stateProperties) { + throw new UnsupportedOperationException(); + } + + @Override + public ReducingState getReducingState(ReducingStateDescriptor stateProperties) { + throw new UnsupportedOperationException(); + } + + @Override + public AggregatingState getAggregatingState( + AggregatingStateDescriptor stateProperties) { + throw new UnsupportedOperationException(); + } + + @Override + public FoldingState getFoldingState( + FoldingStateDescriptor stateProperties) { + throw new UnsupportedOperationException(); + } + } + + private static final class FakeKeyedStateBackend implements KeyedStateBackend { + + @Override + public void applyToAllKeys( + N namespace, + TypeSerializer namespaceSerializer, + StateDescriptor stateDescriptor, + KeyedStateFunction function) {} + + @Override + public Stream getKeys(String state, N namespace) { + throw new UnsupportedOperationException(); + } + + @Override + public S getOrCreateKeyedState( + TypeSerializer namespaceSerializer, StateDescriptor stateDescriptor) { + throw new UnsupportedOperationException(); + } + + @Override + public S getPartitionedState( + N namespace, TypeSerializer namespaceSerializer, StateDescriptor stateDescriptor) { + throw new UnsupportedOperationException(); + } + + @Override + public void dispose() {} + + @Override + public void registerKeySelectionListener(KeySelectionListener listener) {} + + @Override + public boolean deregisterKeySelectionListener(KeySelectionListener listener) { + return false; + } + + @Nonnull + @Override + public IS createInternalState( + @Nonnull TypeSerializer namespaceSerializer, + @Nonnull StateDescriptor stateDesc, + @Nonnull StateSnapshotTransformFactory snapshotTransformFactory) { + throw new UnsupportedOperationException(); + } + + @Nonnull + @Override + public + KeyGroupedInternalPriorityQueue create( + @Nonnull String stateName, @Nonnull TypeSerializer byteOrderedElementSerializer) { + throw new UnsupportedOperationException(); + } + + @Override + public Object getCurrentKey() { + throw new UnsupportedOperationException(); + } + + @Override + public void setCurrentKey(Object newKey) {} + + @Override + public TypeSerializer getKeySerializer() { + throw new UnsupportedOperationException(); + } + } + + private static final class FakeTimerServiceFactory implements TimerServiceFactory { + + @Override + public InternalTimerService createTimerService( + Triggerable triggerable) { + return new FakeTimerService(); + } + } + + private static final class FakeTimerService implements InternalTimerService { + + @Override + public long currentProcessingTime() { + return 0; + } + + @Override + public long currentWatermark() { + return 0; + } + + @Override + public void registerEventTimeTimer(VoidNamespace namespace, long time) { + throw new UnsupportedOperationException(); + } + + @Override + public void registerProcessingTimeTimer(VoidNamespace namespace, long time) { + throw new UnsupportedOperationException(); + } + + @Override + public void deleteEventTimeTimer(VoidNamespace namespace, long time) { + throw new UnsupportedOperationException(); + } + + @Override + public void deleteProcessingTimeTimer(VoidNamespace namespace, long time) { + throw new UnsupportedOperationException(); + } + + @Override + public void forEachEventTimeTimer( + BiConsumerWithException consumer) throws Exception { + throw new UnsupportedOperationException(); + } + + @Override + public void forEachProcessingTimeTimer( + BiConsumerWithException consumer) throws Exception { + throw new UnsupportedOperationException(); + } + } + + private static final class FakeInternalListState + implements InternalListState { + + @Override + public void add(Message value) throws Exception { + throw new UnsupportedOperationException(); + } + + @Override + public void addAll(List values) throws Exception { + throw new UnsupportedOperationException(); + } + + @Override + public void update(List values) throws Exception { + throw new UnsupportedOperationException(); + } + + @Override + public void updateInternal(List valueToStore) throws Exception { + throw new UnsupportedOperationException(); + } + + @Override + public void setCurrentNamespace(Long namespace) { + throw new UnsupportedOperationException(); + } + + @Override + public void clear() { + throw new UnsupportedOperationException(); + } + + @Override + public byte[] getSerializedValue( + byte[] serializedKeyAndNamespace, + TypeSerializer safeKeySerializer, + TypeSerializer safeNamespaceSerializer, + TypeSerializer> safeValueSerializer) + throws Exception { + throw new UnsupportedOperationException(); + } + + @Override + public List getInternal() throws Exception { + throw new UnsupportedOperationException(); + } + + @Override + public Iterable get() throws Exception { + throw new UnsupportedOperationException(); + } + + @Override + public void mergeNamespaces(Long target, Collection sources) throws Exception { + throw new UnsupportedOperationException(); + } + + @Override + public StateIncrementalVisitor> getStateIncrementalVisitor( + int recommendedMaxNumberOfReturnedRecords) { + throw new UnsupportedOperationException(); + } + + @Override + public TypeSerializer getNamespaceSerializer() { + throw new UnsupportedOperationException(); + } + + @Override + public TypeSerializer getKeySerializer() { + throw new UnsupportedOperationException(); + } + + @Override + public TypeSerializer> getValueSerializer() { + throw new UnsupportedOperationException(); + } + } + + private static final class FakeMapState implements MapState { + + @Override + public Message get(Long key) throws Exception { + return null; + } + + @Override + public void put(Long key, Message value) throws Exception {} + + @Override + public void putAll(Map map) throws Exception {} + + @Override + public void remove(Long key) throws Exception {} + + @Override + public boolean contains(Long key) throws Exception { + return false; + } + + @Override + public Iterable> entries() throws Exception { + return null; + } + + @Override + public Iterable keys() throws Exception { + return null; + } + + @Override + public Iterable values() throws Exception { + return null; + } + + @Override + public Iterator> iterator() throws Exception { + return null; + } + + @Override + public void clear() {} + } + + private static final class FakeOutput implements Output> { + + @Override + public void emitWatermark(Watermark mark) {} + + @Override + public void collect(OutputTag outputTag, StreamRecord record) {} + + @Override + public void emitLatencyMarker(LatencyMarker latencyMarker) {} + + @Override + public void collect(StreamRecord record) {} + + @Override + public void close() {} + } + + private static final class FakeMetricGroup implements MetricGroup { + @Override + public Counter counter(int i) { + throw new UnsupportedOperationException(); + } + + @Override + public Counter counter(String s) { + throw new UnsupportedOperationException(); + } + + @Override + public C counter(int i, C c) { + throw new UnsupportedOperationException(); + } + + @Override + public C counter(String s, C c) { + throw new UnsupportedOperationException(); + } + + @Override + public > G gauge(int i, G g) { + throw new UnsupportedOperationException(); + } + + @Override + public > G gauge(String s, G g) { + throw new UnsupportedOperationException(); + } + + @Override + public H histogram(String s, H h) { + throw new UnsupportedOperationException(); + } + + @Override + public H histogram(int i, H h) { + throw new UnsupportedOperationException(); + } + + @Override + public M meter(String s, M m) { + throw new UnsupportedOperationException(); + } + + @Override + public M meter(int i, M m) { + throw new UnsupportedOperationException(); + } + + @Override + public MetricGroup addGroup(int i) { + throw new UnsupportedOperationException(); + } + + @Override + public MetricGroup addGroup(String s) { + throw new UnsupportedOperationException(); + } + + @Override + public MetricGroup addGroup(String s, String s1) { + throw new UnsupportedOperationException(); + } + + @Override + public String[] getScopeComponents() { + return new String[0]; + } + + @Override + public Map getAllVariables() { + throw new UnsupportedOperationException(); + } + + @Override + public String getMetricIdentifier(String s) { + throw new UnsupportedOperationException(); + } + + @Override + public String getMetricIdentifier(String s, CharacterFilter characterFilter) { + throw new UnsupportedOperationException(); + } + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/logger/MemorySegmentPoolTest.java b/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/logger/MemorySegmentPoolTest.java new file mode 100644 index 00000000..2bd1e363 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/logger/MemorySegmentPoolTest.java @@ -0,0 +1,105 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.logger; + +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.CoreMatchers.nullValue; +import static org.junit.Assert.assertThat; + +import org.apache.flink.core.memory.MemorySegment; +import org.hamcrest.CoreMatchers; +import org.junit.Test; + +public class MemorySegmentPoolTest { + + @Test + public void emptyMemorySegmentPoolDoesNotAllocateSegments() { + MemorySegmentPool pool = new MemorySegmentPool(0); + + assertThat(pool.nextSegment(), nullValue()); + } + + @Test + public void emptyMemorySegmentPoolOverdraftsWhenAskedTo() { + MemorySegmentPool pool = new MemorySegmentPool(0); + + pool.ensureAtLeastOneSegmentPresent(); + + assertThat(pool.nextSegment(), notNullValue()); + } + + @Test + public void emptyMemorySegmentPoolOverdraftsTemporally() { + MemorySegmentPool pool = new MemorySegmentPool(0); + + pool.ensureAtLeastOneSegmentPresent(); + final MemorySegment overdraft = pool.nextSegment(); + pool.release(overdraft); + + assertThat(overdraft, notNullValue()); + assertThat(overdraft.isFreed(), is(true)); + assertThat(pool.nextSegment(), nullValue()); + } + + @Test + public void minimalAllocationUnitIsPageSize() { + MemorySegmentPool pool = new MemorySegmentPool(MemorySegmentPool.PAGE_SIZE - 1); + + assertThat(pool.nextSegment(), CoreMatchers.nullValue()); + } + + @Test + public void poolIsAbleToAllocateTheRequiredNumberOfPages() { + final int pageCount = 10; + MemorySegmentPool pool = new MemorySegmentPool(pageCount * MemorySegmentPool.PAGE_SIZE); + + for (int i = 0; i < pageCount; i++) { + MemorySegment segment = pool.nextSegment(); + + assertThat(segment, notNullValue()); + assertThat(segment.size(), is(MemorySegmentPool.PAGE_SIZE)); + } + + assertThat(pool.nextSegment(), nullValue()); + } + + @SuppressWarnings("PointlessArithmeticExpression") + @Test + public void segmentsCanBeReturnedToThePool() { + MemorySegmentPool pool = new MemorySegmentPool(1 * MemorySegmentPool.PAGE_SIZE); + // + // we can allocate at least 1 segment + // + MemorySegment segment = pool.nextSegment(); + assertThat(segment, notNullValue()); + // + // we can allocate exactly 1 segment + // + assertThat(pool.nextSegment(), nullValue()); + // + // return a segment to the pool + // + pool.release(segment); + // + // now we can use the segment + // + MemorySegment pooled = pool.nextSegment(); + assertThat(pooled, notNullValue()); + assertThat(pooled.isFreed(), is(false)); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/logger/UnboundedFeedbackLoggerTest.java b/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/logger/UnboundedFeedbackLoggerTest.java new file mode 100644 index 00000000..55a5c943 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/logger/UnboundedFeedbackLoggerTest.java @@ -0,0 +1,157 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.logger; + +import static com.ververica.statefun.flink.core.TestUtils.DUMMY_PAYLOAD; +import static com.ververica.statefun.flink.core.TestUtils.integerAddress; +import static org.hamcrest.CoreMatchers.is; +import static org.junit.Assert.assertThat; + +import com.ververica.statefun.flink.core.di.ObjectContainer; +import com.ververica.statefun.flink.core.message.Message; +import com.ververica.statefun.flink.core.message.MessageFactory; +import com.ververica.statefun.flink.core.message.MessageFactoryType; +import com.ververica.statefun.flink.core.message.MessageTypeInformation; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.Closeable; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Ignore; +import org.junit.Test; + +@SuppressWarnings("SameParameterValue") +public class UnboundedFeedbackLoggerTest { + private static IOManagerAsync IO_MANAGER; + + @BeforeClass + public static void beforeClass() { + IO_MANAGER = new IOManagerAsync(); + } + + @AfterClass + public static void afterClass() throws Exception { + if (IO_MANAGER != null) { + IO_MANAGER.close(); + IO_MANAGER = null; + } + } + + @Test + public void sanity() { + UnboundedFeedbackLogger logger = instanceUnderTest(128, 1); + + ByteArrayOutputStream output = new ByteArrayOutputStream(); + logger.startLogging(output); + logger.commit(); + + assertThat(output.size(), is(0)); + } + + @Test(expected = IllegalStateException.class) + public void commitWithoutStartLoggingShouldBeIllegal() { + UnboundedFeedbackLogger logger = instanceUnderTest(128, 1); + + logger.commit(); + } + + @Test + public void roundTrip() throws Exception { + roundTrip(100, 1024); + } + + @Ignore + @Test + public void roundTripWithSpill() throws Exception { + roundTrip(1_000_000, 0); + } + + private void roundTrip(int numElements, int maxMemoryInBytes) throws Exception { + InputStream input = serializeKeyGroup(1, maxMemoryInBytes, numElements); + + ArrayList envelopes = new ArrayList<>(); + + UnboundedFeedbackLogger loggerUnderTest = instanceUnderTest(1, 0); + loggerUnderTest.replyLoggedEnvelops(input, envelopes::add); + + MessageFactory factory = MessageFactory.forType(MessageFactoryType.WITH_PROTOBUF_PAYLOADS); + for (int i = 0; i < numElements; i++) { + Message adaptor = envelopes.get(i); + + assertThat(adaptor.source(), is(integerAddress(2 * i))); + assertThat(adaptor.target(), is(integerAddress(2 * i + 1))); + + Object payload = adaptor.payload(factory, getClass().getClassLoader()); + assertThat(payload, is(DUMMY_PAYLOAD)); + } + } + + private ByteArrayInputStream serializeKeyGroup(int maxParallelism, long maxMemory, int numItems) { + ByteArrayOutputStream output = new ByteArrayOutputStream(); + + UnboundedFeedbackLogger loggerUnderTest = instanceUnderTest(maxParallelism, maxMemory); + + loggerUnderTest.startLogging(output); + MessageFactory factory = MessageFactory.forType(MessageFactoryType.WITH_PROTOBUF_PAYLOADS); + for (int i = 0; i < numItems; i++) { + + Message envelope = + factory.from(integerAddress(2 * i), integerAddress(2 * i + 1), DUMMY_PAYLOAD); + + loggerUnderTest.append(envelope); + } + + loggerUnderTest.commit(); + + return new ByteArrayInputStream(output.toByteArray()); + } + + private UnboundedFeedbackLogger instanceUnderTest(int maxParallelism, long totalMemory) { + TypeSerializer serializer = + new MessageTypeInformation(MessageFactoryType.WITH_PROTOBUF_PAYLOADS) + .createSerializer(new ExecutionConfig()); + + ObjectContainer container = + Loggers.unboundedSpillableLoggerContainer( + IO_MANAGER, maxParallelism, totalMemory, serializer); + container.add("checkpoint-stream-ops", CheckpointedStreamOperations.class, NOOP.INSTANCE); + return container.get(UnboundedFeedbackLogger.class); + } + + enum NOOP implements CheckpointedStreamOperations { + INSTANCE; + + @Override + public void requireKeyedStateCheckpointed(OutputStream keyedStateCheckpointOutputStream) { + // noop + } + + @Override + public void startNewKeyGroup(OutputStream stream, int keyGroup) {} + + @Override + public Closeable acquireLease(OutputStream keyedStateCheckpointOutputStream) { + return () -> {}; // NOOP + } + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/message/MessageTest.java b/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/message/MessageTest.java new file mode 100644 index 00000000..79a7e812 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/message/MessageTest.java @@ -0,0 +1,73 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.message; + +import static com.ververica.statefun.flink.core.TestUtils.DUMMY_PAYLOAD; +import static com.ververica.statefun.flink.core.TestUtils.FUNCTION_1_ADDR; +import static com.ververica.statefun.flink.core.TestUtils.FUNCTION_2_ADDR; +import static com.ververica.statefun.flink.core.message.MessageFactoryType.WITH_KRYO_PAYLOADS; +import static com.ververica.statefun.flink.core.message.MessageFactoryType.WITH_PROTOBUF_PAYLOADS; +import static com.ververica.statefun.flink.core.message.MessageFactoryType.WITH_RAW_PAYLOADS; +import static org.hamcrest.CoreMatchers.is; +import static org.junit.Assert.assertThat; + +import java.io.IOException; +import java.util.Arrays; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +@RunWith(Parameterized.class) +public class MessageTest { + private final MessageFactoryType type; + private final Object payload; + + public MessageTest(MessageFactoryType type, Object payload) { + this.type = type; + this.payload = payload; + } + + @Parameters + public static Iterable data() { + return Arrays.asList( + new Object[] {WITH_KRYO_PAYLOADS, DUMMY_PAYLOAD}, + new Object[] {WITH_PROTOBUF_PAYLOADS, DUMMY_PAYLOAD}, + new Object[] {WITH_RAW_PAYLOADS, DUMMY_PAYLOAD.toByteArray()}); + } + + @Test + public void roundTrip() throws IOException { + MessageFactory factory = MessageFactory.forType(type); + + Message fromSdk = factory.from(FUNCTION_1_ADDR, FUNCTION_2_ADDR, payload); + DataOutputSerializer out = new DataOutputSerializer(32); + fromSdk.writeTo(factory, out); + + Message fromEnvelope = factory.from(new DataInputDeserializer(out.getCopyOfBuffer())); + + assertThat(fromEnvelope.source(), is(FUNCTION_1_ADDR)); + assertThat(fromEnvelope.target(), is(FUNCTION_2_ADDR)); + + ClassLoader targetClassLoader = payload.getClass().getClassLoader(); + Object payload = fromEnvelope.payload(factory, targetClassLoader); + + assertThat(payload, is(payload)); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/message/MessageTypeSerializerTest.java b/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/message/MessageTypeSerializerTest.java new file mode 100644 index 00000000..aaffabe6 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/message/MessageTypeSerializerTest.java @@ -0,0 +1,83 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.message; + +import com.ververica.statefun.flink.core.TestUtils; +import java.io.IOException; +import java.util.Arrays; +import java.util.stream.LongStream; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.testutils.DeeplyEqualsChecker; +import org.junit.Ignore; + +public class MessageTypeSerializerTest extends SerializerTestBase { + + public MessageTypeSerializerTest() { + super( + new DeeplyEqualsChecker() { + @Override + public boolean deepEquals(Object o1, Object o2) { + Message a = (Message) o1; + Message b = (Message) o2; + DataOutputSerializer aOut = new DataOutputSerializer(32); + DataOutputSerializer bOut = new DataOutputSerializer(32); + MessageFactory factory = MessageFactory.forType(MessageFactoryType.WITH_KRYO_PAYLOADS); + try { + a.writeTo(factory, aOut); + } catch (IOException e) { + throw new RuntimeException(e); + } + try { + b.writeTo(factory, bOut); + } catch (IOException e) { + throw new RuntimeException(e); + } + return Arrays.equals(aOut.getCopyOfBuffer(), bOut.getCopyOfBuffer()); + } + }); + } + + @Override + protected TypeSerializer createSerializer() { + return new MessageTypeInformation(MessageFactoryType.WITH_KRYO_PAYLOADS) + .createSerializer(new ExecutionConfig()); + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return Message.class; + } + + @Override + protected Message[] getTestData() { + return LongStream.range(1, 100) + .mapToObj(TestUtils.ENVELOPE_FACTORY::from) + .toArray(Message[]::new); + } + + @Ignore + @Override + public void testInstantiate() {} +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/state/StateBinderTest.java b/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/state/StateBinderTest.java new file mode 100644 index 00000000..ce78d49e --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/state/StateBinderTest.java @@ -0,0 +1,129 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.state; + +import static org.hamcrest.CoreMatchers.hasItems; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertTrue; + +import com.ververica.statefun.flink.core.TestUtils; +import com.ververica.statefun.sdk.Address; +import com.ververica.statefun.sdk.FunctionType; +import com.ververica.statefun.sdk.annotations.Persisted; +import com.ververica.statefun.sdk.state.Accessor; +import com.ververica.statefun.sdk.state.PersistedValue; +import java.util.HashSet; +import java.util.Set; +import org.junit.Test; + +public class StateBinderTest { + + // test collaborators + private final FakeState state = new FakeState(); + + // object under test + private final StateBinder binderUnderTest = new StateBinder(state); + + @Test + public void exampleUsage() { + binderUnderTest.bind(TestUtils.FUNCTION_TYPE, new SanityClass()); + + assertThat(state.boundNames, hasItems("name", "last")); + } + + @Test(expected = IllegalArgumentException.class) + public void nonPersistedFieldAnnotated() { + binderUnderTest.bind(TestUtils.FUNCTION_TYPE, new WrongAnnotationClass()); + } + + @Test(expected = IllegalStateException.class) + public void nullValueField() { + binderUnderTest.bind(TestUtils.FUNCTION_TYPE, new NullValueClass()); + } + + @Test + public void nonAnnotatedClass() { + binderUnderTest.bind(TestUtils.FUNCTION_TYPE, new IgnoreNonAnnotated()); + + assertTrue(state.boundNames.isEmpty()); + } + + static final class SanityClass { + + @SuppressWarnings("unused") + @Persisted + PersistedValue name = PersistedValue.of("name", String.class); + + @Persisted + @SuppressWarnings("unused") + PersistedValue last = PersistedValue.of("last", String.class); + } + + static final class WrongAnnotationClass { + + @SuppressWarnings("unused") + @Persisted + String name = ""; + } + + static final class NullValueClass { + + @SuppressWarnings("unused") + @Persisted + PersistedValue last; + } + + static final class IgnoreNonAnnotated { + + @SuppressWarnings("unused") + PersistedValue last = PersistedValue.of("last", String.class); + } + + private static final class FakeState implements State { + Set boundNames = new HashSet<>(); + + @Override + public Accessor createFlinkStateAccessor( + FunctionType functionType, PersistedValue persistedValue) { + boundNames.add(persistedValue.name()); + + return new Accessor() { + T value; + + @Override + public void set(T value) { + this.value = value; + } + + @Override + public T get() { + return value; + } + + @Override + public void clear() { + value = null; + } + }; + } + + @Override + public void setCurrentKey(Address key) { + throw new UnsupportedOperationException(); + } + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/types/protobuf/ProtobufSerializerTest.java b/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/types/protobuf/ProtobufSerializerTest.java new file mode 100644 index 00000000..658de814 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/types/protobuf/ProtobufSerializerTest.java @@ -0,0 +1,125 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.types.protobuf; + +import static com.ververica.statefun.flink.core.TestUtils.ADDRESS; +import static com.ververica.statefun.flink.core.TestUtils.envelopesOfVariousSizes; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.MatcherAssert.assertThat; + +import com.google.protobuf.Message; +import com.ververica.statefun.flink.core.generated.Envelope; +import com.ververica.statefun.flink.core.generated.EnvelopeAddress; +import com.ververica.statefun.flink.core.generated.ProtobufSerializerSnapshot; +import java.io.IOException; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.junit.Test; + +public class ProtobufSerializerTest { + + @Test + public void roundTrip() throws IOException { + EnvelopeAddress address = roundTrip(EnvelopeAddress.class, ADDRESS); + + assertThat(ADDRESS, is(address)); + } + + @Test + public void deserializeCopiedMessage() throws IOException { + DataInputDeserializer in = serialize(EnvelopeAddress.class, ADDRESS); + + ProtobufSerializer serializer = + ProtobufSerializer.forMessageGeneratedClass(EnvelopeAddress.class); + DataOutputSerializer out = new DataOutputSerializer(32); + + serializer.copy(in, out); + EnvelopeAddress message = deserialize(EnvelopeAddress.class, out); + + assertThat(message, is(ADDRESS)); + } + + @Test + public void sequenceOfMessageIsDeserializeProperly() throws IOException { + Envelope[] envelopes = envelopesOfVariousSizes(); + + DataInputDeserializer in = serialize(Envelope.class, envelopes); + + for (Envelope originalEnvelope : envelopes) { + Envelope envelope = deserialize(Envelope.class, in); + assertThat(envelope, is(originalEnvelope)); + } + } + + @Test + public void testSnapshot() { + ProtobufSerializer serializer = + ProtobufSerializer.forMessageGeneratedClass(EnvelopeAddress.class); + ProtobufSerializerSnapshot snapshot = serializer.snapshot(); + + assertThat(snapshot.getGeneratedJavaName(), is(EnvelopeAddress.class.getName())); + assertThat(snapshot.getMessageName(), is(EnvelopeAddress.getDescriptor().getFullName())); + assertThat(snapshot.getDescriptorSet(), notNullValue()); + } + + @Test + public void duplicatedSerializerCanDeserialize() throws IOException { + ProtobufSerializer serializer = + ProtobufSerializer.forMessageGeneratedClass(EnvelopeAddress.class); + + DataOutputSerializer out = new DataOutputSerializer(512); + serializer.serialize(ADDRESS, out); + + DataInputDeserializer in = new DataInputDeserializer(out.getCopyOfBuffer()); + EnvelopeAddress foo = serializer.duplicate().deserialize(in); + + assertThat(foo, is(ADDRESS)); + } + + @SuppressWarnings("SameParameterValue") + private static M roundTrip(Class messageType, M original) + throws IOException { + DataInputDeserializer source = serialize(messageType, original); + return deserialize(messageType, source); + } + + @SafeVarargs + private static DataInputDeserializer serialize(Class type, M... items) + throws IOException { + ProtobufSerializer serializer = ProtobufSerializer.forMessageGeneratedClass(type); + + DataOutputSerializer out = new DataOutputSerializer(512); + for (Object message : items) { + serializer.serialize(type.cast(message), out); + } + return new DataInputDeserializer(out.getCopyOfBuffer()); + } + + @SuppressWarnings("SameParameterValue") + private static M deserialize(Class type, DataOutputSerializer target) + throws IOException { + DataInputDeserializer source = new DataInputDeserializer(target.getCopyOfBuffer()); + return deserialize(type, source); + } + + private static M deserialize(Class type, DataInputDeserializer source) + throws IOException { + ProtobufSerializer serializer = ProtobufSerializer.forMessageGeneratedClass(type); + return serializer.deserialize(source); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/types/protobuf/ProtobufTypeSerializerTest.java b/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/types/protobuf/ProtobufTypeSerializerTest.java new file mode 100644 index 00000000..d451f68d --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-core/src/test/java/com/ververica/statefun/flink/core/types/protobuf/ProtobufTypeSerializerTest.java @@ -0,0 +1,58 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.core.types.protobuf; + +import com.ververica.statefun.flink.core.generated.EnvelopeAddress; +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.junit.Ignore; +import org.junit.Test; + +public class ProtobufTypeSerializerTest extends SerializerTestBase { + + @Override + protected TypeSerializer createSerializer() { + return new ProtobufTypeSerializer<>(EnvelopeAddress.class); + } + + @Ignore + @Test() + @Override + public void testInstantiate() { + // do nothing. + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return EnvelopeAddress.class; + } + + @Override + protected EnvelopeAddress[] getTestData() { + return new EnvelopeAddress[] { + EnvelopeAddress.newBuilder().setType("").setNamespace("").setId("").build(), + EnvelopeAddress.newBuilder().setType("").setNamespace("").setId("").build(), + EnvelopeAddress.newBuilder().setType("").setNamespace("").setId("").build(), + EnvelopeAddress.newBuilder().setType("").setNamespace("").setId("").build() + }; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-distribution/pom.xml b/stateful-functions-flink/stateful-functions-flink-distribution/pom.xml new file mode 100644 index 00000000..c081a461 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-distribution/pom.xml @@ -0,0 +1,145 @@ + + + + 4.0.0 + + + com.ververica + stateful-functions-flink + 1.0-SNAPSHOT + .. + + + stateful-functions-flink-distribution + + + + + org.slf4j + slf4j-log4j12 + 1.7.7 + + + log4j + log4j + 1.2.17 + + + com.google.protobuf + protobuf-java + + + + + com.ververica + stateful-functions-sdk + ${project.version} + + + + + com.ververica + stateful-functions-flink-io + ${project.version} + + + com.ververica + stateful-functions-flink-io-bundle + ${project.version} + + + com.ververica + stateful-functions-flink-core + ${project.version} + + + com.ververica + stateful-functions-flink-launcher + ${project.version} + + + + + org.apache.flink + flink-streaming-java_2.11 + ${flink.version} + provided + + + org.apache.flink + flink-runtime_2.11 + ${flink.version} + provided + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.0.0 + + + + package + + shade + + + false + + + org.apache.flink:force-shading + com.google.code.findbugs:jsr305 + org.slf4j:* + log4j:* + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + com.ververica.statefun.flink.core.StatefulFunctionsJob + + + + + + + + + + + + \ No newline at end of file diff --git a/stateful-functions-flink/stateful-functions-flink-harness/pom.xml b/stateful-functions-flink/stateful-functions-flink-harness/pom.xml new file mode 100644 index 00000000..83e93f96 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-harness/pom.xml @@ -0,0 +1,83 @@ + + + + 4.0.0 + + + com.ververica + stateful-functions-flink + 1.0-SNAPSHOT + .. + + + stateful-functions-flink-harness + + + + + com.ververica + stateful-functions-flink-distribution + ${project.version} + + + + + org.apache.flink + flink-streaming-java_2.11 + ${flink.version} + + + org.apache.flink + flink-runtime_2.11 + ${flink.version} + + + + + + + + + + add-dependencies-for-IDEA + + + + idea.version + + + + + + org.apache.flink + flink-streaming-java_2.11 + ${flink.version} + compile + + + org.apache.flink + flink-runtime-web_2.11 + ${flink.version} + compile + + + + + + \ No newline at end of file diff --git a/stateful-functions-flink/stateful-functions-flink-harness/src/main/java/com/ververica/statefun/flink/harness/Harness.java b/stateful-functions-flink/stateful-functions-flink-harness/src/main/java/com/ververica/statefun/flink/harness/Harness.java new file mode 100644 index 00000000..4b1c6b62 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-harness/src/main/java/com/ververica/statefun/flink/harness/Harness.java @@ -0,0 +1,116 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.harness; + +import com.ververica.statefun.flink.core.StatefulFunctionsJob; +import com.ververica.statefun.flink.core.StatefulFunctionsJobConstants; +import com.ververica.statefun.flink.core.StatefulFunctionsUniverse; +import com.ververica.statefun.flink.core.StatefulFunctionsUniverseProvider; +import com.ververica.statefun.flink.core.common.ConfigurationUtil; +import com.ververica.statefun.flink.core.message.MessageFactoryType; +import com.ververica.statefun.flink.core.spi.Modules; +import com.ververica.statefun.flink.harness.io.ConsumingEgressSpec; +import com.ververica.statefun.flink.harness.io.SerializableConsumer; +import com.ververica.statefun.flink.harness.io.SerializableSupplier; +import com.ververica.statefun.flink.harness.io.SupplyingIngressSpec; +import com.ververica.statefun.flink.io.datastream.SourceFunctionSpec; +import com.ververica.statefun.sdk.io.EgressIdentifier; +import com.ververica.statefun.sdk.io.EgressSpec; +import com.ververica.statefun.sdk.io.IngressIdentifier; +import com.ververica.statefun.sdk.io.IngressSpec; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.source.SourceFunction; + +public class Harness { + private final Configuration configuration = new Configuration(); + + private final Map, IngressSpec> overrideIngress = new HashMap<>(); + private final Map, EgressSpec> overrideEgress = new HashMap<>(); + + public Harness withSupplyingIngress( + IngressIdentifier identifier, SerializableSupplier supplier) { + Objects.requireNonNull(identifier); + Objects.requireNonNull(supplier); + // TODO: consider closure cleaner + overrideIngress.put(identifier, new SupplyingIngressSpec<>(identifier, supplier, 0)); + return this; + } + + public Harness withFlinkSourceFunction( + IngressIdentifier identifier, SourceFunction supplier) { + Objects.requireNonNull(identifier); + Objects.requireNonNull(supplier); + overrideIngress.put(identifier, new SourceFunctionSpec<>(identifier, supplier)); + return this; + } + + public Harness withConsumingEgress( + EgressIdentifier identifier, SerializableConsumer consumer) { + Objects.requireNonNull(identifier); + Objects.requireNonNull(consumer); + // TODO: consider closure cleaner + overrideEgress.put(identifier, new ConsumingEgressSpec<>(identifier, consumer)); + return this; + } + + public Harness withKryoMessageSerializer() { + configuration.setString( + StatefulFunctionsJobConstants.USER_MESSAGE_SERIALIZER, + MessageFactoryType.WITH_KRYO_PAYLOADS.name()); + return this; + } + + public Harness noCheckpointing() { + configuration.setLong(StatefulFunctionsJobConstants.CHECKPOINTING_INTERVAL, -1); + return this; + } + + public void start() throws Exception { + ConfigurationUtil.storeSerializedInstance( + configuration, + StatefulFunctionsJobConstants.STATEFUL_FUNCTIONS_UNIVERSE_INITIALIZER_CLASS_BYTES, + new HarnessProvider(overrideIngress, overrideEgress)); + + StatefulFunctionsJob.main(configuration); + } + + private static final class HarnessProvider implements StatefulFunctionsUniverseProvider { + private static final long serialVersionUID = 1; + + private final Map, IngressSpec> ingressToReplace; + private final Map, EgressSpec> egressToReplace; + + HarnessProvider( + Map, IngressSpec> dummyIngress, + Map, EgressSpec> dummyEgress) { + this.ingressToReplace = dummyIngress; + this.egressToReplace = dummyEgress; + } + + @Override + public StatefulFunctionsUniverse get(ClassLoader classLoader, Configuration configuration) { + Modules modules = Modules.loadFromClassPath(); + StatefulFunctionsUniverse universe = modules.createStatefulFunctionsUniverse(configuration); + ingressToReplace.forEach((id, spec) -> universe.ingress().put(id, spec)); + egressToReplace.forEach((id, spec) -> universe.egress().put(id, spec)); + return universe; + } + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-harness/src/main/java/com/ververica/statefun/flink/harness/io/ConsumingEgressSpec.java b/stateful-functions-flink/stateful-functions-flink-harness/src/main/java/com/ververica/statefun/flink/harness/io/ConsumingEgressSpec.java new file mode 100644 index 00000000..afc1b8fb --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-harness/src/main/java/com/ververica/statefun/flink/harness/io/ConsumingEgressSpec.java @@ -0,0 +1,50 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.harness.io; + +import com.ververica.statefun.sdk.EgressType; +import com.ververica.statefun.sdk.io.EgressIdentifier; +import com.ververica.statefun.sdk.io.EgressSpec; +import java.io.Serializable; +import java.util.Objects; + +public final class ConsumingEgressSpec implements EgressSpec, Serializable { + + private static final long serialVersionUID = 1; + + private final EgressIdentifier id; + private final SerializableConsumer consumer; + + public ConsumingEgressSpec(EgressIdentifier id, SerializableConsumer consumer) { + this.id = Objects.requireNonNull(id); + this.consumer = Objects.requireNonNull(consumer); + } + + @Override + public EgressIdentifier id() { + return id; + } + + @Override + public EgressType type() { + return HarnessConstants.CONSUMING_EGRESS_TYPE; + } + + SerializableConsumer consumer() { + return consumer; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-harness/src/main/java/com/ververica/statefun/flink/harness/io/ConsumingSink.java b/stateful-functions-flink/stateful-functions-flink-harness/src/main/java/com/ververica/statefun/flink/harness/io/ConsumingSink.java new file mode 100644 index 00000000..0e574efe --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-harness/src/main/java/com/ververica/statefun/flink/harness/io/ConsumingSink.java @@ -0,0 +1,36 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.harness.io; + +import java.util.Objects; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; + +final class ConsumingSink extends RichSinkFunction { + + private static final long serialVersionUID = 1; + + private final SerializableConsumer consumer; + + ConsumingSink(SerializableConsumer consumer) { + this.consumer = Objects.requireNonNull(consumer); + } + + @Override + public void invoke(T value, Context context) { + consumer.accept(value); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-harness/src/main/java/com/ververica/statefun/flink/harness/io/HarnessConstants.java b/stateful-functions-flink/stateful-functions-flink-harness/src/main/java/com/ververica/statefun/flink/harness/io/HarnessConstants.java new file mode 100644 index 00000000..1d02eb5c --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-harness/src/main/java/com/ververica/statefun/flink/harness/io/HarnessConstants.java @@ -0,0 +1,29 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.harness.io; + +import com.ververica.statefun.sdk.EgressType; +import com.ververica.statefun.sdk.IngressType; + +@SuppressWarnings("WeakerAccess") +public class HarnessConstants { + + public static final IngressType SUPPLYING_INGRESS_TYPE = + new IngressType("com.ververica.statefun.flink.harness", "supplier"); + public static final EgressType CONSUMING_EGRESS_TYPE = + new EgressType("com.ververica.statefun.flink.harness", "consuming-egress"); +} diff --git a/stateful-functions-flink/stateful-functions-flink-harness/src/main/java/com/ververica/statefun/flink/harness/io/HarnessIoModule.java b/stateful-functions-flink/stateful-functions-flink-harness/src/main/java/com/ververica/statefun/flink/harness/io/HarnessIoModule.java new file mode 100644 index 00000000..072183f5 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-harness/src/main/java/com/ververica/statefun/flink/harness/io/HarnessIoModule.java @@ -0,0 +1,49 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.harness.io; + +import com.ververica.statefun.flink.io.spi.FlinkIoModule; +import com.ververica.statefun.sdk.io.EgressSpec; +import com.ververica.statefun.sdk.io.IngressSpec; +import java.util.Map; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.api.functions.source.SourceFunction; + +public class HarnessIoModule implements FlinkIoModule { + + @Override + public void configure(Map globalConfiguration, Binder binder) { + binder.bindSourceProvider( + HarnessConstants.SUPPLYING_INGRESS_TYPE, HarnessIoModule::supplingIngressSpec); + binder.bindSinkProvider( + HarnessConstants.CONSUMING_EGRESS_TYPE, HarnessIoModule::consumingEgressSpec); + } + + @SuppressWarnings("unchecked") + private static SourceFunction supplingIngressSpec(IngressSpec spec) { + SupplyingIngressSpec casted = (SupplyingIngressSpec) spec; + return new SupplyingSource<>(casted.supplier(), casted.delayInMilliseconds()); + } + + private static SinkFunction consumingEgressSpec(EgressSpec spec) { + if (!(spec instanceof ConsumingEgressSpec)) { + throw new IllegalArgumentException("Unable to provider a source for " + spec); + } + ConsumingEgressSpec casted = (ConsumingEgressSpec) spec; + return new ConsumingSink<>(casted.consumer()); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-harness/src/main/java/com/ververica/statefun/flink/harness/io/SerializableConsumer.java b/stateful-functions-flink/stateful-functions-flink-harness/src/main/java/com/ververica/statefun/flink/harness/io/SerializableConsumer.java new file mode 100644 index 00000000..d84be7c6 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-harness/src/main/java/com/ververica/statefun/flink/harness/io/SerializableConsumer.java @@ -0,0 +1,22 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.harness.io; + +import java.io.Serializable; +import java.util.function.Consumer; + +public interface SerializableConsumer extends Serializable, Consumer {} diff --git a/stateful-functions-flink/stateful-functions-flink-harness/src/main/java/com/ververica/statefun/flink/harness/io/SerializableSupplier.java b/stateful-functions-flink/stateful-functions-flink-harness/src/main/java/com/ververica/statefun/flink/harness/io/SerializableSupplier.java new file mode 100644 index 00000000..52429b49 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-harness/src/main/java/com/ververica/statefun/flink/harness/io/SerializableSupplier.java @@ -0,0 +1,22 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.harness.io; + +import java.io.Serializable; +import java.util.function.Supplier; + +public interface SerializableSupplier extends Serializable, Supplier {} diff --git a/stateful-functions-flink/stateful-functions-flink-harness/src/main/java/com/ververica/statefun/flink/harness/io/SupplyingIngressSpec.java b/stateful-functions-flink/stateful-functions-flink-harness/src/main/java/com/ververica/statefun/flink/harness/io/SupplyingIngressSpec.java new file mode 100644 index 00000000..3875e0fe --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-harness/src/main/java/com/ververica/statefun/flink/harness/io/SupplyingIngressSpec.java @@ -0,0 +1,59 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.harness.io; + +import com.ververica.statefun.sdk.IngressType; +import com.ververica.statefun.sdk.io.IngressIdentifier; +import com.ververica.statefun.sdk.io.IngressSpec; +import java.io.Serializable; +import java.util.Objects; + +public final class SupplyingIngressSpec implements IngressSpec, Serializable { + + private static final long serialVersionUID = 1; + + private final IngressIdentifier id; + private final SerializableSupplier supplier; + private final long delayInMilliseconds; + + public SupplyingIngressSpec( + IngressIdentifier id, + SerializableSupplier supplier, + long productionDelayInMilliseconds) { + this.id = Objects.requireNonNull(id); + this.supplier = Objects.requireNonNull(supplier); + this.delayInMilliseconds = productionDelayInMilliseconds; + } + + @Override + public IngressIdentifier id() { + return id; + } + + @Override + public IngressType type() { + return HarnessConstants.SUPPLYING_INGRESS_TYPE; + } + + SerializableSupplier supplier() { + return supplier; + } + + long delayInMilliseconds() { + return delayInMilliseconds; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-harness/src/main/java/com/ververica/statefun/flink/harness/io/SupplyingSource.java b/stateful-functions-flink/stateful-functions-flink-harness/src/main/java/com/ververica/statefun/flink/harness/io/SupplyingSource.java new file mode 100644 index 00000000..04aff7b1 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-harness/src/main/java/com/ververica/statefun/flink/harness/io/SupplyingSource.java @@ -0,0 +1,50 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.harness.io; + +import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; + +final class SupplyingSource extends RichParallelSourceFunction { + private static final long serialVersionUID = 1; + + private final SerializableSupplier supplier; + private final long delayInMilliseconds; + private transient volatile boolean done; + + SupplyingSource(SerializableSupplier supplier, long delayInMilliseconds) { + this.supplier = supplier; + this.delayInMilliseconds = delayInMilliseconds; + } + + @Override + public void run(SourceContext sourceContext) throws Exception { + while (!done) { + final T nextElement = supplier.get(); + synchronized (sourceContext.getCheckpointLock()) { + sourceContext.collect(nextElement); + } + if (delayInMilliseconds > 0) { + Thread.sleep(delayInMilliseconds); + } + } + } + + @Override + public void cancel() { + done = true; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-harness/src/main/resources/META-INF/services/com.ververica.statefun.flink.io.spi.FlinkIoModule b/stateful-functions-flink/stateful-functions-flink-harness/src/main/resources/META-INF/services/com.ververica.statefun.flink.io.spi.FlinkIoModule new file mode 100644 index 00000000..763f7c1f --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-harness/src/main/resources/META-INF/services/com.ververica.statefun.flink.io.spi.FlinkIoModule @@ -0,0 +1,17 @@ +# +# Copyright 2019 Ververica GmbH. +# +# 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. +# + +com.ververica.statefun.flink.harness.io.HarnessIoModule \ No newline at end of file diff --git a/stateful-functions-flink/stateful-functions-flink-harness/src/main/resources/log4j.properties b/stateful-functions-flink/stateful-functions-flink-harness/src/main/resources/log4j.properties new file mode 100644 index 00000000..c5c57e85 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-harness/src/main/resources/log4j.properties @@ -0,0 +1,21 @@ +# +# Copyright 2019 Ververica GmbH. +# +# 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. +# + +log4j.rootLogger=INFO, console + +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n diff --git a/stateful-functions-flink/stateful-functions-flink-io-bundle/pom.xml b/stateful-functions-flink/stateful-functions-flink-io-bundle/pom.xml new file mode 100644 index 00000000..6b969ca1 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-io-bundle/pom.xml @@ -0,0 +1,86 @@ + + + + 4.0.0 + + + com.ververica + stateful-functions-flink + 1.0-SNAPSHOT + .. + + + stateful-functions-flink-io-bundle + + + + + com.ververica + stateful-functions-sdk + ${project.version} + + + + + com.ververica + stateful-functions-flink-io + ${project.version} + + + + + com.ververica + stateful-functions-kafka-io + ${project.version} + + + + org.apache.kafka + kafka-clients + + + + + + + org.apache.flink + flink-streaming-java_2.11 + ${flink.version} + + + + + org.apache.flink + flink-connector-kafka_2.11 + ${flink.version} + compile + + + + + junit + junit + 4.12 + test + + + + \ No newline at end of file diff --git a/stateful-functions-flink/stateful-functions-flink-io-bundle/src/main/java/com/ververica/statefun/flink/io/common/ReflectionUtil.java b/stateful-functions-flink/stateful-functions-flink-io-bundle/src/main/java/com/ververica/statefun/flink/io/common/ReflectionUtil.java new file mode 100644 index 00000000..04c358af --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-io-bundle/src/main/java/com/ververica/statefun/flink/io/common/ReflectionUtil.java @@ -0,0 +1,40 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.io.common; + +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; +import org.apache.flink.annotation.Internal; + +@Internal +public final class ReflectionUtil { + + private ReflectionUtil() {} + + public static T instantiate(Class type) { + try { + Constructor defaultConstructor = type.getDeclaredConstructor(); + defaultConstructor.setAccessible(true); + return defaultConstructor.newInstance(); + } catch (NoSuchMethodException e) { + throw new IllegalStateException( + "Unable to create an instance of " + type.getName() + " has no default constructor", e); + } catch (IllegalAccessException | InstantiationException | InvocationTargetException e) { + throw new IllegalStateException("Unable to create an instance of " + type.getName(), e); + } + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-io-bundle/src/main/java/com/ververica/statefun/flink/io/common/UnimplementedTypeInfo.java b/stateful-functions-flink/stateful-functions-flink-io-bundle/src/main/java/com/ververica/statefun/flink/io/common/UnimplementedTypeInfo.java new file mode 100644 index 00000000..ddb93329 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-io-bundle/src/main/java/com/ververica/statefun/flink/io/common/UnimplementedTypeInfo.java @@ -0,0 +1,81 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.io.common; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; + +public final class UnimplementedTypeInfo extends TypeInformation { + + private static final long serialVersionUID = 1; + + @Override + public boolean isBasicType() { + return false; + } + + @Override + public boolean isTupleType() { + return false; + } + + @Override + public int getArity() { + return 0; + } + + @Override + public int getTotalFields() { + return 0; + } + + @Override + public Class getTypeClass() { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isKeyType() { + throw new UnsupportedOperationException(); + } + + @Override + public TypeSerializer createSerializer(ExecutionConfig executionConfig) { + return new UnimplementedTypeSerializer<>(); + } + + @Override + public String toString() { + return "UnimplementedTypeInfo"; + } + + @Override + public boolean equals(Object o) { + return o == this; + } + + @Override + public int hashCode() { + return 1337; + } + + @Override + public boolean canEqual(Object o) { + return o instanceof UnimplementedTypeInfo; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-io-bundle/src/main/java/com/ververica/statefun/flink/io/common/UnimplementedTypeSerializer.java b/stateful-functions-flink/stateful-functions-flink-io-bundle/src/main/java/com/ververica/statefun/flink/io/common/UnimplementedTypeSerializer.java new file mode 100644 index 00000000..b1d8e0bb --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-io-bundle/src/main/java/com/ververica/statefun/flink/io/common/UnimplementedTypeSerializer.java @@ -0,0 +1,92 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.io.common; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +public final class UnimplementedTypeSerializer extends TypeSerializer { + + private static final long serialVersionUID = 1L; + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer duplicate() { + throw new UnsupportedOperationException(); + } + + @Override + public T createInstance() { + throw new UnsupportedOperationException(); + } + + @Override + public T copy(T t) { + throw new UnsupportedOperationException(); + } + + @Override + public T copy(T t, T t1) { + throw new UnsupportedOperationException(); + } + + @Override + public int getLength() { + return 0; + } + + @Override + public void serialize(T t, DataOutputView dataOutputView) { + throw new UnsupportedOperationException(); + } + + @Override + public T deserialize(DataInputView dataInputView) { + throw new UnsupportedOperationException(); + } + + @Override + public T deserialize(T t, DataInputView dataInputView) { + throw new UnsupportedOperationException(); + } + + @Override + public void copy(DataInputView dataInputView, DataOutputView dataOutputView) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean equals(Object o) { + return o == this; + } + + @Override + public int hashCode() { + return 7; + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + throw new UnsupportedOperationException(); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-io-bundle/src/main/java/com/ververica/statefun/flink/io/kafka/KafkaDeserializationSchemaDelegate.java b/stateful-functions-flink/stateful-functions-flink-io-bundle/src/main/java/com/ververica/statefun/flink/io/kafka/KafkaDeserializationSchemaDelegate.java new file mode 100644 index 00000000..ad1fd444 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-io-bundle/src/main/java/com/ververica/statefun/flink/io/kafka/KafkaDeserializationSchemaDelegate.java @@ -0,0 +1,59 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.io.kafka; + +import com.ververica.statefun.flink.io.common.UnimplementedTypeInfo; +import com.ververica.statefun.sdk.kafka.KafkaIngressDeserializer; +import java.util.Objects; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; +import org.apache.kafka.clients.consumer.ConsumerRecord; + +final class KafkaDeserializationSchemaDelegate implements KafkaDeserializationSchema { + + private static final long serialVersionUID = 1; + + private final TypeInformation producedTypeInfo; + private final KafkaIngressDeserializer delegate; + + KafkaDeserializationSchemaDelegate(KafkaIngressDeserializer delegate) { + this.producedTypeInfo = new UnimplementedTypeInfo<>(); + this.delegate = Objects.requireNonNull(delegate); + } + + @Override + public boolean isEndOfStream(T t) { + return false; + } + + @Override + public T deserialize(ConsumerRecord consumerRecord) { + return delegate.deserialize(consumerRecord); + } + + @Override + public TypeInformation getProducedType() { + // this would never be actually used, it would be replaced during translation with the type + // information + // of IngressIdentifier's producedType. + // see: Sources#setOutputType. + // if this invriant would not hold in the future, this type information would produce a + // serialier + // that fails immediately. + return producedTypeInfo; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-io-bundle/src/main/java/com/ververica/statefun/flink/io/kafka/KafkaFlinkIoModule.java b/stateful-functions-flink/stateful-functions-flink-io-bundle/src/main/java/com/ververica/statefun/flink/io/kafka/KafkaFlinkIoModule.java new file mode 100644 index 00000000..3253e6af --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-io-bundle/src/main/java/com/ververica/statefun/flink/io/kafka/KafkaFlinkIoModule.java @@ -0,0 +1,30 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.io.kafka; + +import com.ververica.statefun.flink.io.spi.FlinkIoModule; +import com.ververica.statefun.sdk.kafka.Constants; +import java.util.Map; + +public final class KafkaFlinkIoModule implements FlinkIoModule { + + @Override + public void configure(Map globalConfiguration, Binder binder) { + binder.bindSourceProvider(Constants.KAFKA_INGRESS_TYPE, new KafkaSourceProvider()); + binder.bindSinkProvider(Constants.KAFKA_EGRESS_TYPE, new KafkaSinkProvider()); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-io-bundle/src/main/java/com/ververica/statefun/flink/io/kafka/KafkaSerializationSchemaDelegate.java b/stateful-functions-flink/stateful-functions-flink-io-bundle/src/main/java/com/ververica/statefun/flink/io/kafka/KafkaSerializationSchemaDelegate.java new file mode 100644 index 00000000..c2235404 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-io-bundle/src/main/java/com/ververica/statefun/flink/io/kafka/KafkaSerializationSchemaDelegate.java @@ -0,0 +1,39 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.io.kafka; + +import com.ververica.statefun.sdk.kafka.KafkaEgressSerializer; +import java.util.Objects; +import javax.annotation.Nullable; +import org.apache.flink.streaming.connectors.kafka.KafkaSerializationSchema; +import org.apache.kafka.clients.producer.ProducerRecord; + +final class KafkaSerializationSchemaDelegate implements KafkaSerializationSchema { + + private static final long serialVersionUID = 1L; + + private final KafkaEgressSerializer serializer; + + KafkaSerializationSchemaDelegate(KafkaEgressSerializer serializer) { + this.serializer = Objects.requireNonNull(serializer); + } + + @Override + public ProducerRecord serialize(T t, @Nullable Long aLong) { + return serializer.serialize(t); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-io-bundle/src/main/java/com/ververica/statefun/flink/io/kafka/KafkaSinkProvider.java b/stateful-functions-flink/stateful-functions-flink-io-bundle/src/main/java/com/ververica/statefun/flink/io/kafka/KafkaSinkProvider.java new file mode 100644 index 00000000..7ba944dd --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-io-bundle/src/main/java/com/ververica/statefun/flink/io/kafka/KafkaSinkProvider.java @@ -0,0 +1,88 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.io.kafka; + +import static org.apache.flink.util.StringUtils.generateRandomAlphanumericString; + +import com.ververica.statefun.flink.io.common.ReflectionUtil; +import com.ververica.statefun.flink.io.spi.SinkProvider; +import com.ververica.statefun.sdk.io.EgressSpec; +import com.ververica.statefun.sdk.kafka.KafkaEgressSerializer; +import com.ververica.statefun.sdk.kafka.KafkaEgressSpec; +import java.util.Properties; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer.Semantic; +import org.apache.flink.streaming.connectors.kafka.KafkaSerializationSchema; + +public class KafkaSinkProvider implements SinkProvider { + + @Override + public SinkFunction forSpec(EgressSpec egressSpec) { + KafkaEgressSpec spec = asSpec(egressSpec); + + Properties properties = new Properties(); + properties.putAll(spec.properties()); + properties.put("bootstrap.servers", spec.kafkaAddress()); + + Semantic producerSemantic = semanticFromSpec(spec); + if (producerSemantic == Semantic.EXACTLY_ONCE) { + properties.put("transaction.timeout.ms", spec.transactionTimeoutDuration().toMillis()); + } + + return new FlinkKafkaProducer<>( + randomKafkaTopic(), + serializerFromSpec(spec), + properties, + producerSemantic, + spec.kafkaProducerPoolSize()); + } + + private KafkaSerializationSchema serializerFromSpec(KafkaEgressSpec spec) { + KafkaEgressSerializer serializer = ReflectionUtil.instantiate(spec.serializerClass()); + return new KafkaSerializationSchemaDelegate<>(serializer); + } + + private static Semantic semanticFromSpec(KafkaEgressSpec spec) { + switch (spec.semantic()) { + case EXACTLY_ONCE: + return Semantic.EXACTLY_ONCE; + case AT_LEAST_ONCE: + return Semantic.AT_LEAST_ONCE; + case NONE: + return Semantic.NONE; + default: + throw new IllegalArgumentException("Unknown producer semantic " + spec.semantic()); + } + } + + private static KafkaEgressSpec asSpec(EgressSpec spec) { + if (spec instanceof KafkaEgressSpec) { + return (KafkaEgressSpec) spec; + } + if (spec == null) { + throw new NullPointerException("Unable to translate a NULL spec"); + } + throw new IllegalArgumentException(String.format("Wrong type %s", spec.type())); + } + + private static String randomKafkaTopic() { + return "__stateful_functions_random_topic_" + + generateRandomAlphanumericString(ThreadLocalRandom.current(), 16); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-io-bundle/src/main/java/com/ververica/statefun/flink/io/kafka/KafkaSourceProvider.java b/stateful-functions-flink/stateful-functions-flink-io-bundle/src/main/java/com/ververica/statefun/flink/io/kafka/KafkaSourceProvider.java new file mode 100644 index 00000000..3d6576cb --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-io-bundle/src/main/java/com/ververica/statefun/flink/io/kafka/KafkaSourceProvider.java @@ -0,0 +1,58 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.io.kafka; + +import com.ververica.statefun.flink.io.common.ReflectionUtil; +import com.ververica.statefun.flink.io.spi.SourceProvider; +import com.ververica.statefun.sdk.io.IngressSpec; +import com.ververica.statefun.sdk.kafka.KafkaIngressDeserializer; +import com.ververica.statefun.sdk.kafka.KafkaIngressSpec; +import java.util.Properties; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer; +import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; + +public class KafkaSourceProvider implements SourceProvider { + + @Override + public SourceFunction forSpec(IngressSpec ingressSpec) { + KafkaIngressSpec spec = asKafkaSpec(ingressSpec); + + Properties properties = new Properties(); + properties.putAll(spec.properties()); + properties.put("bootstrap.servers", spec.kafkaAddress()); + + return new FlinkKafkaConsumer<>(spec.topics(), deserializationSchemaFromSpec(spec), properties); + } + + private static KafkaIngressSpec asKafkaSpec(IngressSpec ingressSpec) { + if (ingressSpec instanceof KafkaIngressSpec) { + return (KafkaIngressSpec) ingressSpec; + } + if (ingressSpec == null) { + throw new NullPointerException("Unable to translate a NULL spec"); + } + throw new IllegalArgumentException(String.format("Wrong type %s", ingressSpec.type())); + } + + private KafkaDeserializationSchema deserializationSchemaFromSpec( + KafkaIngressSpec spec) { + KafkaIngressDeserializer ingressDeserializer = + ReflectionUtil.instantiate(spec.deserializerClass()); + return new KafkaDeserializationSchemaDelegate<>(ingressDeserializer); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-io-bundle/src/main/resources/META-INF/services/com.ververica.statefun.flink.io.spi.FlinkIoModule b/stateful-functions-flink/stateful-functions-flink-io-bundle/src/main/resources/META-INF/services/com.ververica.statefun.flink.io.spi.FlinkIoModule new file mode 100644 index 00000000..aea42597 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-io-bundle/src/main/resources/META-INF/services/com.ververica.statefun.flink.io.spi.FlinkIoModule @@ -0,0 +1,17 @@ +# +# Copyright 2019 Ververica GmbH. +# +# 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. +# + +com.ververica.statefun.flink.io.kafka.KafkaFlinkIoModule \ No newline at end of file diff --git a/stateful-functions-flink/stateful-functions-flink-io-bundle/src/test/com/ververica/megastar/flink/io/common/ReflectionUtilTest.java b/stateful-functions-flink/stateful-functions-flink-io-bundle/src/test/com/ververica/megastar/flink/io/common/ReflectionUtilTest.java new file mode 100644 index 00000000..92ac8df5 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-io-bundle/src/test/com/ververica/megastar/flink/io/common/ReflectionUtilTest.java @@ -0,0 +1,45 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.io.common; + +import com.ververica.statefun.sdk.kafka.KafkaIngressDeserializer; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.hamcrest.CoreMatchers; +import org.junit.Test; + +import static org.hamcrest.MatcherAssert.assertThat; + +public class ReflectionUtilTest { + + private static final class Serializer implements KafkaIngressDeserializer { + + private static final long serialVersionUID = 1; + + @Override + public String deserialize(ConsumerRecord input) { + return null; + } + } + + @Test + public void example() { + Serializer serializer = ReflectionUtil.instantiate(Serializer.class); + + assertThat(serializer, CoreMatchers.notNullValue()); + } + +} \ No newline at end of file diff --git a/stateful-functions-flink/stateful-functions-flink-io/pom.xml b/stateful-functions-flink/stateful-functions-flink-io/pom.xml new file mode 100644 index 00000000..80e54200 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-io/pom.xml @@ -0,0 +1,45 @@ + + + + 4.0.0 + + + com.ververica + stateful-functions-flink + 1.0-SNAPSHOT + .. + + + stateful-functions-flink-io + + + + com.ververica + stateful-functions-sdk + ${project.version} + + + org.apache.flink + flink-streaming-java_2.11 + ${flink.version} + + + + + \ No newline at end of file diff --git a/stateful-functions-flink/stateful-functions-flink-io/src/main/java/com/ververica/statefun/flink/io/datastream/SinkFunctionSpec.java b/stateful-functions-flink/stateful-functions-flink-io/src/main/java/com/ververica/statefun/flink/io/datastream/SinkFunctionSpec.java new file mode 100644 index 00000000..3ef8aa0b --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-io/src/main/java/com/ververica/statefun/flink/io/datastream/SinkFunctionSpec.java @@ -0,0 +1,62 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.io.datastream; + +import com.ververica.statefun.sdk.EgressType; +import com.ververica.statefun.sdk.io.EgressIdentifier; +import com.ververica.statefun.sdk.io.EgressSpec; +import java.io.Serializable; +import java.util.Objects; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; + +/** + * An {@link EgressSpec} that can run any Apache Flink {@link SinkFunction}. + * + * @param The input type output by the sink. + */ +public final class SinkFunctionSpec implements EgressSpec, Serializable { + private static final long serialVersionUID = 1; + + static final EgressType TYPE = + new EgressType("com.ververica.statefun.flink.io", "sink-function-spec"); + + private final EgressIdentifier id; + private final SinkFunction delegate; + + /** + * @param id A unique egress identifier. + * @param delegate The underlying sink that the egress will delegate to at runtime. + */ + public SinkFunctionSpec(EgressIdentifier id, SinkFunction delegate) { + this.id = Objects.requireNonNull(id); + this.delegate = Objects.requireNonNull(delegate); + } + + @Override + public final EgressIdentifier id() { + return id; + } + + @Override + public final EgressType type() { + return TYPE; + } + + SinkFunction delegate() { + return delegate; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-io/src/main/java/com/ververica/statefun/flink/io/datastream/SourceFunctionSpec.java b/stateful-functions-flink/stateful-functions-flink-io/src/main/java/com/ververica/statefun/flink/io/datastream/SourceFunctionSpec.java new file mode 100644 index 00000000..5a157300 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-io/src/main/java/com/ververica/statefun/flink/io/datastream/SourceFunctionSpec.java @@ -0,0 +1,62 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.io.datastream; + +import com.ververica.statefun.sdk.IngressType; +import com.ververica.statefun.sdk.io.IngressIdentifier; +import com.ververica.statefun.sdk.io.IngressSpec; +import java.io.Serializable; +import java.util.Objects; +import org.apache.flink.streaming.api.functions.source.SourceFunction; + +/** + * An {@link IngressSpec} that can run any Apache Flink {@link SourceFunction}. + * + * @param The input type consumed by the source. + */ +public final class SourceFunctionSpec implements IngressSpec, Serializable { + private static final long serialVersionUID = 1; + + static final IngressType TYPE = + new IngressType("com.ververica.statefun.flink.io", "source-function-spec"); + + private final IngressIdentifier id; + private final SourceFunction delegate; + + /** + * @param id A unique ingress identifier. + * @param delegate The underlying source function that this spec will delegate to at runtime. + */ + public SourceFunctionSpec(IngressIdentifier id, SourceFunction delegate) { + this.id = Objects.requireNonNull(id); + this.delegate = Objects.requireNonNull(delegate); + } + + @Override + public final IngressIdentifier id() { + return id; + } + + @Override + public final IngressType type() { + return TYPE; + } + + SourceFunction delegate() { + return delegate; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-io/src/main/java/com/ververica/statefun/flink/io/datastream/SourceSinkModule.java b/stateful-functions-flink/stateful-functions-flink-io/src/main/java/com/ververica/statefun/flink/io/datastream/SourceSinkModule.java new file mode 100644 index 00000000..1885edfd --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-io/src/main/java/com/ververica/statefun/flink/io/datastream/SourceSinkModule.java @@ -0,0 +1,52 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.io.datastream; + +import com.ververica.statefun.flink.io.spi.FlinkIoModule; +import com.ververica.statefun.flink.io.spi.SinkProvider; +import com.ververica.statefun.flink.io.spi.SourceProvider; +import com.ververica.statefun.sdk.io.EgressSpec; +import com.ververica.statefun.sdk.io.IngressSpec; +import java.util.Map; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.api.functions.source.SourceFunction; + +public class SourceSinkModule implements FlinkIoModule { + + @Override + public void configure(Map globalConfiguration, Binder binder) { + SinkSourceProvider provider = new SinkSourceProvider(); + + binder.bindSourceProvider(SourceFunctionSpec.TYPE, provider); + binder.bindSinkProvider(SinkFunctionSpec.TYPE, provider); + } + + private static final class SinkSourceProvider implements SourceProvider, SinkProvider { + + @Override + public SourceFunction forSpec(IngressSpec spec) { + SourceFunctionSpec casted = (SourceFunctionSpec) spec; + return casted.delegate(); + } + + @Override + public SinkFunction forSpec(EgressSpec spec) { + SinkFunctionSpec casted = (SinkFunctionSpec) spec; + return casted.delegate(); + } + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-io/src/main/java/com/ververica/statefun/flink/io/spi/FlinkIoModule.java b/stateful-functions-flink/stateful-functions-flink-io/src/main/java/com/ververica/statefun/flink/io/spi/FlinkIoModule.java new file mode 100644 index 00000000..c5e4ff99 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-io/src/main/java/com/ververica/statefun/flink/io/spi/FlinkIoModule.java @@ -0,0 +1,33 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.io.spi; + +import com.ververica.statefun.sdk.EgressType; +import com.ververica.statefun.sdk.IngressType; +import java.util.Map; + +public interface FlinkIoModule { + + void configure(Map globalConfiguration, Binder binder); + + interface Binder { + + void bindSourceProvider(IngressType type, SourceProvider provider); + + void bindSinkProvider(EgressType type, SinkProvider provider); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-io/src/main/java/com/ververica/statefun/flink/io/spi/SinkProvider.java b/stateful-functions-flink/stateful-functions-flink-io/src/main/java/com/ververica/statefun/flink/io/spi/SinkProvider.java new file mode 100644 index 00000000..c993e501 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-io/src/main/java/com/ververica/statefun/flink/io/spi/SinkProvider.java @@ -0,0 +1,25 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.io.spi; + +import com.ververica.statefun.sdk.io.EgressSpec; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; + +public interface SinkProvider { + + SinkFunction forSpec(EgressSpec spec); +} diff --git a/stateful-functions-flink/stateful-functions-flink-io/src/main/java/com/ververica/statefun/flink/io/spi/SourceProvider.java b/stateful-functions-flink/stateful-functions-flink-io/src/main/java/com/ververica/statefun/flink/io/spi/SourceProvider.java new file mode 100644 index 00000000..832052d4 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-io/src/main/java/com/ververica/statefun/flink/io/spi/SourceProvider.java @@ -0,0 +1,25 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.io.spi; + +import com.ververica.statefun.sdk.io.IngressSpec; +import org.apache.flink.streaming.api.functions.source.SourceFunction; + +public interface SourceProvider { + + SourceFunction forSpec(IngressSpec spec); +} diff --git a/stateful-functions-flink/stateful-functions-flink-io/src/main/resources/META-INF/services/com.ververica.statefun.flink.io.spi.FlinkIoModule b/stateful-functions-flink/stateful-functions-flink-io/src/main/resources/META-INF/services/com.ververica.statefun.flink.io.spi.FlinkIoModule new file mode 100644 index 00000000..088603e2 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-io/src/main/resources/META-INF/services/com.ververica.statefun.flink.io.spi.FlinkIoModule @@ -0,0 +1,17 @@ +# +# Copyright 2019 Ververica GmbH. +# +# 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. +# + +com.ververica.statefun.flink.io.datastream.SourceSinkModule \ No newline at end of file diff --git a/stateful-functions-flink/stateful-functions-flink-launcher/pom.xml b/stateful-functions-flink/stateful-functions-flink-launcher/pom.xml new file mode 100644 index 00000000..8f94f9da --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-launcher/pom.xml @@ -0,0 +1,53 @@ + + + + 4.0.0 + + + com.ververica + stateful-functions-flink + 1.0-SNAPSHOT + .. + + + stateful-functions-flink-launcher + 1.0-SNAPSHOT + + + + + org.apache.flink + flink-runtime_${scala.binary.version} + ${flink.version} + + + org.apache.flink + flink-clients_${scala.binary.version} + ${flink.version} + + + + + com.ververica + stateful-functions-flink-core + ${project.version} + + + + \ No newline at end of file diff --git a/stateful-functions-flink/stateful-functions-flink-launcher/src/main/java/com/ververica/statefun/flink/launcher/Constants.java b/stateful-functions-flink/stateful-functions-flink-launcher/src/main/java/com/ververica/statefun/flink/launcher/Constants.java new file mode 100644 index 00000000..3216de3d --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-launcher/src/main/java/com/ververica/statefun/flink/launcher/Constants.java @@ -0,0 +1,23 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.launcher; + +public class Constants { + static final String MODULE_DIRECTORY = "/opt/stateful-functions/modules"; + static final String FLINK_JOB_JAR_PATH = "/opt/flink/lib/stateful-functions-flink-core.jar"; + static final String STATEFUL_FUNCTIONS_PACKAGE = "com.ververica.statefun."; +} diff --git a/stateful-functions-flink/stateful-functions-flink-launcher/src/main/java/com/ververica/statefun/flink/launcher/StatefulFunctionsClusterConfiguration.java b/stateful-functions-flink/stateful-functions-flink-launcher/src/main/java/com/ververica/statefun/flink/launcher/StatefulFunctionsClusterConfiguration.java new file mode 100644 index 00000000..0d04e273 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-launcher/src/main/java/com/ververica/statefun/flink/launcher/StatefulFunctionsClusterConfiguration.java @@ -0,0 +1,62 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.launcher; + +import static java.util.Objects.requireNonNull; + +import java.util.Properties; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.entrypoint.EntrypointClusterConfiguration; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; + +/** + * Configuration for the {@link StatefulFunctionsClusterEntryPoint}. + * + *

This class was copied from Apache Flink. + */ +final class StatefulFunctionsClusterConfiguration extends EntrypointClusterConfiguration { + + @Nonnull private final SavepointRestoreSettings savepointRestoreSettings; + + @Nullable private final JobID jobId; + + StatefulFunctionsClusterConfiguration( + @Nonnull String configDir, + @Nonnull Properties dynamicProperties, + @Nonnull String[] args, + @Nullable String hostname, + int restPort, + @Nonnull SavepointRestoreSettings savepointRestoreSettings, + @Nullable JobID jobId) { + super(configDir, dynamicProperties, args, hostname, restPort); + this.savepointRestoreSettings = + requireNonNull(savepointRestoreSettings, "savepointRestoreSettings"); + this.jobId = jobId; + } + + @Nonnull + SavepointRestoreSettings getSavepointRestoreSettings() { + return savepointRestoreSettings; + } + + @Nullable + JobID getJobId() { + return jobId; + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-launcher/src/main/java/com/ververica/statefun/flink/launcher/StatefulFunctionsClusterConfigurationParserFactory.java b/stateful-functions-flink/stateful-functions-flink-launcher/src/main/java/com/ververica/statefun/flink/launcher/StatefulFunctionsClusterConfigurationParserFactory.java new file mode 100644 index 00000000..097da24e --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-launcher/src/main/java/com/ververica/statefun/flink/launcher/StatefulFunctionsClusterConfigurationParserFactory.java @@ -0,0 +1,115 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.launcher; + +import static org.apache.flink.runtime.entrypoint.parser.CommandLineOptions.CONFIG_DIR_OPTION; +import static org.apache.flink.runtime.entrypoint.parser.CommandLineOptions.DYNAMIC_PROPERTY_OPTION; +import static org.apache.flink.runtime.entrypoint.parser.CommandLineOptions.HOST_OPTION; +import static org.apache.flink.runtime.entrypoint.parser.CommandLineOptions.REST_PORT_OPTION; + +import java.util.Properties; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.Options; +import org.apache.flink.api.common.JobID; +import org.apache.flink.client.cli.CliFrontendParser; +import org.apache.flink.runtime.entrypoint.FlinkParseException; +import org.apache.flink.runtime.entrypoint.parser.ParserResultFactory; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; + +/** + * Parser factory which generates a {@link StatefulFunctionsClusterConfiguration} from a given list + * of command line arguments. + * + *

This class was copied from Apache Flink. + */ +public class StatefulFunctionsClusterConfigurationParserFactory + implements ParserResultFactory { + + private static final Option JOB_ID_OPTION = + Option.builder("jid") + .longOpt("job-id") + .required(false) + .hasArg(true) + .argName("job id") + .desc("Job ID of the job to run.") + .build(); + + @Nullable + private static JobID getJobId(CommandLine commandLine) throws FlinkParseException { + String jobId = commandLine.getOptionValue(JOB_ID_OPTION.getOpt()); + if (jobId == null) { + return null; + } + try { + return JobID.fromHexString(jobId); + } catch (IllegalArgumentException e) { + throw createFlinkParseException(JOB_ID_OPTION, e); + } + } + + private static FlinkParseException createFlinkParseException(Option option, Exception cause) { + return new FlinkParseException( + String.format("Failed to parse '--%s' option", option.getLongOpt()), cause); + } + + @Override + public Options getOptions() { + final Options options = new Options(); + options.addOption(CONFIG_DIR_OPTION); + options.addOption(REST_PORT_OPTION); + options.addOption(JOB_ID_OPTION); + options.addOption(DYNAMIC_PROPERTY_OPTION); + options.addOption(CliFrontendParser.SAVEPOINT_PATH_OPTION); + options.addOption(CliFrontendParser.SAVEPOINT_ALLOW_NON_RESTORED_OPTION); + + return options; + } + + @Override + public StatefulFunctionsClusterConfiguration createResult(@Nonnull CommandLine commandLine) + throws FlinkParseException { + final String configDir = commandLine.getOptionValue(CONFIG_DIR_OPTION.getOpt()); + final Properties dynamicProperties = + commandLine.getOptionProperties(DYNAMIC_PROPERTY_OPTION.getOpt()); + final int restPort = getRestPort(commandLine); + final String hostname = commandLine.getOptionValue(HOST_OPTION.getOpt()); + final SavepointRestoreSettings savepointRestoreSettings = + CliFrontendParser.createSavepointRestoreSettings(commandLine); + final JobID jobId = getJobId(commandLine); + + return new StatefulFunctionsClusterConfiguration( + configDir, + dynamicProperties, + commandLine.getArgs(), + hostname, + restPort, + savepointRestoreSettings, + jobId); + } + + private int getRestPort(CommandLine commandLine) throws FlinkParseException { + final String restPortString = commandLine.getOptionValue(REST_PORT_OPTION.getOpt(), "-1"); + try { + return Integer.parseInt(restPortString); + } catch (NumberFormatException e) { + throw createFlinkParseException(REST_PORT_OPTION, e); + } + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-launcher/src/main/java/com/ververica/statefun/flink/launcher/StatefulFunctionsClusterEntryPoint.java b/stateful-functions-flink/stateful-functions-flink-launcher/src/main/java/com/ververica/statefun/flink/launcher/StatefulFunctionsClusterEntryPoint.java new file mode 100644 index 00000000..1ab077d9 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-launcher/src/main/java/com/ververica/statefun/flink/launcher/StatefulFunctionsClusterEntryPoint.java @@ -0,0 +1,144 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.launcher; + +import static java.util.Objects.requireNonNull; + +import java.util.Optional; +import javax.annotation.Nonnull; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; +import org.apache.flink.runtime.entrypoint.JobClusterEntrypoint; +import org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponentFactory; +import org.apache.flink.runtime.entrypoint.component.JobDispatcherResourceManagerComponentFactory; +import org.apache.flink.runtime.entrypoint.parser.CommandLineParser; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; +import org.apache.flink.runtime.resourcemanager.StandaloneResourceManagerFactory; +import org.apache.flink.runtime.util.EnvironmentInformation; +import org.apache.flink.runtime.util.JvmShutdownSafeguard; +import org.apache.flink.runtime.util.SignalHandler; + +/** {@link JobClusterEntrypoint} which is started with a job in a predefined location. */ +public final class StatefulFunctionsClusterEntryPoint extends JobClusterEntrypoint { + + public static final JobID ZERO_JOB_ID = new JobID(0, 0); + + @Nonnull private final JobID jobId; + + @Nonnull private final SavepointRestoreSettings savepointRestoreSettings; + + @Nonnull private final String[] programArguments; + + private StatefulFunctionsClusterEntryPoint( + Configuration configuration, + @Nonnull JobID jobId, + @Nonnull SavepointRestoreSettings savepointRestoreSettings, + @Nonnull String[] programArguments) { + super(configuration); + this.jobId = requireNonNull(jobId, "jobId"); + this.savepointRestoreSettings = + requireNonNull(savepointRestoreSettings, "savepointRestoreSettings"); + this.programArguments = requireNonNull(programArguments, "programArguments"); + } + + public static void main(String[] args) { + EnvironmentInformation.logEnvironmentInfo( + LOG, StatefulFunctionsClusterEntryPoint.class.getSimpleName(), args); + SignalHandler.register(LOG); + JvmShutdownSafeguard.installAsShutdownHook(LOG); + + final CommandLineParser commandLineParser = + new CommandLineParser<>(new StatefulFunctionsClusterConfigurationParserFactory()); + StatefulFunctionsClusterConfiguration clusterConfiguration = null; + + try { + clusterConfiguration = commandLineParser.parse(args); + } catch (Exception e) { + LOG.error("Could not parse command line arguments {}.", args, e); + commandLineParser.printHelp(StatefulFunctionsClusterEntryPoint.class.getSimpleName()); + System.exit(1); + } + + Configuration configuration = loadConfiguration(clusterConfiguration); + addStatefulFunctionsConfiguration(configuration); + setDefaultExecutionModeIfNotConfigured(configuration); + + StatefulFunctionsClusterEntryPoint entrypoint = + new StatefulFunctionsClusterEntryPoint( + configuration, + resolveJobIdForCluster( + Optional.ofNullable(clusterConfiguration.getJobId()), configuration), + clusterConfiguration.getSavepointRestoreSettings(), + clusterConfiguration.getArgs()); + + ClusterEntrypoint.runClusterEntrypoint(entrypoint); + } + + @VisibleForTesting + @Nonnull + static JobID resolveJobIdForCluster(Optional optionalJobID, Configuration configuration) { + return optionalJobID.orElseGet(() -> createJobIdForCluster(configuration)); + } + + @Nonnull + private static JobID createJobIdForCluster(Configuration globalConfiguration) { + if (HighAvailabilityMode.isHighAvailabilityModeActivated(globalConfiguration)) { + return ZERO_JOB_ID; + } else { + return JobID.generate(); + } + } + + @VisibleForTesting + static void setDefaultExecutionModeIfNotConfigured(Configuration configuration) { + if (isNoExecutionModeConfigured(configuration)) { + // In contrast to other places, the default for standalone job clusters is + // ExecutionMode.DETACHED + configuration.setString(ClusterEntrypoint.EXECUTION_MODE, ExecutionMode.DETACHED.toString()); + } + } + + private static boolean isNoExecutionModeConfigured(Configuration configuration) { + return configuration.getString(ClusterEntrypoint.EXECUTION_MODE, null) == null; + } + + private static void addStatefulFunctionsConfiguration(Configuration configuration) { + String parentFirst = + configuration.getString(CoreOptions.ALWAYS_PARENT_FIRST_LOADER_PATTERNS_ADDITIONAL, ""); + if (parentFirst.isEmpty()) { + parentFirst = Constants.STATEFUL_FUNCTIONS_PACKAGE; + } else if (parentFirst.endsWith(";")) { + parentFirst = parentFirst + Constants.STATEFUL_FUNCTIONS_PACKAGE; + } else { + parentFirst = parentFirst + ";" + Constants.STATEFUL_FUNCTIONS_PACKAGE; + } + configuration.setString( + CoreOptions.ALWAYS_PARENT_FIRST_LOADER_PATTERNS_ADDITIONAL, parentFirst); + } + + @Override + protected DispatcherResourceManagerComponentFactory + createDispatcherResourceManagerComponentFactory(Configuration configuration) { + return new JobDispatcherResourceManagerComponentFactory( + StandaloneResourceManagerFactory.INSTANCE, + new StatefulFunctionsJobGraphRetriever(jobId, savepointRestoreSettings, programArguments)); + } +} diff --git a/stateful-functions-flink/stateful-functions-flink-launcher/src/main/java/com/ververica/statefun/flink/launcher/StatefulFunctionsJobGraphRetriever.java b/stateful-functions-flink/stateful-functions-flink-launcher/src/main/java/com/ververica/statefun/flink/launcher/StatefulFunctionsJobGraphRetriever.java new file mode 100644 index 00000000..a548a2f5 --- /dev/null +++ b/stateful-functions-flink/stateful-functions-flink-launcher/src/main/java/com/ververica/statefun/flink/launcher/StatefulFunctionsJobGraphRetriever.java @@ -0,0 +1,119 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.flink.launcher; + +import static java.util.Objects.requireNonNull; + +import com.ververica.statefun.flink.core.ModuleSpecs; +import com.ververica.statefun.flink.core.ModuleSpecs.ModuleSpec; +import com.ververica.statefun.flink.core.StatefulFunctionsJob; +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.net.URL; +import java.util.ArrayList; +import java.util.List; +import org.apache.flink.api.common.JobID; +import org.apache.flink.client.program.PackagedProgram; +import org.apache.flink.client.program.PackagedProgramUtils; +import org.apache.flink.client.program.ProgramInvocationException; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.runtime.entrypoint.component.JobGraphRetriever; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.flink.util.FlinkException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * {@link JobGraphRetriever} which creates the {@link JobGraph} from a class on the class path. + * + *

This class is based on a class present in Apache Flink but it sets the correct class path for + * the child first classloader. + */ +final class StatefulFunctionsJobGraphRetriever implements JobGraphRetriever { + + private static final Logger LOG = + LoggerFactory.getLogger(StatefulFunctionsJobGraphRetriever.class); + + private final JobID jobId; + private final SavepointRestoreSettings savepointRestoreSettings; + private final String[] programArguments; + + StatefulFunctionsJobGraphRetriever( + JobID jobId, SavepointRestoreSettings savepointRestoreSettings, String[] programArguments) { + this.jobId = requireNonNull(jobId, "jobId"); + this.savepointRestoreSettings = + requireNonNull(savepointRestoreSettings, "savepointRestoreSettings"); + this.programArguments = requireNonNull(programArguments, "programArguments"); + } + + private static List obtainModuleAdditionalClassPath() { + try { + ModuleSpecs specs = ModuleSpecs.fromPath(Constants.MODULE_DIRECTORY); + List classPath = new ArrayList<>(); + for (ModuleSpec spec : specs) { + for (URI uri : spec.artifactUris()) { + classPath.add(uri.toURL()); + } + } + LOG.info( + "Found {} additional module jars in {} modules", + classPath.size(), + specs.modules().size()); + return classPath; + } catch (IOException e) { + throw new RuntimeException( + "Unable to load modules from path " + Constants.MODULE_DIRECTORY, e); + } + } + + @Override + public JobGraph retrieveJobGraph(Configuration configuration) throws FlinkException { + final PackagedProgram packagedProgram = createPackagedProgram(); + + final int defaultParallelism = configuration.getInteger(CoreOptions.DEFAULT_PARALLELISM); + try { + final JobGraph jobGraph = + PackagedProgramUtils.createJobGraph( + packagedProgram, configuration, defaultParallelism, jobId); + jobGraph.setAllowQueuedScheduling(true); + jobGraph.setSavepointRestoreSettings(savepointRestoreSettings); + + return jobGraph; + } catch (Exception e) { + throw new FlinkException("Could not create the JobGraph from the provided user code jar.", e); + } + } + + private PackagedProgram createPackagedProgram() { + File mainJar = new File(Constants.FLINK_JOB_JAR_PATH); + if (!mainJar.exists()) { + throw new IllegalStateException("Unable to locate the launcher jar"); + } + try { + return new PackagedProgram( + mainJar, + obtainModuleAdditionalClassPath(), + StatefulFunctionsJob.class.getName(), + programArguments); + } catch (ProgramInvocationException e) { + throw new RuntimeException("Unable to construct a packaged program", e); + } + } +} diff --git a/stateful-functions-kafka-io/pom.xml b/stateful-functions-kafka-io/pom.xml new file mode 100644 index 00000000..a5fa38c6 --- /dev/null +++ b/stateful-functions-kafka-io/pom.xml @@ -0,0 +1,59 @@ + + + + + stateful-functions-parent + com.ververica + 1.0-SNAPSHOT + .. + + 4.0.0 + + + 2.2.0 + + + stateful-functions-kafka-io + + + com.ververica + stateful-functions-sdk + ${project.version} + + + + + org.apache.kafka + kafka-clients + ${kafka.version} + compile + + + org.xerial.snappy + snappy-java + + + + + + diff --git a/stateful-functions-kafka-io/src/main/java/com/ververica/statefun/sdk/kafka/Constants.java b/stateful-functions-kafka-io/src/main/java/com/ververica/statefun/sdk/kafka/Constants.java new file mode 100644 index 00000000..2810b40d --- /dev/null +++ b/stateful-functions-kafka-io/src/main/java/com/ververica/statefun/sdk/kafka/Constants.java @@ -0,0 +1,29 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.sdk.kafka; + +import com.ververica.statefun.sdk.EgressType; +import com.ververica.statefun.sdk.IngressType; + +public final class Constants { + public static final IngressType KAFKA_INGRESS_TYPE = + new IngressType("com.ververica.statefun.sdk.kafka", "universal-kafka-connector"); + public static final EgressType KAFKA_EGRESS_TYPE = + new EgressType("com.ververica.statefun.sdk.kafka", "universal-kafka-connector"); + + private Constants() {} +} diff --git a/stateful-functions-kafka-io/src/main/java/com/ververica/statefun/sdk/kafka/KafkaEgressBuilder.java b/stateful-functions-kafka-io/src/main/java/com/ververica/statefun/sdk/kafka/KafkaEgressBuilder.java new file mode 100644 index 00000000..2c41a6b9 --- /dev/null +++ b/stateful-functions-kafka-io/src/main/java/com/ververica/statefun/sdk/kafka/KafkaEgressBuilder.java @@ -0,0 +1,144 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.sdk.kafka; + +import com.ververica.statefun.sdk.io.EgressIdentifier; +import com.ververica.statefun.sdk.io.EgressSpec; +import java.time.Duration; +import java.util.Objects; +import java.util.Properties; + +/** + * A builder class for creating an {@link EgressSpec} that writes data out to a Kafka cluster. By + * default the egress will use {@link #withAtLeastOnceProducerSemantics()}. + * + * @param The type written out to the cluster by the Egress. + */ +public final class KafkaEgressBuilder { + private final EgressIdentifier id; + private Class> serializer; + private String kafkaAddress; + private Properties properties = new Properties(); + private int kafkaProducerPoolSize = 5; + private KafkaProducerSemantic semantic = KafkaProducerSemantic.AT_LEAST_ONCE; + private Duration transactionTimeoutDuration = Duration.ZERO; + + private KafkaEgressBuilder(EgressIdentifier id) { + this.id = Objects.requireNonNull(id); + } + + /** + * @param egressIdentifier A unique egress identifier. + * @param The type the egress will output. + * @return A {@link KafkaIngressBuilder}. + */ + public static KafkaEgressBuilder forIdentifier( + EgressIdentifier egressIdentifier) { + return new KafkaEgressBuilder<>(egressIdentifier); + } + + /** @param kafkaAddress Comma separated addresses of the brokers. */ + public KafkaEgressBuilder withKafkaAddress(String kafkaAddress) { + this.kafkaAddress = Objects.requireNonNull(kafkaAddress); + return this; + } + + /** A configuration property for the KafkaProducer. */ + public KafkaEgressBuilder withProperty(String key, Object value) { + Objects.requireNonNull(key); + Objects.requireNonNull(value); + properties.put(key, value); + return this; + } + + /** Configuration properties for the KafkaProducer. */ + public KafkaEgressBuilder withProperties(Properties properties) { + Objects.requireNonNull(properties); + this.properties.putAll(properties); + return this; + } + + /** + * @param serializer A serializer schema for turning user objects into a kafka-consumable byte[] + * supporting key/value messages. + */ + public KafkaEgressBuilder withSerializer( + Class> serializer) { + this.serializer = Objects.requireNonNull(serializer); + return this; + } + + /** @param poolSize Overwrite default KafkaProducers pool size. The default is 5. */ + public KafkaEgressBuilder withKafkaProducerPoolSize(int poolSize) { + this.kafkaProducerPoolSize = poolSize; + return this; + } + + /** + * KafkaProducerSemantic.EXACTLY_ONCE the egress will write all messages in a Kafka transaction + * that will be committed to Kafka on a checkpoint. + * + *

With exactly-once producer semantics, users must also specify the transaction timeout. Note + * that this value must not be larger than the {@code transaction.max.timeout.ms} value configured + * on Kafka brokers (by default, this is 15 minutes). + * + * @param transactionTimeoutDuration the transaction timeout. + */ + public KafkaEgressBuilder withExactlyOnceProducerSemantics( + Duration transactionTimeoutDuration) { + Objects.requireNonNull( + transactionTimeoutDuration, "a transaction timeout duration must be provided."); + if (transactionTimeoutDuration == Duration.ZERO) { + throw new IllegalArgumentException( + "Transaction timeout durations must be larger than 0 when using exactly-once producer semantics."); + } + + this.semantic = KafkaProducerSemantic.EXACTLY_ONCE; + this.transactionTimeoutDuration = transactionTimeoutDuration; + return this; + } + + /** + * KafkaProducerSemantic.AT_LEAST_ONCE the egress will wait for all outstanding messages in the + * Kafka buffers to be acknowledged by the Kafka producer on a checkpoint. + */ + public KafkaEgressBuilder withAtLeastOnceProducerSemantics() { + this.semantic = KafkaProducerSemantic.AT_LEAST_ONCE; + return this; + } + + /** + * KafkaProducerSemantic.NONE means that nothing will be guaranteed. Messages can be lost and/or + * duplicated in case of failure. + */ + public KafkaEgressBuilder withNoProducerSemantics() { + this.semantic = KafkaProducerSemantic.NONE; + return this; + } + + /** @return An {@link EgressSpec} that can be used in a {@code StatefulFunctionModule}. */ + public EgressSpec build() { + return new KafkaEgressSpec<>( + id, + serializer, + kafkaAddress, + properties, + kafkaProducerPoolSize, + semantic, + transactionTimeoutDuration); + } +} diff --git a/stateful-functions-kafka-io/src/main/java/com/ververica/statefun/sdk/kafka/KafkaEgressSerializer.java b/stateful-functions-kafka-io/src/main/java/com/ververica/statefun/sdk/kafka/KafkaEgressSerializer.java new file mode 100644 index 00000000..78402397 --- /dev/null +++ b/stateful-functions-kafka-io/src/main/java/com/ververica/statefun/sdk/kafka/KafkaEgressSerializer.java @@ -0,0 +1,37 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.sdk.kafka; + +import java.io.Serializable; +import org.apache.kafka.clients.producer.ProducerRecord; + +/** + * A {@link KafkaEgressSerializer} defines how to serialize values of type {@code T} into {@link + * ProducerRecord ProducerRecords}. + * + * @param the type of values being serialized + */ +public interface KafkaEgressSerializer extends Serializable { + + /** + * Serializes given element and returns it as a {@link ProducerRecord}. + * + * @param t element to be serialized + * @return Kafka {@link ProducerRecord} + */ + ProducerRecord serialize(OutT t); +} diff --git a/stateful-functions-kafka-io/src/main/java/com/ververica/statefun/sdk/kafka/KafkaEgressSpec.java b/stateful-functions-kafka-io/src/main/java/com/ververica/statefun/sdk/kafka/KafkaEgressSpec.java new file mode 100644 index 00000000..6b58ac15 --- /dev/null +++ b/stateful-functions-kafka-io/src/main/java/com/ververica/statefun/sdk/kafka/KafkaEgressSpec.java @@ -0,0 +1,85 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.sdk.kafka; + +import com.ververica.statefun.sdk.EgressType; +import com.ververica.statefun.sdk.io.EgressIdentifier; +import com.ververica.statefun.sdk.io.EgressSpec; +import java.time.Duration; +import java.util.Objects; +import java.util.Properties; + +public final class KafkaEgressSpec implements EgressSpec { + private final Class> serializer; + private final String kafkaAddress; + private final Properties properties; + private final EgressIdentifier id; + private final int kafkaProducerPoolSize; + private final KafkaProducerSemantic semantic; + private final Duration transactionTimeoutDuration; + + KafkaEgressSpec( + EgressIdentifier id, + Class> serializer, + String kafkaAddress, + Properties properties, + int kafkaProducerPoolSize, + KafkaProducerSemantic semantic, + Duration transactionTimeoutDuration) { + this.serializer = Objects.requireNonNull(serializer); + this.kafkaAddress = Objects.requireNonNull(kafkaAddress); + this.properties = Objects.requireNonNull(properties); + this.id = Objects.requireNonNull(id); + this.kafkaProducerPoolSize = kafkaProducerPoolSize; + this.semantic = Objects.requireNonNull(semantic); + this.transactionTimeoutDuration = Objects.requireNonNull(transactionTimeoutDuration); + } + + @Override + public EgressIdentifier id() { + return id; + } + + @Override + public EgressType type() { + return Constants.KAFKA_EGRESS_TYPE; + } + + public Class> serializerClass() { + return serializer; + } + + public String kafkaAddress() { + return kafkaAddress; + } + + public Properties properties() { + return properties; + } + + public int kafkaProducerPoolSize() { + return kafkaProducerPoolSize; + } + + public KafkaProducerSemantic semantic() { + return semantic; + } + + public Duration transactionTimeoutDuration() { + return transactionTimeoutDuration; + } +} diff --git a/stateful-functions-kafka-io/src/main/java/com/ververica/statefun/sdk/kafka/KafkaIngressBuilder.java b/stateful-functions-kafka-io/src/main/java/com/ververica/statefun/sdk/kafka/KafkaIngressBuilder.java new file mode 100644 index 00000000..6a881bb7 --- /dev/null +++ b/stateful-functions-kafka-io/src/main/java/com/ververica/statefun/sdk/kafka/KafkaIngressBuilder.java @@ -0,0 +1,96 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.sdk.kafka; + +import com.ververica.statefun.sdk.io.IngressIdentifier; +import com.ververica.statefun.sdk.io.IngressSpec; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.Properties; + +/** + * A builder for creating an {@link IngressSpec} for consuming data from Apache Kafka. + * + * @param The type consumed from Kafka. + */ +public final class KafkaIngressBuilder { + + private final IngressIdentifier id; + private final List topics = new ArrayList<>(); + private final Properties properties = new Properties(); + private Class> deserializerClass; + private String kafkaAddress; + + private KafkaIngressBuilder(IngressIdentifier id) { + this.id = Objects.requireNonNull(id); + } + + /** + * @param id A unique ingress identifier. + * @param The type consumed from Kafka. + * @return A new {@link KafkaIngressBuilder}. + */ + public static KafkaIngressBuilder forIdentifier(IngressIdentifier id) { + return new KafkaIngressBuilder<>(id); + } + + /** @param kafkaAddress Comma separated addresses of the brokers. */ + public KafkaIngressBuilder withKafkaAddress(String kafkaAddress) { + this.kafkaAddress = Objects.requireNonNull(kafkaAddress); + return this; + } + + /** @param topic The name of the topic that should be consumed. */ + public KafkaIngressBuilder withTopic(String topic) { + topics.add(topic); + return this; + } + + /** @param topics A list of topics that should be consumed. */ + public KafkaIngressBuilder addTopics(List topics) { + this.topics.addAll(topics); + return this; + } + + /** A configuration property for the KafkaConsumer. */ + public KafkaIngressBuilder withProperties(Properties properties) { + this.properties.putAll(properties); + return this; + } + + /** A configuration property for the KafkaProducer. */ + public KafkaIngressBuilder withProperty(String name, String value) { + this.properties.put(name, value); + return this; + } + + /** + * @param deserializerClass The deserializer used to convert between Kafka's byte messages and + * java objects. + */ + public KafkaIngressBuilder withDeserializer( + Class> deserializerClass) { + this.deserializerClass = Objects.requireNonNull(deserializerClass); + return this; + } + + /** @return A new {@link IngressSpec}. */ + public IngressSpec build() { + return new KafkaIngressSpec<>(id, kafkaAddress, properties, topics, deserializerClass); + } +} diff --git a/stateful-functions-kafka-io/src/main/java/com/ververica/statefun/sdk/kafka/KafkaIngressDeserializer.java b/stateful-functions-kafka-io/src/main/java/com/ververica/statefun/sdk/kafka/KafkaIngressDeserializer.java new file mode 100644 index 00000000..49220288 --- /dev/null +++ b/stateful-functions-kafka-io/src/main/java/com/ververica/statefun/sdk/kafka/KafkaIngressDeserializer.java @@ -0,0 +1,37 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.sdk.kafka; + +import java.io.Serializable; +import org.apache.kafka.clients.consumer.ConsumerRecord; + +/** + * The deserialization schema describes how to turn the Kafka ConsumerRecords into data types that + * are processed by the system. + * + * @param The type created by the keyed deserialization schema. + */ +public interface KafkaIngressDeserializer extends Serializable { + + /** + * Deserializes the Kafka record. + * + * @param input Kafka record to be deserialized. + * @return The deserialized message as an object (null if the message cannot be deserialized). + */ + T deserialize(ConsumerRecord input); +} diff --git a/stateful-functions-kafka-io/src/main/java/com/ververica/statefun/sdk/kafka/KafkaIngressSpec.java b/stateful-functions-kafka-io/src/main/java/com/ververica/statefun/sdk/kafka/KafkaIngressSpec.java new file mode 100644 index 00000000..842c983e --- /dev/null +++ b/stateful-functions-kafka-io/src/main/java/com/ververica/statefun/sdk/kafka/KafkaIngressSpec.java @@ -0,0 +1,71 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.sdk.kafka; + +import com.ververica.statefun.sdk.IngressType; +import com.ververica.statefun.sdk.io.IngressIdentifier; +import com.ververica.statefun.sdk.io.IngressSpec; +import java.util.List; +import java.util.Objects; +import java.util.Properties; + +public class KafkaIngressSpec implements IngressSpec { + private final String kafkaAddress; + private final Properties properties; + private final List topics; + private final Class> deserializerClass; + private final IngressIdentifier ingressIdentifier; + + KafkaIngressSpec( + IngressIdentifier id, + String kafkaAddress, + Properties properties, + List topics, + Class> deserializerClass) { + this.kafkaAddress = Objects.requireNonNull(kafkaAddress); + this.properties = Objects.requireNonNull(properties); + this.topics = Objects.requireNonNull(topics); + this.deserializerClass = Objects.requireNonNull(deserializerClass); + this.ingressIdentifier = Objects.requireNonNull(id); + } + + @Override + public IngressIdentifier id() { + return ingressIdentifier; + } + + @Override + public IngressType type() { + return Constants.KAFKA_INGRESS_TYPE; + } + + public String kafkaAddress() { + return kafkaAddress; + } + + public Properties properties() { + return properties; + } + + public List topics() { + return topics; + } + + public Class> deserializerClass() { + return deserializerClass; + } +} diff --git a/stateful-functions-kafka-io/src/main/java/com/ververica/statefun/sdk/kafka/KafkaProducerSemantic.java b/stateful-functions-kafka-io/src/main/java/com/ververica/statefun/sdk/kafka/KafkaProducerSemantic.java new file mode 100644 index 00000000..ea55be03 --- /dev/null +++ b/stateful-functions-kafka-io/src/main/java/com/ververica/statefun/sdk/kafka/KafkaProducerSemantic.java @@ -0,0 +1,23 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.sdk.kafka; + +public enum KafkaProducerSemantic { + EXACTLY_ONCE, + AT_LEAST_ONCE, + NONE +} diff --git a/stateful-functions-quickstart/pom.xml b/stateful-functions-quickstart/pom.xml new file mode 100644 index 00000000..c7c77f00 --- /dev/null +++ b/stateful-functions-quickstart/pom.xml @@ -0,0 +1,62 @@ + + + + + stateful-functions-parent + com.ververica + 1.0-SNAPSHOT + + 4.0.0 + + stateful-functions-quickstart + maven-archetype + + + + + org.apache.maven.archetype + archetype-packaging + 3.1.2 + + + + + + + org.apache.maven.plugins + maven-archetype-plugin + 3.1.2 + + + + + + + org.apache.maven.plugins + maven-resources-plugin + + false + + @ + + + + + + \ No newline at end of file diff --git a/stateful-functions-quickstart/src/main/resources/META-INF/maven/archetype-metadata.xml b/stateful-functions-quickstart/src/main/resources/META-INF/maven/archetype-metadata.xml new file mode 100644 index 00000000..3af97ac5 --- /dev/null +++ b/stateful-functions-quickstart/src/main/resources/META-INF/maven/archetype-metadata.xml @@ -0,0 +1,42 @@ + + + + + + src/main/java + + **/*.java + + + + src/main/resources + + + + + Dockerfile + + + + \ No newline at end of file diff --git a/stateful-functions-quickstart/src/main/resources/archetype-resources/Dockerfile b/stateful-functions-quickstart/src/main/resources/archetype-resources/Dockerfile new file mode 100644 index 00000000..cf5a43b1 --- /dev/null +++ b/stateful-functions-quickstart/src/main/resources/archetype-resources/Dockerfile @@ -0,0 +1,20 @@ +# +# Copyright 2019 Ververica GmbH. +# +# 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. +# + +FROM stateful-functions + +RUN mkdir -p /opt/stateful-functions/modules/${artifactId} +COPY target/${artifactId}*jar /opt/stateful-functions/modules/${artifactId}/ \ No newline at end of file diff --git a/stateful-functions-quickstart/src/main/resources/archetype-resources/pom.xml b/stateful-functions-quickstart/src/main/resources/archetype-resources/pom.xml new file mode 100644 index 00000000..a7ba3168 --- /dev/null +++ b/stateful-functions-quickstart/src/main/resources/archetype-resources/pom.xml @@ -0,0 +1,57 @@ + + + + + ${groupId} + ${artifactId} + ${version} + jar + + Stateful Functions Quickstart + + + UTF-8 + @project.version@ + 1.8 + 2.11 + ${java.version} + ${java.version} + + + + + + com.ververica + stateful-functions-sdk + ${stateful-functions.version} + provided + + + + + + + diff --git a/stateful-functions-quickstart/src/main/resources/archetype-resources/src/main/java/Module.java b/stateful-functions-quickstart/src/main/resources/archetype-resources/src/main/java/Module.java new file mode 100644 index 00000000..7a0d3d85 --- /dev/null +++ b/stateful-functions-quickstart/src/main/resources/archetype-resources/src/main/java/Module.java @@ -0,0 +1,29 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package ${package}; + +import com.ververica.statefun.sdk.spi.StatefulFunctionModule; + +import java.util.Map; + +public final class Module implements StatefulFunctionModule { + + @Override + public void configure(Map globalConfiguration, Binder binder) { + + } +} \ No newline at end of file diff --git a/stateful-functions-quickstart/src/main/resources/archetype-resources/src/main/resources/META-INF/services/com.ververica.statefun.sdk.spi.StatefulFunctionModule b/stateful-functions-quickstart/src/main/resources/archetype-resources/src/main/resources/META-INF/services/com.ververica.statefun.sdk.spi.StatefulFunctionModule new file mode 100644 index 00000000..e44bd6cb --- /dev/null +++ b/stateful-functions-quickstart/src/main/resources/archetype-resources/src/main/resources/META-INF/services/com.ververica.statefun.sdk.spi.StatefulFunctionModule @@ -0,0 +1,17 @@ +# +# Copyright 2019 Ververica GmbH. +# +# 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. +# + +${package}.Module \ No newline at end of file diff --git a/stateful-functions-sdk/pom.xml b/stateful-functions-sdk/pom.xml new file mode 100644 index 00000000..60aadf97 --- /dev/null +++ b/stateful-functions-sdk/pom.xml @@ -0,0 +1,41 @@ + + + + + stateful-functions-parent + com.ververica + 1.0-SNAPSHOT + + 4.0.0 + + stateful-functions-sdk + + + 3.0.2 + + + + + com.google.code.findbugs + jsr305 + ${jsr305.version} + + + + diff --git a/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/Address.java b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/Address.java new file mode 100644 index 00000000..249271c4 --- /dev/null +++ b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/Address.java @@ -0,0 +1,84 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.sdk; + +import java.util.Objects; + +/** + * An {@link Address} is the unique identity of an individual {@link StatefulFunction}, containing + * of the function's {@link FunctionType} and an unique identifier within the type. The function's + * type denotes the class of function to invoke, while the unique identifier addresses the + * invocation to a specific function instance. + */ +public final class Address { + private final FunctionType type; + private final String id; + + /** + * Creates an {@link Address}. + * + * @param type type of the function. + * @param id unique id within the function type. + */ + public Address(FunctionType type, String id) { + this.type = Objects.requireNonNull(type); + this.id = Objects.requireNonNull(id); + } + + /** + * Returns the {@link FunctionType} that this address identifies. + * + * @return type of the function + */ + public FunctionType type() { + return type; + } + + /** + * Returns the unique function id, within its type, that this address identifies. + * + * @return unique id within the function type. + */ + public String id() { + return id; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Address address = (Address) o; + return type.equals(address.type) && id.equals(address.id); + } + + @Override + public int hashCode() { + int hash = 0; + hash = 37 * hash + type.hashCode(); + hash = 37 * hash + id.hashCode(); + return hash; + } + + @Override + public String toString() { + return String.format("Address(%s, %s, %s)", type.namespace(), type.name(), id); + } +} diff --git a/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/AsyncOperationResult.java b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/AsyncOperationResult.java new file mode 100644 index 00000000..4b075697 --- /dev/null +++ b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/AsyncOperationResult.java @@ -0,0 +1,107 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.sdk; + +import com.ververica.statefun.sdk.annotations.ForRuntime; +import java.util.Objects; +import java.util.concurrent.CompletableFuture; + +/** + * AsyncOperationResult - An asynchronous operation result. + * + *

{@code AsyncOperationResult} represents a completion of an asynchronous operation, registered + * by a stateful function instance via a {@link Context#registerAsyncOperation(Object, + * CompletableFuture)}. + * + *

The status of the asynchronous operation can be obtain via {@link #status()}, and it can be + * one of: + * + *

    + *
  • {@code success} - The asynchronous operation has succeeded, and the produced result can be + * obtained via {@link #value()}. + *
  • {@code failure} - The asynchronous operation has failed, and the cause can be obtained via + * ({@link #throwable()}. + *
  • {@code unknown} - the stateful function was restarted, possibly on a different machine, + * before the {@link CompletableFuture} was completed, therefore it is unknown what is the + * status of the asynchronous operation. + *
+ * + * @param metadata type + * @param result type. + */ +public final class AsyncOperationResult { + + public enum Status { + SUCCESS, + FAILURE, + UNKNOWN + } + + private final M metadata; + private final Status status; + private final T value; + private final Throwable throwable; + + @ForRuntime + public AsyncOperationResult(M metadata, Status status, T value, Throwable throwable) { + this.metadata = Objects.requireNonNull(metadata); + this.status = Objects.requireNonNull(status); + this.value = value; + this.throwable = throwable; + } + + /** + * @return the metadata assosicted with this async operation, as supplied at {@link + * Context#registerAsyncOperation(Object, CompletableFuture)}. + */ + public M metadata() { + return metadata; + } + + /** @return the status of this async operation. */ + public Status status() { + return status; + } + + /** @return the successfully completed value. */ + public T value() { + if (status != Status.SUCCESS) { + throw new IllegalStateException("Not a successful result, but rather " + status); + } + return value; + } + + /** @return the exception thrown during an attempt to complete the asynchronous operation. */ + public Throwable throwable() { + if (status != Status.FAILURE) { + throw new IllegalStateException("Not a failure, but rather " + status); + } + return throwable; + } + + public boolean successful() { + return status == Status.SUCCESS; + } + + public boolean unknown() { + return status == Status.UNKNOWN; + } + + public boolean failure() { + return status == Status.FAILURE; + } +} diff --git a/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/Context.java b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/Context.java new file mode 100644 index 00000000..59445428 --- /dev/null +++ b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/Context.java @@ -0,0 +1,141 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.sdk; + +import com.ververica.statefun.sdk.io.EgressIdentifier; +import java.time.Duration; +import java.util.concurrent.CompletableFuture; + +/** + * Provides context for a single {@link StatefulFunction} invocation. + * + *

The invocation's context may be used to obtain the {@link Address} of itself or the calling + * function (if the function was invoked by another function), or used to invoke other functions + * (including itself) and to send messages to egresses. + */ +public interface Context { + + /** + * Returns the {@link Address} of the invoked function. + * + * @return the invoked function's own address. + */ + Address self(); + + /** + * Returns the {@link Address} of the invoking function. This is {@code null} if the function + * under context was not invoked by another function. + * + * @return the address of the invoking function; {@code null} if the function under context was + * not invoked by another function. + */ + Address caller(); + + /** + * Invokes another function with an input, identified by the target function's {@link Address}. + * + * @param to the target function's address. + * @param message the input to provide for the invocation. + */ + void send(Address to, Object message); + + /** + * Sends an output to an egress, identified by the egress' {@link EgressIdentifier}. + * + * @param egress the target egress' identifier + * @param message the output to send + * @param type of the inputs that the target egress consumes + */ + void send(EgressIdentifier egress, T message); + + /** + * Invokes another function with an input, identified by the target function's {@link Address}, + * after a given delay. + * + * @param delay the amount of delay before invoking the target function. Value needs to be >= 0. + * @param to the target function's address. + * @param message the input to provide for the delayed invocation. + */ + void sendAfter(Duration delay, Address to, Object message); + + /** + * Invokes another function with an input, identified by the target function's {@link + * FunctionType} and unique id. + * + * @param functionType the target function's type. + * @param id the target function's id within its type. + * @param message the input to provide for the invocation. + */ + default void send(FunctionType functionType, String id, Object message) { + send(new Address(functionType, id), message); + } + + /** + * Invokes another function with an input, identified by the target function's {@link + * FunctionType} and unique id. + * + * @param delay the amount of delay before invoking the target function. Value needs to be >= 0. + * @param functionType the target function's type. + * @param id the target function's id within its type. + * @param message the input to provide for the delayed invocation. + */ + default void sendAfter(Duration delay, FunctionType functionType, String id, Object message) { + sendAfter(delay, new Address(functionType, id), message); + } + + /** + * Invokes the calling function of the current invocation under context. This has the same effect + * as calling {@link #send(Address, Object)} with the address obtained from {@link #caller()}, and + * will not work if the current function was not invoked by another function. + * + * @param message the input to provide to the replying invocation. + */ + default void reply(Object message) { + send(caller(), message); + } + + /** + * Registers an asynchronous operation. + * + *

Register an asynchronous operation represented by a {@code future}, and associated with + * {@code metadata}. + * + *

The runtime would invoke (at some time in the future) the currently executing stateful + * function with a {@link AsyncOperationResult} argument, that represents the completion of that + * asynchronous operation. + * + *

If the supplied future was completed successfully, then the result can be obtained via + * {@link AsyncOperationResult#value()}. If it is completed exceptionally, then the failure cause + * can be obtain via {@link AsyncOperationResult#throwable()}. + * + *

Please note that if, for some reason, the processes executing the stateful had fail, the + * status of the asynchronous operation is unknown (it might have succeeded or failed before the + * stateful function was notified). In that case the status of the {@code AsyncOperationResult} + * would be {@code UNKNOWN}. + * + *

{@code metadata} - Each asynchronous operation is also associated with a metadata object + * that can be used to correlate multiple in flight asynchronous operations. This object can be + * obtained via {@link AsyncOperationResult#metadata()}. This object would be serialized with the + * same serializer used to serializer the messages. + * + * @param metadata a meta data object to associated with this in flight async operation. + * @param future the {@link CompletableFuture} that represents the async operation. + * @param metadata type. + * @param value type. + */ + void registerAsyncOperation(M metadata, CompletableFuture future); +} diff --git a/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/EgressType.java b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/EgressType.java new file mode 100644 index 00000000..679509a6 --- /dev/null +++ b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/EgressType.java @@ -0,0 +1,85 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.sdk; + +import com.ververica.statefun.sdk.io.EgressSpec; +import java.util.Objects; + +/** + * Defines the type of an egress, represented by a namespace and the type's name. + * + *

This is used by the system to translate an {@link EgressSpec} to a physical runtime-specific + * representation. + */ +public final class EgressType { + private final String namespace; + private final String type; + + /** + * Creates an {@link EgressType}. + * + * @param namespace the type's namespace. + * @param type the type's name. + */ + public EgressType(String namespace, String type) { + this.namespace = Objects.requireNonNull(namespace); + this.type = Objects.requireNonNull(type); + } + + /** + * Returns the namespace of this egress type. + * + * @return the namespace of this egress type. + */ + public String namespace() { + return namespace; + } + + /** + * Returns the name of this egress type. + * + * @return the name of this egress type. + */ + public String type() { + return type; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + EgressType that = (EgressType) o; + return namespace.equals(that.namespace) && type.equals(that.type); + } + + @Override + public int hashCode() { + int hash = 0; + hash = 37 * hash + namespace.hashCode(); + hash = 37 * hash + type.hashCode(); + return hash; + } + + @Override + public String toString() { + return String.format("IngressType(%s, %s)", namespace, type); + } +} diff --git a/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/FunctionType.java b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/FunctionType.java new file mode 100644 index 00000000..8fb4ab62 --- /dev/null +++ b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/FunctionType.java @@ -0,0 +1,87 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.sdk; + +import java.util.Objects; + +/** + * This class represents the type of a {@link StatefulFunction}, consisting of a namespace of the + * function type as well as the type's name. + * + *

A function's type is part of a function's {@link Address} and serves as integral part of an + * individual function's identity. + * + * @see Address + */ +public final class FunctionType { + private final String namespace; + private final String type; + + /** + * Creates a {@link FunctionType}. + * + * @param namespace the function type's namepsace. + * @param type the function type's name. + */ + public FunctionType(String namespace, String type) { + this.namespace = Objects.requireNonNull(namespace); + this.type = Objects.requireNonNull(type); + } + + /** + * Returns the namespace of the function type. + * + * @return the namespace of the function type. + */ + public String namespace() { + return namespace; + } + + /** + * Returns the name of the function type. + * + * @return the name of the function type. + */ + public String name() { + return type; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + FunctionType functionType = (FunctionType) o; + return namespace.equals(functionType.namespace) && type.equals(functionType.type); + } + + @Override + public int hashCode() { + int hash = 0; + hash = 37 * hash + namespace.hashCode(); + hash = 37 * hash + type.hashCode(); + return hash; + } + + @Override + public String toString() { + return String.format("FunctionType(%s, %s)", namespace, type); + } +} diff --git a/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/IngressType.java b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/IngressType.java new file mode 100644 index 00000000..d79b91da --- /dev/null +++ b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/IngressType.java @@ -0,0 +1,85 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.sdk; + +import com.ververica.statefun.sdk.io.IngressSpec; +import java.util.Objects; + +/** + * Defines the type of an ingress, represented by a namespace and the type's name. + * + *

This is used by the system to translate an {@link IngressSpec} to a physical runtime-specific + * representation. + */ +public final class IngressType { + private final String namespace; + private final String type; + + /** + * Creates an {@link IngressType}. + * + * @param namespace the type's namespace. + * @param type the type's name. + */ + public IngressType(String namespace, String type) { + this.namespace = Objects.requireNonNull(namespace); + this.type = Objects.requireNonNull(type); + } + + /** + * Returns the namespace of this ingress type. + * + * @return the namespace of this ingress type. + */ + public String namespace() { + return namespace; + } + + /** + * Returns the name of this ingress type. + * + * @return the name of this ingress type. + */ + public String type() { + return type; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + IngressType that = (IngressType) o; + return namespace.equals(that.namespace) && type.equals(that.type); + } + + @Override + public int hashCode() { + int hash = 0; + hash = 37 * hash + namespace.hashCode(); + hash = 37 * hash + type.hashCode(); + return hash; + } + + @Override + public String toString() { + return String.format("IngressType(%s, %s)", namespace, type); + } +} diff --git a/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/StatefulFunction.java b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/StatefulFunction.java new file mode 100644 index 00000000..ab828df9 --- /dev/null +++ b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/StatefulFunction.java @@ -0,0 +1,84 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.sdk; + +import com.ververica.statefun.sdk.annotations.Persisted; +import com.ververica.statefun.sdk.io.EgressIdentifier; +import com.ververica.statefun.sdk.io.Router; +import com.ververica.statefun.sdk.state.PersistedValue; + +/** + * A {@link StatefulFunction} is a user-defined function that can be invoked with a given input. + * This is the primitive building block for a Stateful Functions application. + * + *

Concept

+ * + *

Each individual {@code StatefulFunction} is an uniquely invokable "instance" of a {@link + * FunctionType}. Each function is identified by an {@link Address}, representing the function's + * unique id (a string) within its type. From a user's perspective, it would seem as if for each + * unique function id, there exists a stateful instance of the function that is always available to + * be invoked within a Stateful Functions application. + * + *

Invoking a {@code StatefulFunction}

+ * + *

An individual {@code StatefulFunction} can be invoked with arbitrary input from any another + * {@code StatefulFunction} (including itself), or routed from ingresses via a {@link Router}. To + * invoke a {@code StatefulFunction}, the caller simply needs to know the {@code Address} of the + * target function. + * + *

As a result of invoking a {@code StatefulFunction}, the function may continue to invoke other + * functions, modify its state, or send messages to egresses addressed by an {@link + * EgressIdentifier}. + * + *

State

+ * + *

Each individual {@code StatefulFunction} may have state that is maintained by the system, + * providing exactly-once guarantees. Below is a code example of how to register and access state in + * functions: + * + *

{@code
+ * public class MyFunction implements StatefulFunction {
+ *
+ *     @Persisted
+ *     PersistedValue intState = PersistedValue.of("state-name", Integer.class);
+ *
+ *     @Override
+ *     public void invoke(Context context, Object input) {
+ *         Integer stateValue = intState.get();
+ *         //...
+ *         intState.set(1108);
+ *         // send messages using context
+ *     }
+ * }
+ * }
+ * + * @see Address + * @see FunctionType + * @see Persisted + * @see PersistedValue + */ +public interface StatefulFunction { + + /** + * Invokes this function with a given input. + * + * @param context context for the current invocation. The provided context instance should not be + * used outside the scope of the current invocation. + * @param input input for the current invocation. + */ + void invoke(Context context, Object input); +} diff --git a/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/StatefulFunctionProvider.java b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/StatefulFunctionProvider.java new file mode 100644 index 00000000..b08ca61b --- /dev/null +++ b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/StatefulFunctionProvider.java @@ -0,0 +1,29 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.sdk; + +/** Provides instances of {@link StatefulFunction}s for a given {@link FunctionType}. */ +public interface StatefulFunctionProvider { + + /** + * Creates a {@link StatefulFunction} instance for the given {@link FunctionType}, + * + * @param type the type of function to create an instance for. + * @return an instance of a function for the given type. + */ + StatefulFunction functionOfType(FunctionType type); +} diff --git a/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/annotations/ForRuntime.java b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/annotations/ForRuntime.java new file mode 100644 index 00000000..9d329ccb --- /dev/null +++ b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/annotations/ForRuntime.java @@ -0,0 +1,29 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.sdk.annotations; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Target; + +/** + * Methods or constructors annotated with this annotation, are used for the runtime to extend the + * API with specialized implementation + */ +@Documented +@Target({ElementType.CONSTRUCTOR, ElementType.METHOD}) +public @interface ForRuntime {} diff --git a/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/annotations/Persisted.java b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/annotations/Persisted.java new file mode 100644 index 00000000..85fe3cbd --- /dev/null +++ b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/annotations/Persisted.java @@ -0,0 +1,26 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.sdk.annotations; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +@Retention(RetentionPolicy.RUNTIME) +@Target(ElementType.FIELD) +public @interface Persisted {} diff --git a/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/io/EgressIdentifier.java b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/io/EgressIdentifier.java new file mode 100644 index 00000000..964c5138 --- /dev/null +++ b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/io/EgressIdentifier.java @@ -0,0 +1,101 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.sdk.io; + +import java.io.Serializable; +import java.util.Objects; + +/** + * This class identifies an egress within a Stateful Functions application, and is part of an {@link + * EgressSpec}. + */ +public final class EgressIdentifier implements Serializable { + + private static final long serialVersionUID = 1L; + + private final String namespace; + private final String name; + private final Class consumedType; + + /** + * Creates an {@link EgressIdentifier}. + * + * @param namespace the namespace of the egress. + * @param name the name of the egress. + * @param consumedType the type of messages consumed by the egress. + */ + public EgressIdentifier(String namespace, String name, Class consumedType) { + this.namespace = Objects.requireNonNull(namespace); + this.name = Objects.requireNonNull(name); + this.consumedType = Objects.requireNonNull(consumedType); + } + + /** + * Returns the namespace of the egress. + * + * @return the namespace of the egress. + */ + public String namespace() { + return namespace; + } + + /** + * Returns the name of the egress. + * + * @return the name of the egress. + */ + public String name() { + return name; + } + + /** + * Returns the type of messages consumed by the egress. + * + * @return the type of messages consumed by the egress. + */ + public Class consumedType() { + return consumedType; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + EgressIdentifier egressIdentifier = (EgressIdentifier) o; + return namespace.equals(egressIdentifier.namespace) + && name.equals(egressIdentifier.name) + && consumedType.equals(egressIdentifier.consumedType); + } + + @Override + public int hashCode() { + int hash = 0; + hash = 37 * hash + namespace.hashCode(); + hash = 37 * hash + name.hashCode(); + hash = 37 * hash + consumedType.hashCode(); + return hash; + } + + @Override + public String toString() { + return String.format("EgressKey(%s, %s, %s)", namespace, name, consumedType); + } +} diff --git a/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/io/EgressSpec.java b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/io/EgressSpec.java new file mode 100644 index 00000000..8a6fab54 --- /dev/null +++ b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/io/EgressSpec.java @@ -0,0 +1,46 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.sdk.io; + +import com.ververica.statefun.sdk.EgressType; + +/** + * Complete specification for an egress, containing of the egress' {@link EgressIdentifier} and the + * {@link EgressType}. This fully defines an egress within a Stateful Functions application. + * + *

This serves as a "logical" representation of an output sink that stateful functions within an + * application can send messages to. Under the scenes, the system translates this to a physical + * runtime-specific representation corresponding to the specified {@link EgressType}. + * + * @param the type of messages consumed by this egress. + */ +public interface EgressSpec { + + /** + * Returns the unique identifier of the egress. + * + * @return the unique identifier of the egress. + */ + EgressIdentifier id(); + + /** + * Returns the type of the egress. + * + * @return the type of the egress. + */ + EgressType type(); +} diff --git a/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/io/IngressIdentifier.java b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/io/IngressIdentifier.java new file mode 100644 index 00000000..e6b6f0a5 --- /dev/null +++ b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/io/IngressIdentifier.java @@ -0,0 +1,101 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.sdk.io; + +import java.io.Serializable; +import java.util.Objects; + +/** + * This class identifies an ingress within a Stateful Functions application, and is part of an + * {@link IngressSpec}. + */ +public final class IngressIdentifier implements Serializable { + + private static final long serialVersionUID = 1L; + + private final String namespace; + private final String name; + private final Class producedType; + + /** + * Creates an {@link IngressIdentifier}. + * + * @param producedType the type of messages produced by the ingress. + * @param namespace the namespace of the ingress. + * @param name the name of the ingress. + */ + public IngressIdentifier(Class producedType, String namespace, String name) { + this.namespace = Objects.requireNonNull(namespace); + this.name = Objects.requireNonNull(name); + this.producedType = Objects.requireNonNull(producedType); + } + + /** + * Returns the namespace of the ingress. + * + * @return the namespace of the ingress. + */ + public String namespace() { + return namespace; + } + + /** + * Returns the name of the ingress. + * + * @return the name of the ingress. + */ + public String name() { + return name; + } + + /** + * Returns the type of messages produced by the ingress. + * + * @return the type of messages produced by the ingress. + */ + public Class producedType() { + return producedType; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + IngressIdentifier that = (IngressIdentifier) o; + return namespace.equals(that.namespace) + && name.equals(that.name) + && producedType.equals(that.producedType); + } + + @Override + public int hashCode() { + int hash = 0; + hash = 37 * hash + namespace.hashCode(); + hash = 37 * hash + name.hashCode(); + hash = 37 * hash + producedType.hashCode(); + return hash; + } + + @Override + public String toString() { + return String.format("IngressIdentifier(%s, %s, %s)", namespace, name, producedType); + } +} diff --git a/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/io/IngressSpec.java b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/io/IngressSpec.java new file mode 100644 index 00000000..4ff74d8c --- /dev/null +++ b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/io/IngressSpec.java @@ -0,0 +1,46 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.sdk.io; + +import com.ververica.statefun.sdk.IngressType; + +/** + * Complete specification for an ingress, containing of the ingress' {@link IngressIdentifier} and + * the {@link IngressType}. This fully defines an ingress within a Stateful Functions application. + * + *

This serves as a "logical" representation of an input source for invoking stateful functions + * within an application. Under the scenes, the system translates this to a physical + * runtime-specific representation corresponding to the specified {@link IngressType}. + * + * @param the type of messages produced by this ingress. + */ +public interface IngressSpec { + + /** + * Returns the unique identifier of the ingress. + * + * @return the unique identifier of the ingress. + */ + IngressIdentifier id(); + + /** + * Returns the type of the ingress. + * + * @return the type of the ingress. + */ + IngressType type(); +} diff --git a/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/io/Router.java b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/io/Router.java new file mode 100644 index 00000000..c2ff95f5 --- /dev/null +++ b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/io/Router.java @@ -0,0 +1,69 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.sdk.io; + +import com.ververica.statefun.sdk.Address; +import com.ververica.statefun.sdk.FunctionType; +import com.ververica.statefun.sdk.StatefulFunction; + +/** + * A {@link Router} routes messages from ingresses to individual {@link StatefulFunction}s. + * + *

Implementations should be stateless, as any state in routers are not persisted by the system. + * + * @param the type of messages being routed. + */ +public interface Router { + + /** + * Routes a given message to downstream {@link StatefulFunction}s. A single message may result in + * multiple functions being invoked. + * + * @param message the message to route. + * @param downstream used to invoke downstream functions. + */ + void route(InT message, Downstream downstream); + + /** + * Interface for invoking downstream functions. + * + * @param the type of messages being routed to downstream functions. + */ + interface Downstream { + + /** + * Forwards the message as an input to a downstream function, addressed by a specified {@link + * Address}. + * + * @param to the target function's address. + * @param message the message being forwarded. + */ + void forward(Address to, T message); + + /** + * Forwards the message as an input to a downstream function, addressed by a specified {@link + * FunctionType} and the functions unique id within its type. + * + * @param functionType the target function's type. + * @param id the target function's unique id. + * @param message the message being forwarded. + */ + default void forward(FunctionType functionType, String id, T message) { + forward(new Address(functionType, id), message); + } + } +} diff --git a/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/match/MatchBinder.java b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/match/MatchBinder.java new file mode 100644 index 00000000..0238123d --- /dev/null +++ b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/match/MatchBinder.java @@ -0,0 +1,157 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.sdk.match; + +import com.ververica.statefun.sdk.Context; +import java.util.ArrayList; +import java.util.Collections; +import java.util.IdentityHashMap; +import java.util.List; +import java.util.Objects; +import java.util.function.BiConsumer; +import java.util.function.Predicate; + +/** + * Binds patterns to be matched on inputs and their corresponding actions to the processing logic of + * a {@link StatefulMatchFunction}. + * + *

The following methods are supported for binding patterns, in order of precedence that they are + * checked for matches: + * + *

    + *
  • {@link #predicate(Class, Predicate, BiConsumer)}: matches on the input's type and a + * conditional predicate on the input object state. + *
  • {@link #predicate(Class, BiConsumer)}: simple type match on the input's type. + *
  • {@link #otherwise(BiConsumer)}: default action to take if no match can be found for the + * input. + *
+ */ +public final class MatchBinder { + + private final IdentityHashMap, List>> multimethods = + new IdentityHashMap<>(); + + private final IdentityHashMap, BiConsumer> noPredicateMethods = + new IdentityHashMap<>(); + + private boolean customDefaultAction = false; + + private BiConsumer defaultAction = MatchBinder::unhandledDefaultAction; + + MatchBinder() {} + + /** + * Binds a simple type pattern which matches on the input's type. + * + *

This has a lower precedence than matches found on patterns registered via {@link + * #predicate(Class, Predicate, BiConsumer)}. If no conditional predicates matches for a given + * input of type {@code type}, then the action registered here will be used. + * + * @param type the expected input type. + * @param action the action to take if this pattern matches. + * @param the expected input type. + */ + @SuppressWarnings("unchecked") + public MatchBinder predicate(Class type, BiConsumer action) { + Objects.requireNonNull(type); + Objects.requireNonNull(action); + + if (noPredicateMethods.containsKey(type)) { + throw new IllegalStateException("There is already a catch all case for class " + type); + } + + noPredicateMethods.put(type, (BiConsumer) action); + return this; + } + + /** + * Binds a pattern which matches on a function's input type, as well as a conditional predicate on + * the input object's state. + * + *

Precedence of conditional predicate matches is determined by the order in which they were + * bind; predicates that were bind first have higher precedence. Patterns bind via this method + * have the highest precedence over other methods. + * + * @param type the expected input type. + * @param predicate a predicate on the input object state to match on. + * @param action the action to take if this patten matches. + * @param the expected input type. + */ + @SuppressWarnings("unchecked") + public MatchBinder predicate( + Class type, Predicate predicate, BiConsumer action) { + List> methods = multimethods.computeIfAbsent(type, ignored -> new ArrayList<>()); + BiConsumer a = (BiConsumer) action; + Predicate p = (Predicate) predicate; + methods.add(new Method<>(p, a)); + return this; + } + + /** + * Binds a default action for inputs that fail to match any of the patterns bind via the {@link + * #predicate(Class, Predicate, BiConsumer)} and {@link #predicate(Class, BiConsumer)} methods. If + * no default action was bind using this method, then a {@link IllegalStateException} would be + * thrown for inputs that fail to match. + * + * @param action the default action + */ + public MatchBinder otherwise(BiConsumer action) { + if (customDefaultAction) { + throw new IllegalStateException("There can only be one default action"); + } + + customDefaultAction = true; + defaultAction = Objects.requireNonNull(action); + return this; + } + + void invoke(Context context, Object input) { + final Class type = input.getClass(); + + List> methods = multimethods.getOrDefault(type, Collections.emptyList()); + for (Method m : methods) { + if (m.canApply(input)) { + m.apply(context, input); + return; + } + } + + noPredicateMethods.getOrDefault(type, defaultAction).accept(context, input); + } + + private static final class Method { + private final Predicate predicate; + private final BiConsumer apply; + + Method(Predicate predicate, BiConsumer method) { + this.predicate = Objects.requireNonNull(predicate); + this.apply = Objects.requireNonNull(method); + } + + boolean canApply(T input) { + return predicate.test(input); + } + + void apply(Context context, T input) { + apply.accept(context, input); + } + } + + private static void unhandledDefaultAction(Context context, Object input) { + throw new IllegalStateException("Don't know how to handle " + input); + } +} diff --git a/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/match/StatefulMatchFunction.java b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/match/StatefulMatchFunction.java new file mode 100644 index 00000000..ecf60d3a --- /dev/null +++ b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/match/StatefulMatchFunction.java @@ -0,0 +1,55 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.sdk.match; + +import com.ververica.statefun.sdk.Context; +import com.ververica.statefun.sdk.StatefulFunction; + +/** + * A {@link StatefulMatchFunction} is an utility {@link StatefulFunction} that supports pattern + * matching on function inputs to decide how the inputs should be processed. + * + *

Please see {@link MatchBinder} for the supported types of pattern matching. + * + * @see MatchBinder + */ +public abstract class StatefulMatchFunction implements StatefulFunction { + + private boolean setup = false; + + private MatchBinder matcher = new MatchBinder(); + + /** + * Configures the patterns to match for the function's inputs. + * + * @param binder a {@link MatchBinder} to bind patterns on. + */ + public abstract void configure(MatchBinder binder); + + @Override + public final void invoke(Context context, Object input) { + ensureInitialized(); + matcher.invoke(context, input); + } + + private void ensureInitialized() { + if (!setup) { + setup = true; + configure(matcher); + } + } +} diff --git a/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/spi/StatefulFunctionModule.java b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/spi/StatefulFunctionModule.java new file mode 100644 index 00000000..11926ab6 --- /dev/null +++ b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/spi/StatefulFunctionModule.java @@ -0,0 +1,112 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.sdk.spi; + +import com.ververica.statefun.sdk.FunctionType; +import com.ververica.statefun.sdk.StatefulFunction; +import com.ververica.statefun.sdk.StatefulFunctionProvider; +import com.ververica.statefun.sdk.io.EgressSpec; +import com.ververica.statefun.sdk.io.IngressIdentifier; +import com.ververica.statefun.sdk.io.IngressSpec; +import com.ververica.statefun.sdk.io.Router; +import java.util.Map; + +/** + * A {@link StatefulFunctionModule} is the entry point for adding to a Stateful Functions + * application the core building block primitives, i.e. {@link IngressSpec}s, {@link EgressSpec}s, + * {@link Router}s, and {@link StatefulFunction}s. + * + *

Extensibility of a Stateful Functions application

+ * + *

A Stateful Functions application is built up of ingresses, egresses, routers, and stateful + * functions that are added to the application by multiple different {@link + * StatefulFunctionModule}s. This allows different parts of the application to be contributed by + * different modules; for example, one module may provide ingresses and egresses, while other + * modules may individually contribute specific parts of the application as stateful functions. + * + *

The extensibility is achieved by leveraging the Java + * Service Loader. In this context, each module is essentially a service provider. + * + *

Registering a {@code StatefulFunctionModule}

+ * + *

In order for an application to discover a given module, likewise to how the Java Service + * Loader works, a UTF-8 encoded provider configuration file needs to be stored in the {@code + * MEGA-INF/services} directory of the module's containing JAR file. The name of the file should be + * {@code com.ververica.statefun.sdk.spi.StatefulFunctionModule}, i.e. the fully qualified name of + * the {@link StatefulFunctionModule} class. Each line in the file should be the fully qualified + * class name of a module in that JAR that you want to register for the Stateful Functions + * application. + * + *

Finally, to allow the Stateful Functions runtime to discover the registered modules, the JAR + * files containing the modules and provider configuration files should be added to a + * system-specific class path directory, {@code /opt/stateful-functions/modules/}. + * + *

For a simple demonstration, you can consult the {@code stateful-functions-greeter-example} + * example. + */ +public interface StatefulFunctionModule { + + /** + * This method is the entry point for extending a Stateful Functions application by binding + * ingresses, egresses, routers, and functions. + * + * @param globalConfiguration global configuration of the Stateful Functions application. + * @param binder the binder to be used to bind ingresses, egresses, routers, and functions. + */ + void configure(Map globalConfiguration, Binder binder); + + /** + * A {@link Binder} binds ingresses, egresses, routers, and functions to a Stateful Functions + * application. + */ + interface Binder { + + /** + * Binds an {@link IngressSpec} to the Stateful Functions application. + * + * @param spec the {@link IngressSpec} to bind. + * @param the output type of the ingress. + */ + void bindIngress(IngressSpec spec); + + /** + * Binds an {@link EgressSpec} to the Stateful Functions application. + * + * @param spec the {@link EgressSpec} to bind. + * @param the type of inputs that the egress consumes. + */ + void bindEgress(EgressSpec spec); + + /** + * Binds a {@link StatefulFunctionProvider} to the Stateful Functions application. + * + * @param functionType the type of functions that the {@link StatefulFunctionProvider} provides. + * @param provider the provider to bind. + */ + void bindFunctionProvider(FunctionType functionType, StatefulFunctionProvider provider); + + /** + * Binds a {@link Router} for a given ingress to the Stateful Functions application. + * + * @param id the id of the ingress to bind the router to. + * @param router the router to bind. + * @param the type of messages that is bbeing routed. + */ + void bindIngressRouter(IngressIdentifier id, Router router); + } +} diff --git a/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/state/Accessor.java b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/state/Accessor.java new file mode 100644 index 00000000..0b7148e1 --- /dev/null +++ b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/state/Accessor.java @@ -0,0 +1,26 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.sdk.state; + +public interface Accessor { + + void set(T value); + + T get(); + + void clear(); +} diff --git a/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/state/PersistedValue.java b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/state/PersistedValue.java new file mode 100644 index 00000000..f1d670b2 --- /dev/null +++ b/stateful-functions-sdk/src/main/java/com/ververica/statefun/sdk/state/PersistedValue.java @@ -0,0 +1,164 @@ +/* + * Copyright 2019 Ververica GmbH. + * + * 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. + */ + +package com.ververica.statefun.sdk.state; + +import com.ververica.statefun.sdk.StatefulFunction; +import com.ververica.statefun.sdk.annotations.ForRuntime; +import com.ververica.statefun.sdk.annotations.Persisted; +import java.util.Objects; +import java.util.function.Function; +import java.util.function.Supplier; + +/** + * A {@link PersistedValue} is a value registered within {@link StatefulFunction}s and is persisted + * and maintained by the system for fault-tolerance. + * + *

Created persisted values must be registered by using the {@link Persisted} annotation. Please + * see the class-level Javadoc of {@link StatefulFunction} for an example on how to do that. + * + * @see StatefulFunction + * @param type of the state. + */ +public final class PersistedValue { + private final String name; + private final Class type; + private Accessor accessor; + + private PersistedValue(String name, Class type, Accessor accessor) { + this.name = Objects.requireNonNull(name); + this.type = Objects.requireNonNull(type); + this.accessor = Objects.requireNonNull(accessor); + } + + /** + * Creates a {@link PersistedValue} instance that may be used to access persisted state managed by + * the system. Access to the persisted value is identified by an unique name and type of the + * value. These may not change across multiple executions of the application. + * + * @param name the unique name of the persisted state. + * @param type the type of the state values of this {@code PersistedValue}. + * @param the type of the state values. + * @return a {@code PersistedValue} instance. + */ + public static PersistedValue of(String name, Class type) { + return new PersistedValue<>(name, type, new NonFaultTolerantAccessor<>()); + } + + /** + * Returns the unique name of the persisted value. + * + * @return unique name of the persisted value. + */ + public String name() { + return name; + } + + /** + * Returns the type of the persisted values. + * + * @return the type of the persisted values. + */ + public Class type() { + return type; + } + + /** + * Returns the persisted value. + * + * @return the persisted value. + */ + public T get() { + return accessor.get(); + } + + /** + * Updates the persisted value. + * + * @param value the new value. + */ + public void set(T value) { + accessor.set(value); + } + + /** Clears the persisted value. After being cleared, the value would be {@code null}. */ + public void clear() { + accessor.clear(); + } + + /** + * Updates the persisted value and returns it, in a single operation. + * + * @param update function to process the previous value to obtain the new value. + * @return the new updated value. + */ + public T updateAndGet(Function update) { + T current = accessor.get(); + T updated = update.apply(current); + accessor.set(updated); + return updated; + } + + /** + * Attempts to get the persisted value. If the current value is {@code null}, then a specified + * default is returned instead. + * + * @param orElse the default value to return if the current value is not present. + * @return the persisted value, or the provided default if it isn't present. + */ + public T getOrDefault(T orElse) { + T value = accessor.get(); + return value != null ? value : orElse; + } + + /** + * Attempts to get the persisted value. If the current value is {@code null}, then a default value + * obtained from a specified supplier is returned instead. + * + * @param defaultSupplier supplier for a default value to be used if the current value is not + * present. + * @return the persisted value, or a default value if it isn't present. + */ + public T getOrDefault(Supplier defaultSupplier) { + T value = accessor.get(); + return value != null ? value : defaultSupplier.get(); + } + + @ForRuntime + void setAccessor(Accessor newAccessor) { + Objects.requireNonNull(newAccessor); + this.accessor = newAccessor; + } + + private static final class NonFaultTolerantAccessor implements Accessor { + private E element; + + @Override + public void set(E element) { + this.element = element; + } + + @Override + public E get() { + return element; + } + + @Override + public void clear() { + element = null; + } + } +} diff --git a/tools/docker/Dockerfile b/tools/docker/Dockerfile new file mode 100644 index 00000000..0bf11233 --- /dev/null +++ b/tools/docker/Dockerfile @@ -0,0 +1,41 @@ +# +# Copyright 2019 Ververica GmbH. +# +# 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. +# + +FROM flink:1.9.0 + +ENV ROLE worker +ENV MASTER_HOST localhost +ENV STATEFUL_FUNCTIONS_HOME /opt/stateful-functions +ENV STATEFUL_FUNCTIONS_MODULES $STATEFUL_FUNCTIONS_HOME/modules + +# cleanup flink-lib +RUN rm -fr $FLINK_HOME/lib/flink-table*jar + +# copy our distriubtion template +COPY flink/ $FLINK_HOME/ + +# add user modules +USER root + +RUN mkdir -p $STATEFUL_FUNCTIONS_MODULES && \ + useradd --system --home-dir $STATEFUL_FUNCTIONS_HOME --uid=9998 --gid=flink stateful-functions && \ + chown -R stateful-functions:flink $STATEFUL_FUNCTIONS_HOME && \ + chmod -R g+rw $STATEFUL_FUNCTIONS_HOME + +# entry point +ADD docker-entry-point.sh /docker-entry-point.sh + +ENTRYPOINT ["/docker-entry-point.sh"] diff --git a/tools/docker/build-stateful-functions.sh b/tools/docker/build-stateful-functions.sh new file mode 100755 index 00000000..8beacdda --- /dev/null +++ b/tools/docker/build-stateful-functions.sh @@ -0,0 +1,62 @@ +#!/bin/bash +# +# Copyright 2019 Ververica GmbH. +# +# 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. +# + +set -e + +# +# setup the environment +# +basedir="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null && pwd )" +project_root="${basedir}/../../" # ditch tools/docker +flink_template="${basedir}/flink-distribution-template" + +# +# check if the artifacts were build +# +distribution_jar=$(find ${project_root} -type f -name "stateful-functions-flink-distribution*jar" -not -name "*example*") +if [[ -z "${distribution_jar}" ]]; then + echo "unable to find stateful-functions-flink-distribution jar, please build the maven project first" + exit 1 +fi +core_jar=$(find ${project_root} -type f -name "stateful-functions-flink-core*jar") +if [[ -z "${core_jar}" ]]; then + echo "unable to find stateful-functions-flink-core jar, please build the maven project first" + exit 2 +fi + +# +# create a scratch space for a minimal docker context +# +docker_context_root=`mktemp -d 2>/dev/null || mktemp -d -t 'stateful-functions-docker-context'` +docker_context_flink="${docker_context_root}/flink" + +# +# prepare the adjustments to the vanilla flink distribution +# +mkdir -p ${docker_context_flink} +cp -r ${flink_template}/* ${docker_context_flink}/ +mkdir -p ${docker_context_flink}/lib +cp ${distribution_jar} ${docker_context_flink}/lib/stateful-functions-flink-distribution.jar +cp ${core_jar} ${docker_context_flink}/lib/stateful-functions-flink-core.jar +# build the docker image +cd ${docker_context_root} +cp ${basedir}/Dockerfile ${docker_context_root} +cp ${basedir}/docker-entry-point.sh ${docker_context_root} +docker build . -t stateful-functions + +# clean again +rm -rf ${docker_context_root} diff --git a/tools/docker/docker-entry-point.sh b/tools/docker/docker-entry-point.sh new file mode 100755 index 00000000..59637140 --- /dev/null +++ b/tools/docker/docker-entry-point.sh @@ -0,0 +1,56 @@ +#!/bin/bash + +# +# Copyright 2019 Ververica GmbH. +# +# 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. +# + +# +# Role types +# +WORKER="worker" +MASTER="master" + +# +# Environment +# +FLINK_HOME=${FLINK_HOME:-"/opt/flink/bin"} +ROLE=${ROLE:-"worker"} +MASTER_HOST=${MASTER_HOST:-"localhost"} + +# +# Start a service depending on the role. +# +if [[ "${ROLE}" == "${WORKER}" ]]; then + # + # start the TaskManager (worker role) + # + exec ${FLINK_HOME}/bin/taskmanager.sh start-foreground \ + -Djobmanager.rpc.address=${MASTER_HOST} + +elif [[ "${ROLE}" == "${MASTER}" ]]; then + # + # start the JobManager (master role) with our predefined job. + # + exec $FLINK_HOME/bin/standalone-job.sh \ + start-foreground \ + -Djobmanager.rpc.address=${MASTER_HOST} \ + "$@" +else + # + # unknown role + # + echo "unknown role ${ROLE}" + exit 1 +fi diff --git a/tools/docker/flink-distribution-template/bin/flink-console.sh b/tools/docker/flink-distribution-template/bin/flink-console.sh new file mode 100755 index 00000000..0d702f17 --- /dev/null +++ b/tools/docker/flink-distribution-template/bin/flink-console.sh @@ -0,0 +1,77 @@ +#!/usr/bin/env bash +# +# Copyright 2019 Ververica GmbH. +# +# 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. +# + +# This file was taken from Apache Flink, and modified to include another entry point + +# Start a Flink service as a console application. Must be stopped with Ctrl-C +# or with SIGTERM by kill or the controlling process. +USAGE="Usage: flink-console.sh (taskexecutor|zookeeper|historyserver|standalonesession|standalonejob|statefun) [args]" + +SERVICE=$1 +ARGS=("${@:2}") # get remaining arguments as array + +bin=`dirname "$0"` +bin=`cd "$bin"; pwd` + +. "$bin"/config.sh + +case $SERVICE in + (taskexecutor) + CLASS_TO_RUN=org.apache.flink.runtime.taskexecutor.TaskManagerRunner + ;; + + (historyserver) + CLASS_TO_RUN=org.apache.flink.runtime.webmonitor.history.HistoryServer + ;; + + (zookeeper) + CLASS_TO_RUN=org.apache.flink.runtime.zookeeper.FlinkZooKeeperQuorumPeer + ;; + + (standalonesession) + CLASS_TO_RUN=org.apache.flink.runtime.entrypoint.StandaloneSessionClusterEntrypoint + ;; + + (standalonejob) + CLASS_TO_RUN=org.apache.flink.container.entrypoint.StandaloneJobClusterEntryPoint + ;; + + (statefun) + CLASS_TO_RUN=com.ververica.statefun.flink.launcher.StatefulFunctionsClusterEntryPoint + ;; + + (*) + echo "Unknown service '${SERVICE}'. $USAGE." + exit 1 + ;; +esac + +FLINK_TM_CLASSPATH=`constructFlinkClassPath` + +log_setting=("-Dlog4j.configuration=file:${FLINK_CONF_DIR}/log4j-console.properties" "-Dlogback.configurationFile=file:${FLINK_CONF_DIR}/logback-console.xml") + +JAVA_VERSION=$(${JAVA_RUN} -version 2>&1 | sed 's/.*version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q') + +# Only set JVM 8 arguments if we have correctly extracted the version +if [[ ${JAVA_VERSION} =~ ${IS_NUMBER} ]]; then + if [ "$JAVA_VERSION" -lt 18 ]; then + JVM_ARGS="$JVM_ARGS -XX:MaxPermSize=256m" + fi +fi + +echo "Starting $SERVICE as a console application on host $HOSTNAME." +exec $JAVA_RUN $JVM_ARGS ${FLINK_ENV_JAVA_OPTS} "${log_setting[@]}" -classpath "`manglePathList "$FLINK_TM_CLASSPATH:$INTERNAL_HADOOP_CLASSPATHS"`" ${CLASS_TO_RUN} "${ARGS[@]}" diff --git a/tools/docker/flink-distribution-template/bin/standalone-job.sh b/tools/docker/flink-distribution-template/bin/standalone-job.sh new file mode 100755 index 00000000..11785eb1 --- /dev/null +++ b/tools/docker/flink-distribution-template/bin/standalone-job.sh @@ -0,0 +1,64 @@ +#!/usr/bin/env bash +# +# Copyright 2019 Ververica GmbH. +# +# 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. +# + +# This file was taken from Apache Flink, and modified to include another entry point + +# Start/stop a Flink JobManager. +USAGE="Usage: standalone-job.sh ((start|start-foreground))|stop [args]" + +STARTSTOP=$1 +ENTRY_POINT_NAME="statefun" + +if [[ $STARTSTOP != "start" ]] && [[ $STARTSTOP != "start-foreground" ]] && [[ $STARTSTOP != "stop" ]]; then + echo $USAGE + exit 1 +fi + +bin=`dirname "$0"` +bin=`cd "$bin"; pwd` + +. "$bin"/config.sh + +# Startup parameters +ARGS=("--configDir" "${FLINK_CONF_DIR}" "${@:2}") + +if [[ $STARTSTOP == "start" ]] || [[ $STARTSTOP == "start-foreground" ]]; then + if [ ! -z "${FLINK_JM_HEAP_MB}" ] && [ "${FLINK_JM_HEAP}" == 0 ]; then + echo "used deprecated key \`${KEY_JOBM_MEM_MB}\`, please replace with key \`${KEY_JOBM_MEM_SIZE}\`" + else + flink_jm_heap_bytes=$(parseBytes ${FLINK_JM_HEAP}) + FLINK_JM_HEAP_MB=$(getMebiBytes ${flink_jm_heap_bytes}) + fi + + if [[ ! ${FLINK_JM_HEAP_MB} =~ $IS_NUMBER ]] || [[ "${FLINK_JM_HEAP_MB}" -lt "0" ]]; then + echo "[ERROR] Configured memory size is not a valid value. Please set '${KEY_JOBM_MEM_SIZE}' in ${FLINK_CONF_FILE}." + exit 1 + fi + + if [ "${FLINK_JM_HEAP_MB}" -gt "0" ]; then + export JVM_ARGS="$JVM_ARGS -Xms"$FLINK_JM_HEAP_MB"m -Xmx"$FLINK_JM_HEAP_MB"m" + fi + + # Add cluster entry point specific JVM options + export FLINK_ENV_JAVA_OPTS="${FLINK_ENV_JAVA_OPTS} ${FLINK_ENV_JAVA_OPTS_JM}" +fi + +if [[ $STARTSTOP == "start-foreground" ]]; then + exec "${FLINK_BIN_DIR}"/flink-console.sh ${ENTRY_POINT_NAME} "${ARGS[@]}" +else + "${FLINK_BIN_DIR}"/flink-daemon.sh ${STARTSTOP} ${ENTRY_POINT_NAME} "${ARGS[@]}" +fi diff --git a/tools/docker/flink-distribution-template/conf/flink-conf.yaml b/tools/docker/flink-distribution-template/conf/flink-conf.yaml new file mode 100644 index 00000000..85e9e82b --- /dev/null +++ b/tools/docker/flink-distribution-template/conf/flink-conf.yaml @@ -0,0 +1,24 @@ +# +# Copyright 2019 Ververica GmbH. +# +# 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. +# + +# This file is the base for the Apache Flink configuration + +classloader.parent-first-patterns.additional: com.ververica.statefun;org.apache.kafka +state.backend: rocksdb +state.backend.rocksdb.timer-service.factory: ROCKSDB +state.checkpoints.dir: file:///checkpoint-dir + + diff --git a/tools/maven/spotbugs-exclude.xml b/tools/maven/spotbugs-exclude.xml new file mode 100644 index 00000000..70a62483 --- /dev/null +++ b/tools/maven/spotbugs-exclude.xml @@ -0,0 +1,77 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +