Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
30 changes: 30 additions & 0 deletions python/flink_agents/api/runner_context.py
Original file line number Diff line number Diff line change
Expand Up @@ -196,6 +196,7 @@ def durable_execute(
self,
func: Callable[[Any], Any],
*args: Any,
reconciler: Callable[[], Any] | None = None,
**kwargs: Any,
) -> Any:
"""Synchronously execute the provided function with durable execution support.
Expand All @@ -212,6 +213,16 @@ def durable_execute(
will always make the durable_execute call with the same arguments and in the
same order during job recovery. Otherwise, the behavior is undefined.

If `reconciler` is provided, recovery invokes it only when revisiting
this durable call and no terminal outcome from the previous durable
invocation has been persisted yet. The reconciler may:

* return a result to provide the recovered successful outcome for this
durable call; The runtime persists and replays that recovered result
* raise an exception to provide the recovered failed outcome for this
durable call; The runtime persists and replays that recovered
failure

Usage::

def my_action(event, ctx):
Expand All @@ -224,6 +235,10 @@ def my_action(event, ctx):
The function to be executed.
*args : Any
Positional arguments to pass to the function.
reconciler : Callable[[], Any] | None
Optional zero-argument reconciler callable used only during recovery.
This is a reserved keyword-only parameter and is not forwarded to
`func`.
**kwargs : Any
Keyword arguments to pass to the function.

Expand All @@ -238,6 +253,7 @@ def durable_execute_async(
self,
func: Callable[[Any], Any],
*args: Any,
reconciler: Callable[[], Any] | None = None,
**kwargs: Any,
) -> "AsyncExecutionResult":
"""Asynchronously execute the provided function with durable execution support.
Expand All @@ -251,6 +267,16 @@ def durable_execute_async(
will always make the durable_execute_async call with the same arguments and in
the same order during job recovery. Otherwise, the behavior is undefined.

If `reconciler` is provided, recovery invokes it only when revisiting
this durable call and no terminal outcome from the previous durable
invocation has been persisted yet. The reconciler may:

* return a result to provide the recovered successful outcome for this
durable call; The runtime persists and replays that recovered result
* raise an exception to provide the recovered failed outcome for this
durable call; The runtime persists and replays that recovered
failure

Usage::

async def my_action(event, ctx):
Expand All @@ -267,6 +293,10 @@ async def my_action(event, ctx):
The function to be executed asynchronously.
*args : Any
Positional arguments to pass to the function.
reconciler : Callable[[], Any] | None
Optional zero-argument reconciler callable used only during recovery.
This is a reserved keyword-only parameter and is not forwarded to
`func`.
**kwargs : Any
Keyword arguments to pass to the function.

Expand Down
75 changes: 75 additions & 0 deletions python/flink_agents/runtime/durable_execution.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,75 @@
################################################################################
# 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 hashlib
import inspect
from typing import Any, Callable

import cloudpickle


def _compute_function_id(func: Callable) -> str:
"""Compute a stable function identifier from a callable."""
module_obj = inspect.getmodule(func)
module = (
module_obj.__name__
if module_obj is not None
else getattr(func, "__module__", "<unknown>")
)
qualname = getattr(func, "__qualname__", getattr(func, "__name__", "<unknown>"))
return f"{module}.{qualname}"


def _compute_args_digest(args: tuple, kwargs: dict) -> str:
"""Compute a stable digest of the serialized arguments."""
try:
serialized = cloudpickle.dumps((args, kwargs))
return hashlib.sha256(serialized).hexdigest()[:16]
except Exception:
return hashlib.sha256(str((args, kwargs)).encode()).hexdigest()[:16]


def _can_bind_call(
func: Callable,
*args: Any,
**kwargs: Any,
) -> bool:
"""Return whether the callable signature can bind the provided arguments."""
try:
inspect.signature(func).bind(*args, **kwargs)
except (TypeError, ValueError):
return False
else:
return True


def _validate_reconciler_callable(
reconciler: Callable[[], Any] | None,
) -> Callable[[], Any] | None:
"""Validate that the reconciler callable is either absent or zero-argument."""
if reconciler is None:
return None

if not callable(reconciler):
err_msg = "reconciler must be callable"
raise TypeError(err_msg)

if not _can_bind_call(reconciler):
err_msg = "reconciler must be a callable that takes no arguments"
raise TypeError(err_msg)

return reconciler
Loading
Loading