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.
To get started, add the Java SDK as a dependency to your application.
A stateful function is any class that implements the StatefulFunction interface.
The following is an example of a simple hello world function.
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.
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.
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 definition.
Below is an example function type for the hello world function.
This type can then be referenced from other functions to create an address and message a particular instance.
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.
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.
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.
Attention: All PersistedValue, PersistedTable, and PersistedAppendingBuffer fields must be marked with an @Persisted annotation or they will not be made fault tolerant by the runtime.
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.
State Expiration
Persisted states may be configured to expire and be deleted after a specified duration.
This is supported by all types of state:
There are two expiration modes supported:
State TTL configurations are made fault-tolerant by the runtime. In the case of downtime, state entries that should have been removed during said downtime will be purged immediately on restart.
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.
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.