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/26 14:21:17 UTC
[flink-statefun] 05/14: [FLINK-16758][docs] Port SDK content
This is an automated email from the ASF dual-hosted git repository.
tzulitai pushed a commit to branch release-2.0
in repository https://gitbox.apache.org/repos/asf/flink-statefun.git
commit 2b1ab3f90d7061ab1337a1af8fa67b46f9f92ff7
Author: Seth Wiesman <sj...@gmail.com>
AuthorDate: Tue Mar 24 13:48:45 2020 -0500
[FLINK-16758][docs] Port SDK content
---
docs/index.md | 8 +-
docs/sdk/index.md | 27 ++++
docs/sdk/java.md | 413 ++++++++++++++++++++++++++++++++++++++++++++++++++++
docs/sdk/modules.md | 125 ++++++++++++++++
docs/sdk/python.md | 268 ++++++++++++++++++++++++++++++++++
5 files changed, 837 insertions(+), 4 deletions(-)
diff --git a/docs/index.md b/docs/index.md
index 6ca7616..3d04c4f 100644
--- a/docs/index.md
+++ b/docs/index.md
@@ -29,7 +29,7 @@ under the License.
It brings together the benefits of stream processing with Apache Flink® and Function-as-a-Service (FaaS) to provide a powerful abstraction for the next generation of event-driven architectures.
<p class="text-center">
- <img width="80%" src="{{ site.baseurl }}/fig/stateful_functions.png"/>
+ <img width="80%" src="{{ site.baseurl }}/fig/stateful_functions.png"/>
</p>
There are many ways to get started with Stateful Functions.
@@ -38,16 +38,16 @@ Whether you prefer a more theoretical or a practical approach, we hope you’ll
## Learn By Doing
-If you prefer to learn by doing, start with our code :ref:`walkthrough <walkthrough>`.
+If you prefer to learn by doing, start with our code [walkthrough]({{ site.baseurl }}/getting-started/walkthrough.html).
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 <concepts>`.
+If you prefer to learn concepts step by step, start with our guide to [main concepts]({{ site.baseurl }}/concepts/application-building-blocks.html).
It will walk you through all the API's and concepts to build advanced stateful systems.
## Start A New Project
-The :ref:`project setup <project_setup>` instructions show you how to create a project for a new Stateful Functions application in just a few steps.
+The [project setup]({{ site.baseurl }}/getting-started/project-setup.html) instructions show you how to create a project for a new Stateful Functions application in just a few steps.
diff --git a/docs/sdk/index.md b/docs/sdk/index.md
new file mode 100644
index 0000000..c398744
--- /dev/null
+++ b/docs/sdk/index.md
@@ -0,0 +1,27 @@
+---
+title: SDK
+nav-id: sdk
+nav-pos: 3
+nav-title: 'SDK'
+nav-parent_id: root
+nav-show_overview: false
+permalink: /sdk/index.html
+---
+<!--
+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.
+-->
diff --git a/docs/sdk/java.md b/docs/sdk/java.md
new file mode 100644
index 0000000..1fda0d3
--- /dev/null
+++ b/docs/sdk/java.md
@@ -0,0 +1,413 @@
+---
+title: Java SDK
+nav-id: java-sdk
+nav-pos: 1
+nav-title: Java
+nav-parent_id: sdk
+---
+<!--
+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.
+-->
+
+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 [embedded_module]({{ site.baseurl }}/sdk/modules.html#embedded-module).
+
+To get started, add the Java SDK as a dependency to your application.
+
+{% highlight xml %}
+<dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>statefun-sdk</artifactId>
+ <version>{{ site.version }}</version>
+</dependency>
+{% endhighlight %}
+
+* This will be replaced by the TOC
+{:toc}
+
+## 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.
+
+{% highlight java %}
+package org.apache.flink.statefun.docs;
+
+import org.apache.flink.statefun.sdk.Context;
+import org.apache.flink.statefun.sdk.StatefulFunction;
+
+public class FnHelloWorld implements StatefulFunction {
+
+ @Override
+ public void invoke(Context context, Object input) {
+ System.out.println("Hello " + input.toString());
+ }
+}
+{% endhighlight %}
+
+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.
+
+### Stateful Match Function
+
+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.
+
+#### 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.
+
+{% highlight java %}
+package org.apache.flink.statefun.docs.match;
+
+import org.apache.flink.statefun.sdk.Context;
+import org.apache.flink.statefun.sdk.match.MatchBinder;
+import org.apache.flink.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());
+ }
+}
+{% endhighlight %}
+
+#### 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.
+
+{% highlight java %}
+package org.apache.flink.statefun.docs.match;
+
+import org.apache.flink.statefun.sdk.Context;
+import org.apache.flink.statefun.sdk.match.MatchBinder;
+import org.apache.flink.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");
+ }
+}
+{% endhighlight %}
+
+#### 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.
+
+## 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 [module]({{ site.baseurl }}/sdk/modules.html#embedded-module) definition.
+Below is an example function type for the hello world function.
+
+{% highlight java %}
+package org.apache.flink.statefun.docs;
+
+import org.apache.flink.statefun.sdk.FunctionType;
+
+/** A function type that will be bound to {@link FnHelloWorld}. */
+public class Identifiers {
+
+ public static final FunctionType HELLO_TYPE = new FunctionType("apache/flink", "hello");
+}
+{% endhighlight %}
+
+This type can then be referenced from other functions to create an address and message a particular instance.
+
+{% highlight java %}
+package org.apache.flink.statefun.docs;
+
+import org.apache.flink.statefun.sdk.Context;
+import org.apache.flink.statefun.sdk.StatefulFunction;
+
+/** A simple stateful function that sends a message to the user with id "user1" */
+public class FnCaller implements StatefulFunction {
+
+ @Override
+ public void invoke(Context context, Object input) {
+ context.send(Identifiers.HELLO_TYPE, "user1", new MyUserMessage());
+ }
+}
+{% endhighlight %}
+
+## 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.
+
+{% highlight java %}
+package org.apache.flink.statefun.docs.delay;
+
+import java.time.Duration;
+import org.apache.flink.statefun.sdk.Context;
+import org.apache.flink.statefun.sdk.StatefulFunction;
+
+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!");
+ }
+ }
+}
+{% endhighlight %}
+
+## 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.
+
+{% highlight java %}
+package org.apache.flink.statefun.docs.async;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.flink.statefun.sdk.AsyncOperationResult;
+import org.apache.flink.statefun.sdk.Context;
+import org.apache.flink.statefun.sdk.StatefulFunction;
+
+@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<UserEnrichment> future = client.getDataAsync(user.getUserId());
+ context.registerAsyncOperation(user, future);
+ }
+
+ private void onAsyncResult(AsyncOperationResult<User, UserEnrichment> 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");
+ }
+ }
+}
+{% endhighlight %}
+
+## 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 defining one or more persisted fields.
+
+The simplest way to get started is with a ``PersistedValue``, which 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.
+
+<div class="alert alert-info">
+ <strong>Attention:</strong> All PersistedValue, PersistedTable, and PersistedAppendingBuffer fields must be marked with an @Persisted annotation or they will not be made fault tolerant by the runtime.
+</div>
+
+{% highlight java %}
+package org.apache.flink.statefun.docs;
+
+import org.apache.flink.statefun.sdk.Context;
+import org.apache.flink.statefun.sdk.FunctionType;
+import org.apache.flink.statefun.sdk.StatefulFunction;
+import org.apache.flink.statefun.sdk.annotations.Persisted;
+import org.apache.flink.statefun.sdk.state.PersistedValue;
+
+public class FnUserGreeter implements StatefulFunction {
+
+ public static FunctionType TYPE = new FunctionType("example", "greeter");
+
+ @Persisted
+ private final PersistedValue<Integer> 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);
+ }
+}
+{% endhighlight %}
+
+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.
+
+### Collection Types
+
+Along with ``PersistedValue``, the Java SDK supports two persisted collection types.
+``PersistedTable`` is a collection of keys and values, and ``PersistedAppendingBuffer`` is an append-only buffer.
+
+These types are functionally equivalent to ``PersistedValue<Map>`` and ``PersistedValue<Collection>`` respectively but may provide better performance in some situations.
+
+{% highlight java %}
+@Persisted
+PersistedTable<String, Integer> table = PersistedTable.of("my-table", String.class, Integer.class);
+
+@Persisted
+PersistedAppendingBuffer<Integer> buffer = PersistedAppendingBuffer.of("my-buffer", Integer.class);
+{% endhighlight %}
+
+## 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.
+
+{% highlight java %}
+package org.apache.flink.statefun.docs;
+
+import org.apache.flink.statefun.docs.dependency.ProductionDependency;
+import org.apache.flink.statefun.docs.dependency.RuntimeDependency;
+import org.apache.flink.statefun.sdk.FunctionType;
+import org.apache.flink.statefun.sdk.StatefulFunction;
+import org.apache.flink.statefun.sdk.StatefulFunctionProvider;
+
+public class CustomProvider implements StatefulFunctionProvider {
+
+ public StatefulFunction functionOfType(FunctionType type) {
+ RuntimeDependency dependency = new ProductionDependency();
+ return new FnWithDependency(dependency);
+ }
+}
+{% endhighlight %}
+
+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.
+
+{% highlight java %}
+package org.apache.flink.statefun.docs;
+
+import org.apache.flink.statefun.docs.dependency.RuntimeDependency;
+import org.apache.flink.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);
+ }
+}
+{% endhighlight %}
diff --git a/docs/sdk/modules.md b/docs/sdk/modules.md
new file mode 100644
index 0000000..2f980ac
--- /dev/null
+++ b/docs/sdk/modules.md
@@ -0,0 +1,125 @@
+---
+title: Modules
+nav-id: modules
+nav-pos: 3
+nav-title: Modules
+nav-parent_id: sdk
+---
+<!--
+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.
+-->
+
+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.
+
+* This will be replaced by the TOC
+{:toc}
+
+## 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 [function type]({{ site.baseurl }}/concepts/logical.html#function-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``.
+
+{% highlight java %}
+package org.apache.flink.statefun.docs;
+
+import java.util.Map;
+import org.apache.flink.statefun.sdk.spi.StatefulFunctionModule;
+
+public class BasicFunctionModule implements StatefulFunctionModule {
+
+ public void configure(Map<String, String> 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(Identifiers.HELLO_TYPE, unused -> new FnHelloWorld());
+ }
+}
+{% endhighlight %}
+
+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.
+
+{% highlight none %}
+org.apache.flink.statefun.docs.BasicFunctionModule
+{% endhighlight %}
+
+## 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.
+
+### Specification
+
+A remote module configuration consists of a ``meta`` section and a ``spec`` section.
+``meta`` contains auxillary information about the module.
+The ``spec`` describes the functions contained within the module and defines their persisted values.
+
+### Defining Functions
+
+``module.spec.functions`` declares a list of ``function`` objects that are implemented by the remote module.
+A ``function`` is described via a number of properties.
+
+* ``function.meta.kind``
+ * The protocol used to communicate with the remote function.
+ * Supported Values - ``http``
+* ``function.meta.type``
+ * The function type, defined as ``<namespace>/<name>``.
+* ``function.spec.endpoint``
+ * The endpoint at which the function is reachable.
+* ``function.spec.states``
+ * A list of the names of the persisted values decalred within the remote function.
+* ``function.spec.maxNumBatchRequests``
+ * The maximum number of records that can be processed by a function for a particular ``address`` before invoking backpressure on the system.
+ * Default - 1000
+* ``function.spec.timeout``
+ * The maximum amount of time for the runtime to wait for the remote function to return before failing.
+ * Default - 1 min
+
+### Full Example
+
+{% highlight yaml %}
+version: "1.0"
+
+module:
+ meta:
+ type: remote
+ spec:
+ functions:
+ - function:
+ meta:
+ kind: http
+ type: example/greeter
+ spec:
+ endpoint: http://<host-name>/statefun
+ states:
+ - seen_count
+ maxNumBatchRequests: 500
+ timeout: 2min
+{% endhighlight %}
diff --git a/docs/sdk/python.md b/docs/sdk/python.md
new file mode 100644
index 0000000..d9eaefc
--- /dev/null
+++ b/docs/sdk/python.md
@@ -0,0 +1,268 @@
+---
+title: Python SDK
+nav-id: python-sdk
+nav-pos: 1
+nav-title: Python
+nav-parent_id: sdk
+---
+<!--
+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.
+-->
+
+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 [remote module]({{ site.baseurl}}/sdk/modules.html#remote-module).
+
+To get started, add the Python SDK as a dependency to your application.
+
+{% highlight bash %}
+apache-flink-statefun=={{ site.version }}
+{% endhighlight %}
+
+* This will be replaced by the TOC
+{:toc}
+
+## 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.
+
+{% highlight python %}
+from statefun import StatefulFunctions
+
+functions = StatefulFunctions()
+
+@functions.bind("example/hello")
+def hello_function(context, message):
+ """A simple hello world function"""
+ user = User()
+ message.Unpack(user)
+
+ print("Hello " + user.name)
+{% endhighlight %}
+
+This code declares a function with in the namespace ``example`` 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 [bottom of this page]({{ site.baseurl }}/sdk/python.html#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.
+
+{% highlight python %}
+import typing
+from statefun import StatefulFunctions
+
+functions = StatefulFunctions()
+
+@functions.bind("example/hello")
+def hello_function(context, message: User):
+ """A simple hello world function with typing"""
+
+ print("Hello " + message.name)
+
+@function.bind("example/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")
+{% endhighlight %}
+
+## 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.
+
+{% highlight python %}
+from google.protobuf.any_pb2 import Any
+from statefun import StatefulFunctions
+
+functions = StatefulFunctions()
+
+@functions.bind("example/caller")
+def caller_function(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("example/hello", user.user_id, envelope)
+{% endhighlight %}
+
+Alternatively, functions can be manually bound to the runtime.
+
+{% highlight python %}
+functions.register("example/caller", caller_function)
+{% endhighlight %}
+
+## 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).
+
+{% highlight python %}
+from google.protobuf.any_pb2 import Any
+from statefun import StatefulFunctions
+
+functions = StatefulFunctions()
+
+@functions.bind("example/caller")
+def caller_function(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("example/hello", user.user_id, envelope)
+{% endhighlight %}
+
+## 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``.
+
+<div class="alert alert-info">
+ <strong>Attention:</strong> [Remote modules]({{ site.baseurl}}/sdk/modules.html#remote-module) require that all state values are eagerly registered at module.yaml.
+</div>
+
+Below is a stateful function that greets users based on the number of times they have been seen.
+
+{% highlight python %}
+from google.protobuf.any_pb2 import Any
+from statefun import StatefulFunctions
+
+functions = StatefulFunctions()
+
+@functions.bind("example/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"
+{% endhighlight %}
+
+Additionally, persisted values may be cleared by deleting its value.
+
+{% highlight python %}
+del context["count"]
+{% endhighlight %}
+
+## 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.
+
+{% highlight python %}
+from statefun import RequestReplyHandler
+
+handler RequestReplyHandler(functions)
+{% endhighlight %}
+
+
+### 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``.
+
+{% highlight 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()
+{% endhighlight %}
+
+## 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.