You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by tz...@apache.org on 2020/03/12 02:45:51 UTC

[flink-statefun] 06/08: [FLINK-16515][docs] Add Python SDK documentation

This is an automated email from the ASF dual-hosted git repository.

tzulitai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink-statefun.git

commit eea6d104f95fc4df2e10a72a816329d1d20da4a2
Author: Seth Wiesman <sj...@gmail.com>
AuthorDate: Wed Mar 4 19:19:52 2020 -0600

    [FLINK-16515][docs] Add Python SDK documentation
---
 statefun-docs/docs/concepts/index.rst        |   1 -
 statefun-docs/docs/concepts/logical.rst      |   6 +-
 statefun-docs/docs/concepts/modules.rst      |  34 ------
 statefun-docs/docs/sdk/index.rst             |   2 +
 statefun-docs/docs/sdk/java/index.rst        |  26 +---
 statefun-docs/docs/sdk/modules.rst           |  66 +++++++++++
 statefun-docs/docs/sdk/python.rst            | 170 +++++++++++++++++++++++++++
 statefun-docs/src/main/python/Caller.py      |  35 ++++++
 statefun-docs/src/main/python/Counter.py     |  55 +++++++++
 statefun-docs/src/main/python/Delayed.py     |  44 +++++++
 statefun-docs/src/main/python/HelloWorld.py  |  29 +++++
 statefun-docs/src/main/python/TypeHint.py    |  37 ++++++
 statefun-docs/src/main/resources/module.yaml |  31 +++++
 13 files changed, 477 insertions(+), 59 deletions(-)

diff --git a/statefun-docs/docs/concepts/index.rst b/statefun-docs/docs/concepts/index.rst
index d9656fe..783e16c 100644
--- a/statefun-docs/docs/concepts/index.rst
+++ b/statefun-docs/docs/concepts/index.rst
@@ -22,7 +22,6 @@ Concepts
 .. toctree::
   :hidden:
 
-  modules
   logical
 
 Stateful Functions provides a framework for building event drivent applications.
diff --git a/statefun-docs/docs/concepts/logical.rst b/statefun-docs/docs/concepts/logical.rst
index 01aa2d4..28e596e 100644
--- a/statefun-docs/docs/concepts/logical.rst
+++ b/statefun-docs/docs/concepts/logical.rst
@@ -25,6 +25,8 @@ Users are encouraged to model their applications as granularly as possible, base
 
 .. contents:: :local:
 
+.. _address:
+
 Function Address
 ================
 
@@ -37,7 +39,7 @@ Instead, an ``Address`` is used to reference a specific stateful functions in th
   syntax = "proto3";
 
   message Address {
-        
+
     message FunctionType {
       string namespace = 1;
       string name      = 2;
@@ -64,7 +66,7 @@ The application is free to create as many instances as there are types of items
 Function Lifecycle
 ==================
 
-Logical functions are neither created nor destoryed, but always exist throughout the lifetime of an application.
+Logical functions are neither created nor destroyed, but always exist throughout the lifetime of an application.
 When an application starts, each parallel worker of the framework will create one physical object per function type.
 This object will be used to execute all logical instances of that type that are run by that particular worker.
 The first time a message is sent to an address, it will be as if that instance had always existed with its persisted values returning ``NULL``.
diff --git a/statefun-docs/docs/concepts/modules.rst b/statefun-docs/docs/concepts/modules.rst
deleted file mode 100644
index cb06fec..0000000
--- a/statefun-docs/docs/concepts/modules.rst
+++ /dev/null
@@ -1,34 +0,0 @@
-.. 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.
-
-.. _modules:
-
-#######
-Modules
-#######
-
-Stateful Function applications are composed of one or more ``Modules``.
-A module is a bundle of functions that are loaded by the runtime and available to be messaged.
-Functions from all loaded modules are multiplexed and free to message each other arbitrarily.
-
-Stateful Functions supports two types of modules: Embedded and Remote.
-
-.. contents:: :local:
-
-Embedded Module
-===============
-
-Remote Module
-=============
diff --git a/statefun-docs/docs/sdk/index.rst b/statefun-docs/docs/sdk/index.rst
index 7080487..046e779 100644
--- a/statefun-docs/docs/sdk/index.rst
+++ b/statefun-docs/docs/sdk/index.rst
@@ -23,6 +23,8 @@ SDK
   :hidden:
   
   java/index
+  python
+  modules
 
 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.
diff --git a/statefun-docs/docs/sdk/java/index.rst b/statefun-docs/docs/sdk/java/index.rst
index 8dd14cf..a9a46e9 100644
--- a/statefun-docs/docs/sdk/java/index.rst
+++ b/statefun-docs/docs/sdk/java/index.rst
@@ -27,6 +27,7 @@ Java
 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 user, device, or session) and encode its behavior.
 Stateful functions can interact with each other, and external systems, through message passing.
+The Java SDK is supported as an :ref:`embedded_module`.
 
 To get started, add the Java SDK as a dependency to your application.
 
@@ -44,7 +45,8 @@ To get started, add the Java SDK as a dependency to your application.
 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.
+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/org/apache/flink/statefun/docs/FnHelloWorld.java
     :language: java
@@ -60,7 +62,7 @@ Function Types and Messaging
 ^^^^^^^^^^^^^^^^^^^^^^^^^^^^
 
 In Java, function types are defined as a stringly typed reference containing a namespace and name.
-The type is bound to the implementing class in the :ref:`module <java_module>` definition.
+The type is bound to the implementing class in the :ref:`module <embedded_module>` definition.
 Below is an example function type for the hello world function.
 
 .. literalinclude:: ../../../src/main/java/org/apache/flink/statefun/docs/Identifiers.java
@@ -174,23 +176,3 @@ Now, tests can quickly provide mock, or test dependencies, without the need for
 .. literalinclude:: ../../../src/main/java/org/apache/flink/statefun/docs/FunctionTest.java
     :language: java
     :lines: 18-
-
-.. _java_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`` under the prefix ``statefun.module.global-config`` and any command line arguments passed in the form ``--key value``.
-
-.. literalinclude:: ../../../src/main/java/org/apache/flink/statefun/docs/BasicFunctionModule.java
-    :language: java
-    :lines: 18-
-
-Modules leverage `Java’s Service Provider Interfaces (SPI) <https://docs.oracle.com/javase/8/docs/api/java/util/ServiceLoader.html>`_ for discovery.
-This means that every JAR should contain a file ``org.apache.flink.statefun.sdk.spi.StatefulFunctionModule`` in the ``META_INF/services`` resource directory that lists all available modules that it provides.
-
-.. code-block:: yaml
-
-    org.apache.flink.statefun.docs.BasicFunctionModule
diff --git a/statefun-docs/docs/sdk/modules.rst b/statefun-docs/docs/sdk/modules.rst
new file mode 100644
index 0000000..d6103da
--- /dev/null
+++ b/statefun-docs/docs/sdk/modules.rst
@@ -0,0 +1,66 @@
+.. 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.
+
+.. _modules:
+
+#######
+Modules
+#######
+
+Stateful Function applications are composed of one or more ``Modules``.
+A module is a bundle of functions that are loaded by the runtime and available to be messaged.
+Functions from all loaded modules are multiplexed and free to message each other arbitrarily.
+
+Stateful Functions supports two types of modules: Embedded and Remote.
+
+.. contents:: :local:
+
+.. _embedded_module:
+
+Embedded Module
+===============
+
+Embedded modules are co-located with, and embedded within, the {flink} runtime.
+
+This module type only supports JVM based languages and are defined by implementing the ``StatefulFunctionModule`` interface.
+Embedded modules offer a single configuration method where stateful functions are bound to the system based on their :ref:`function type <address>`.
+Runtime configurations are available through the ``globalConfiguration``, which is the union of all configurations in the applications ``flink-conf.yaml`` under the prefix ``statefun.module.global-config`` and any command line arguments passed in the form ``--key value``.
+
+.. literalinclude:: ../../src/main/java/org/apache/flink/statefun/docs/BasicFunctionModule.java
+    :language: java
+    :lines: 18-
+
+Embedded modules leverage `Java’s Service Provider Interfaces (SPI) <https://docs.oracle.com/javase/8/docs/api/java/util/ServiceLoader.html>`_ for discovery.
+This means that every JAR should contain a file ``org.apache.flink.statefun.sdk.spi.StatefulFunctionModule`` in the ``META_INF/services`` resource directory that lists all available modules that it provides.
+
+.. code-block:: yaml
+
+    org.apache.flink.statefun.docs.BasicFunctionModule
+
+.. _remote_module:
+
+Remote Module
+=============
+
+Remote modules are run as external processes from the {flink} runtime; in the same container, as a sidecar, or other external location.
+
+This module type can support any number of language SDK's.
+Remote modules are registered with the system via ``YAML`` configuration files.
+
+.. literalinclude:: ../../src/main/resources/module.yaml
+    :language: yaml
+    :lines: 16-
+
+Remote module definitions eagerly register each function type, the endpoint under which the function is available, along with all states for a particular function.
diff --git a/statefun-docs/docs/sdk/python.rst b/statefun-docs/docs/sdk/python.rst
new file mode 100644
index 0000000..f394eb7
--- /dev/null
+++ b/statefun-docs/docs/sdk/python.rst
@@ -0,0 +1,170 @@
+.. 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.
+
+.. _python:
+
+######
+Python
+######
+
+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 user, device, or session) and encode its behavior.
+Stateful functions can interact with each other, and external systems, through message passing.
+The Python SDK is supported as a :ref:`remote_module`.
+
+To get started, add the Python SDK as a dependency to your application.
+
+.. code-block:: bash
+
+    apache-flink-statefun=={version}
+
+.. contents:: :local:
+
+Defining A Stateful Function
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+A stateful function is any function that that takes two parameters, a ``context`` and ``message``.
+The function is bound to the runtime through the stateful functions decorator.
+The following is an example of a simple hello world function.
+
+.. literalinclude:: ../../src/main/python/HelloWorld.py
+    :language: python
+    :lines: 19-
+
+This code declares a function with in the namespace ``flink`` and of type ``hello`` and binds it to the ``hello_function`` Python instance.
+
+Messages's are untyped and passed through the system as ``google.protobuf.Any`` 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.
+A full reference of all methods supported by the context object are listed at the :ref:`bottom of this page <context_reference>`.
+
+Type Hints
+==========
+
+If the function has a static set of known supported types, they may be specified as `type hints <https://docs.python.org/3/library/typing.html>`_.
+This includes `union types <https://docs.python.org/3/library/typing.html#typing.Union>`_ for functions that support multiple input message types.
+
+
+.. literalinclude:: ../../src/main/python/TypeHint.py
+    :language: python
+    :lines: 19-
+
+Function Types and Messaging
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+The decorator ``bind`` registers each function with the runtime under a function type.
+The function type must take the form ``<namespace>/<name>``.
+Function types can then be referenced from other functions to create an address and message a particular instance.
+
+.. literalinclude:: ../../src/main/python/Caller.py
+    :language: python
+    :lines: 19-
+
+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.
+The delay is specified via a `Python timedelta <https://docs.python.org/3/library/datetime.html#datetime.timedelta>`_.
+
+.. literalinclude:: ../../src/main/python/Delayed.py
+    :language: python
+    :lines: 19-
+
+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.
+All stateful functions may contain state by merely storing values within the ``context`` object.
+The data is always scoped to a specific function type and identifier.
+State values could be absent, ``None``, or a ``google.protobuf.Any``.
+
+.. warning::
+
+    :ref:`remote_module`'s require that all state values are eagerly registered at ``module.yaml``.
+
+Below is a stateful function that greets users based on the number of times they have been seen.
+
+.. literalinclude:: ../../src/main/python/Counter.py
+    :language: python
+    :lines: 19-
+
+Additionally, persisted values may be cleared by deleting its value.
+
+.. code-block:: python
+
+    del context["count"]
+
+
+Exposing Functions
+^^^^^^^^^^^^^^^^^^
+
+The Python SDK ships with a ``RequestReplyHandler`` that automatically dispatches function calls based on RESTful HTTP ``POSTS``.
+The ``RequestReplyHandler`` may be exposed using any HTTP framework.
+
+.. code-block:: python
+
+    from statefun import RequestReplyHandler
+
+    handler RequestReplyHandler(functions)
+
+Serving Functions With Flask
+============================
+
+One popular Python web framework is `Flask <https://palletsprojects.com/p/flask/>`_.
+It can be used to quickly and easily expose a ``RequestResponseHandler``.
+
+.. code-block:: python
+
+    @app.route('/statefun', methods=['POST'])
+    def handle():
+        response_data = handler(request.data)
+        response = make_response(response_data)
+        response.headers.set('Content-Type', 'application/octet-stream')
+        return response
+
+
+    if __name__ == "__main__":
+        app.run()
+
+.. _context_reference:
+
+Context Reference
+^^^^^^^^^^^^^^^^^
+
+The ``context`` object passed to each function has the following attributes / methods.
+
+* send(self, typename: str, id: str, message: Any)
+    * Send a message to any function with the function type of the the form ``<namesapce>/<type>`` and message of type ``google.protobuf.Any``
+* pack_and_send(self, typename: str, id: str, message)
+    * The same as above, but it will pack the protobuf message in an ``Any``
+* reply(self, message: Any)
+    * Sends a message to the invoking function
+* pack_and_reply(self, message)
+    * The same as above, but it will pack the protobuf message in an ``Any``
+* send_after(self, delay: timedelta, typename: str, id: str, message: Any)
+    * Sends a message after a delay
+* pack_and_send_after(self, delay: timedelta, typename: str, id: str, message)
+    * The same as above, but it will pack the protobuf message in an ``Any``
+* send_egress(self, typename, message: Any)
+    * Emits a message to an egress with a typename of the form ``<namespace>/<name>``
+* pack_and_send_egress(self, typename, message)
+    * The same as above, but it will pack the protobuf message in an ``Any``
+* __getitem__(self, name)
+    * Retrieves the state registered under the name as an ``Any`` or ``None`` if no value is set
+* __delitem__(self, name)
+    * Deletes the state registered under the name
+* __setitem__(self, name, value: Any)
+    * Stores the value under the given name in state.
diff --git a/statefun-docs/src/main/python/Caller.py b/statefun-docs/src/main/python/Caller.py
new file mode 100644
index 0000000..c176b80
--- /dev/null
+++ b/statefun-docs/src/main/python/Caller.py
@@ -0,0 +1,35 @@
+################################################################################
+#  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.
+################################################################################
+
+from google.protobuf.any_pb2 import Any
+from statefun import StatefulFunctions
+
+functions = StatefulFunctions()
+
+@functions.bind("flink/caller")
+def caller(context, message):
+    """A simple stateful function that sends a message to the user with id `user1`"""
+
+    user = User()
+    user.user_id = "user1"
+    user.name = "Seth"
+
+    envelope = Any()
+    envelope.Pack(user)
+
+    context.send("flink/hello", user.user_id, envelope)
\ No newline at end of file
diff --git a/statefun-docs/src/main/python/Counter.py b/statefun-docs/src/main/python/Counter.py
new file mode 100644
index 0000000..4c1a79d
--- /dev/null
+++ b/statefun-docs/src/main/python/Counter.py
@@ -0,0 +1,55 @@
+################################################################################
+#  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.
+################################################################################
+
+from google.protobuf.any_pb2 import Any
+from statefun import StatefulFunctions
+
+functions = StatefulFunctions()
+
+@functions.bind("flink/count")
+def count_greeter(context, message):
+    """Function that greets a user based on
+    the number of times it has been called"""
+    user = User()
+    message.Unpack(user)
+
+
+    state = context["count"]
+    if state is None:
+        state = Any()
+        state.Pack(Count(1))
+        output = generate_message(1, user)
+    else:
+        counter = Count()
+        state.Unpack(counter)
+        counter.value += 1
+        output = generate_message(counter.value, user)
+        state.Pack(counter)
+
+    context["count"] = state
+    print(output)
+
+def generate_message(count, user):
+    if count == 1:
+        return "Hello " + user.name
+    elif count == 2:
+        return "Hello again!"
+    elif count == 3:
+        return "Third time's the charm"
+    else:
+        return "Hello for the " + count + "th time"
\ No newline at end of file
diff --git a/statefun-docs/src/main/python/Delayed.py b/statefun-docs/src/main/python/Delayed.py
new file mode 100644
index 0000000..a68a048
--- /dev/null
+++ b/statefun-docs/src/main/python/Delayed.py
@@ -0,0 +1,44 @@
+################################################################################
+#  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.
+################################################################################
+
+from google.protobuf.any_pb2 import Any
+from datetime import timedelta
+from statefun import StatefulFunctions
+
+functions = StatefulFunctions()
+
+@functions.bind("flink/delayed")
+def delayed(context, message):
+    """A function that sends itself a message after a delay """
+
+    if message.Is(Message.DESCRIPTOR):
+        print("Hello!")
+
+        envelope = Any()
+        envelope.Pack(DelayedMessage())
+
+        context.send_after(
+            context.self.typename(),
+            context.self.identity,
+            timedelta(minutes=1),
+            envelope)
+        return
+
+    if message.Is(DelayedMessage.DESCRIPTOR):
+        print("Hello from the future!")
+        return
\ No newline at end of file
diff --git a/statefun-docs/src/main/python/HelloWorld.py b/statefun-docs/src/main/python/HelloWorld.py
new file mode 100644
index 0000000..9d7ea72
--- /dev/null
+++ b/statefun-docs/src/main/python/HelloWorld.py
@@ -0,0 +1,29 @@
+################################################################################
+#  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.
+################################################################################
+
+from statefun import StatefulFunctions
+
+functions = StatefulFunctions()
+
+@functions.bind("flink/hello")
+def hello_function(context, message):
+    """A simple hello world function"""
+    user = User()
+    message.Unpack(user)
+
+    print("Hello " + user.name)
\ No newline at end of file
diff --git a/statefun-docs/src/main/python/TypeHint.py b/statefun-docs/src/main/python/TypeHint.py
new file mode 100644
index 0000000..2af4d47
--- /dev/null
+++ b/statefun-docs/src/main/python/TypeHint.py
@@ -0,0 +1,37 @@
+################################################################################
+#  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.
+################################################################################
+
+import typing
+from statefun import StatefulFunctions
+
+functions = StatefulFunctions()
+
+@functions.bind("flink/hello")
+def hello_function(context, message: User):
+    """A simple hello world function with typing"""
+
+    print("Hello " + message.name)
+
+@function.bind("flink/goodbye")
+def goodbye_function(context, message: typing.Union[User, Admin]):
+    """A function that dispatches on types"""
+
+    if isinstance(message, User):
+        print("Goodbye user")
+    elif isinstance(message, Admin):
+        print("Goodbye Admin")
\ No newline at end of file
diff --git a/statefun-docs/src/main/resources/module.yaml b/statefun-docs/src/main/resources/module.yaml
new file mode 100644
index 0000000..e3e6425
--- /dev/null
+++ b/statefun-docs/src/main/resources/module.yaml
@@ -0,0 +1,31 @@
+# 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.
+# This file is the base for the Apache Flink configuration
+
+module:
+  meta:
+    type: remote
+  spec:
+    functions:
+    - function:
+        meta:
+          kind: http
+          type: flink/greeter
+        spec:
+          endpoint: http://greeter:8000/statefun
+          states:
+          - seen_count
+          maxNumBatchRequests: 500
+    timeout: 2m
\ No newline at end of file