diff --git a/dttools/src/progress_bar.c b/dttools/src/progress_bar.c index 614d4f7439..ab3478310b 100644 --- a/dttools/src/progress_bar.c +++ b/dttools/src/progress_bar.c @@ -253,6 +253,24 @@ void progress_bar_update_part(struct ProgressBar *bar, struct ProgressBarPart *p } } +/** Set the current value for a part (clamped to total). */ +void progress_bar_set_part_current(struct ProgressBar *bar, struct ProgressBarPart *part, uint64_t current) +{ + if (!bar || !part) { + return; + } + + if (current > part->total) { + current = part->total; + } + part->current = current; + + timestamp_t now_us = timestamp_get(); + if (!bar->has_drawn_once || (now_us - bar->last_draw_time_us) >= bar->update_interval_us) { + print_progress_bar(bar); + } +} + /** Set the start time for the progress bar. */ void progress_bar_set_start_time(struct ProgressBar *bar, timestamp_t start_time) { diff --git a/dttools/src/progress_bar.h b/dttools/src/progress_bar.h index 7fe3171a3f..6c4f31b414 100644 --- a/dttools/src/progress_bar.h +++ b/dttools/src/progress_bar.h @@ -77,6 +77,13 @@ void progress_bar_set_part_total(struct ProgressBar *bar, struct ProgressBarPart */ void progress_bar_update_part(struct ProgressBar *bar, struct ProgressBarPart *part, uint64_t increment); +/** Set the current value for a part (clamped to [0, total]) and redraw if the update interval allows. +@param bar Progress bar. +@param part Part to update. +@param current New absolute current value. +*/ +void progress_bar_set_part_current(struct ProgressBar *bar, struct ProgressBarPart *part, uint64_t current); + /** Set the start time for the progress bar. @param bar Progress bar. @param start_time Start timestamp. diff --git a/poncho/src/poncho/library_network_code.py b/poncho/src/poncho/library_network_code.py index a5bedebc30..38a4524309 100755 --- a/poncho/src/poncho/library_network_code.py +++ b/poncho/src/poncho/library_network_code.py @@ -28,6 +28,9 @@ r, w = os.pipe() exec_method = None +# infile load mode for function tasks inside this library +function_infile_load_mode = None + # This class captures how results from FunctionCalls are conveyed from # the library to the manager. @@ -85,6 +88,18 @@ def sigchld_handler(signum, frame): os.write(w, b"a") +# Load the infile for a function task inside this library +def load_function_infile(in_file_path): + if function_infile_load_mode == "cloudpickle": + with open(in_file_path, "rb") as f: + return cloudpickle.load(f) + elif function_infile_load_mode == "json": + with open(in_file_path, "r", encoding="utf-8") as f: + return json.load(f) + else: + raise ValueError(f"invalid infile load mode: {function_infile_load_mode}") + + # Read data from worker, start function, and dump result to `outfile`. def start_function(in_pipe_fd, thread_limit=1): # read length of buffer to read @@ -131,8 +146,7 @@ def start_function(in_pipe_fd, thread_limit=1): os.chdir(function_sandbox) # parameters are represented as infile. - with open("infile", "rb") as f: - event = cloudpickle.load(f) + event = load_function_infile("infile") # output of execution should be dumped to outfile. result = globals()[function_name](event) @@ -158,11 +172,10 @@ def start_function(in_pipe_fd, thread_limit=1): return -1, function_id elif exec_method == "fork": try: - arg_infile = os.path.join(function_sandbox, "infile") - with open(arg_infile, "rb") as f: - event = cloudpickle.load(f) + infile_path = os.path.join(function_sandbox, "infile") + event = load_function_infile(infile_path) except Exception: - stdout_timed_message(f"TASK {function_id} error: can't load the arguments from {arg_infile}") + stdout_timed_message(f"TASK {function_id} error: can't load the arguments from {infile_path}") return -1, function_id p = os.fork() if p == 0: @@ -368,11 +381,16 @@ def main(): global exec_method exec_method = library_info['exec_mode'] + # set infile load mode of functions in this library + global function_infile_load_mode + function_infile_load_mode = library_info['function_infile_load_mode'] + # send configuration of library, just its name for now config = { "name": library_info['library_name'], "taskid": args.task_id, "exec_mode": exec_method, + "function_infile_load_mode": function_infile_load_mode, } send_configuration(config, out_pipe_fd, args.worker_pid) diff --git a/poncho/src/poncho/package_serverize.py b/poncho/src/poncho/package_serverize.py index 4a6e5e7a29..cfc789a11b 100755 --- a/poncho/src/poncho/package_serverize.py +++ b/poncho/src/poncho/package_serverize.py @@ -178,6 +178,7 @@ def pack_library_code(path, envpath): # @param exec_mode The execution mode of functions in this library. # @param hoisting_modules A list of modules imported at the preamble of library, including packages, functions and classes. # @param library_context_info A list containing [library_context_func, library_context_args, library_context_kwargs]. Used to create the library context on remote nodes. +# @param function_infile_load_mode The mode to load infile for function tasks inside this library. # @return A hash value. def generate_library_hash(library_name, function_list, @@ -186,7 +187,8 @@ def generate_library_hash(library_name, add_env, exec_mode, hoisting_modules, - library_context_info): + library_context_info, + function_infile_load_mode): library_info = [library_name] function_list = list(function_list) function_names = set() @@ -234,6 +236,8 @@ def generate_library_hash(library_name, for kwarg in library_context_info[2]: library_info.append(str(kwarg)) library_info.append(str(library_context_info[2][kwarg])) + + library_info.append(str(function_infile_load_mode)) library_info = ''.join(library_info) # linear time complexity msg = hashlib.sha1() @@ -293,6 +297,7 @@ def generate_taskvine_library_code(library_path, hoisting_modules=None): # @param exec_mode execution mode of functions in this library # @param hoisting_modules a list of modules to be imported at the preamble of library # @param library_context_info a list containing a library's context to be created remotely +# @param function_infile_load_mode The mode to load infile for function tasks inside this library. # @return name of the file containing serialized information about the library def generate_library(library_cache_path, library_code_path, @@ -303,7 +308,8 @@ def generate_library(library_cache_path, need_pack=True, exec_mode='fork', hoisting_modules=None, - library_context_info=None + library_context_info=None, + function_infile_load_mode='cloudpickle' ): # create library_info.clpk library_info = {} @@ -313,6 +319,7 @@ def generate_library(library_cache_path, library_info['library_name'] = library_name library_info['exec_mode'] = exec_mode library_info['context_info'] = cloudpickle.dumps(library_context_info) + library_info['function_infile_load_mode'] = function_infile_load_mode with open(library_info_path, 'wb') as f: cloudpickle.dump(library_info, f) diff --git a/taskvine/.gitignore b/taskvine/.gitignore new file mode 100644 index 0000000000..50f0e4da1e --- /dev/null +++ b/taskvine/.gitignore @@ -0,0 +1 @@ +taskvine-insights-blogs \ No newline at end of file diff --git a/taskvine/src/Makefile b/taskvine/src/Makefile index 8f828fd7bf..1bd3ef4f43 100644 --- a/taskvine/src/Makefile +++ b/taskvine/src/Makefile @@ -1,12 +1,13 @@ include ../../config.mk include ../../rules.mk -TARGETS=manager worker tools bindings examples +TARGETS=manager worker tools bindings examples graph all: $(TARGETS) worker: manager -bindings: manager +graph: manager +bindings: manager graph tools: manager examples: manager worker tools bindings diff --git a/taskvine/src/bindings/python3/Makefile b/taskvine/src/bindings/python3/Makefile index ca4ca6a52b..79b6a3ccd5 100644 --- a/taskvine/src/bindings/python3/Makefile +++ b/taskvine/src/bindings/python3/Makefile @@ -10,6 +10,7 @@ LOCAL_LINKAGE = $(CCTOOLS_PYTHON3_LDFLAGS) -lz $(CCTOOLS_OPENSSL_LDFLAGS) $(CCTO CCTOOLS_FLAKE8_IGNORE_FILES = "cvine.py" DSPYTHONSO = ndcctools/taskvine/_cvine.$(CCTOOLS_DYNAMIC_SUFFIX) +DAGVINE_MODULE_DIR = $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/dagvine LIBRARIES = $(DSPYTHONSO) OBJECTS = vine_wrap.o TARGETS = $(LIBRARIES) @@ -29,9 +30,15 @@ lint: clean: rm -rf $(OBJECTS) $(TARGETS) ndcctools/taskvine/cvine.py vine_wrap.c vine_wrap.o *.pyc __pycache__ + rm -rf ndcctools/taskvine/dagvine/__pycache__ + find ndcctools/taskvine/dagvine -name __pycache__ -type d -prune -exec rm -rf {} + install: all mkdir -p $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/compat cp ndcctools/taskvine/*.py $(DSPYTHONSO) $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine cp ndcctools/taskvine/compat/*.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/compat + rm -rf $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/vine_graph + rm -rf $(DAGVINE_MODULE_DIR) + cd ndcctools/taskvine/dagvine && find . -path ./test -prune -o -type d -print | while read dir; do mkdir -p "$(DAGVINE_MODULE_DIR)/$$dir"; done + cd ndcctools/taskvine/dagvine && find . -path ./test -prune -o -name '*.py' -print | while read file; do cp "$$file" "$(DAGVINE_MODULE_DIR)/$$file"; done cp taskvine.py $(CCTOOLS_PYTHON3_PATH)/ diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/__init__.py b/taskvine/src/bindings/python3/ndcctools/taskvine/__init__.py index a2a02ed095..09adf9d491 100644 --- a/taskvine/src/bindings/python3/ndcctools/taskvine/__init__.py +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/__init__.py @@ -77,6 +77,13 @@ class DaskVineWarning(UserWarning): from .compat import DaskVine from .compat import DaskVineDag +try: + from . import dagvine as _dagvine + VineGraph = _dagvine.VineGraph + VineGraphDaskAdaptor = _dagvine.VineGraphDaskAdaptor +except (ImportError, ModuleNotFoundError): + pass + __all__ = [ "Manager", @@ -95,6 +102,11 @@ class DaskVineWarning(UserWarning): "DaskVineDag", ] +if "VineGraph" in globals(): + __all__.append("VineGraph") +if "VineGraphDaskAdaptor" in globals(): + __all__.append("VineGraphDaskAdaptor") + __version__ = cvine.vine_version_string() # vim: set sts=4 sw=4 ts=4 expandtab ft=python: diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/__init__.py b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/__init__.py new file mode 100644 index 0000000000..060c67c9b7 --- /dev/null +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/__init__.py @@ -0,0 +1,8 @@ +# Copyright (C) 2025- The University of Notre Dame +# This software is distributed under the GNU General Public License. +# See the file COPYING for details. + +from .vine_graph import VineGraph, VineGraphConfig +from .dask_adaptor import VineGraphDaskAdaptor + +__all__ = ["VineGraph", "VineGraphConfig", "VineGraphDaskAdaptor"] diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/dask_adaptor/.gitignore b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/dask_adaptor/.gitignore new file mode 100644 index 0000000000..c18dd8d83c --- /dev/null +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/dask_adaptor/.gitignore @@ -0,0 +1 @@ +__pycache__/ diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/dask_adaptor/__init__.py b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/dask_adaptor/__init__.py new file mode 100644 index 0000000000..7b228acbee --- /dev/null +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/dask_adaptor/__init__.py @@ -0,0 +1,3 @@ +from .adaptor import VineGraphDaskAdaptor, workflow_to_dask_graph + +__all__ = ["VineGraphDaskAdaptor", "workflow_to_dask_graph"] diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/dask_adaptor/adaptor.py b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/dask_adaptor/adaptor.py new file mode 100644 index 0000000000..19f7f3df7f --- /dev/null +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/dask_adaptor/adaptor.py @@ -0,0 +1,167 @@ +from collections.abc import Mapping + +try: + import dask +except ImportError: + dask = None + +try: + from dask.base import is_dask_collection +except ImportError: + is_dask_collection = None + +try: + import importlib + + dts = importlib.import_module("dask._task_spec") +except Exception: + dts = None + +from .legacy import expand_legacy_subgraph_dsk +from .task_spec import ( + DaskTaskSpecConverter, + _build_expr, + identity, + _resolve_graph_key_if_task, +) +from ..workflow import TaskOutputRef, Workflow + + +_identity = identity + + +def _apply_with_kwargs_kvlist(func, args_list, kwargs_kvlist): + """Call ``func`` when kwargs are encoded as ``[[key, value], ...]``.""" + return func(*args_list, **{key: value for key, value in kwargs_kvlist}) + + +def workflow_to_dask_graph(workflow): + assert isinstance(workflow, Workflow), "workflow must be a Workflow" + + def ref_to_key(ref): + return ref.workflow_key + + dsk = {} + for workflow_key, (func_id, args, kwargs) in workflow.task_dict.items(): + func = workflow.callables[func_id] + new_args = workflow._visit_task_output_refs(args, ref_to_key, rewrite=True) + new_kwargs = workflow._visit_task_output_refs(kwargs, ref_to_key, rewrite=True) + + if new_kwargs: + dsk[workflow_key] = (_apply_with_kwargs_kvlist, func, list(new_args), [[k, v] for k, v in new_kwargs.items()]) + else: + dsk[workflow_key] = (func, *new_args) + + return dsk + + +class VineGraphDaskAdaptor: + """Convert Dask graph forms into VineGraph ``Workflow`` task expressions.""" + + def __init__(self, task_dict, expand_subgraphs=False): + if isinstance(task_dict, Workflow): + self.converted = task_dict + return + + self._expand_subgraphs = expand_subgraphs + normalized = self._normalize_task_dict(task_dict) + self.converted = self._convert_to_workflow_tasks(normalized) + + @property + def task_dict(self): + return self.converted + + def _normalize_task_dict(self, task_dict): + if self._is_dask_collection_dict(task_dict): + task_dict = self._dask_collections_to_task_dict(task_dict) + else: + # Plain dicts are VineGraph sexprs; don't let Dask reinterpret kwargs dicts. + task_dict = dict(task_dict) + + if self._expand_subgraphs and not dts and task_dict: + task_dict = expand_legacy_subgraph_dsk(task_dict, dask) + return task_dict + + def _is_dask_collection_dict(self, task_dict): + return bool(is_dask_collection and any(is_dask_collection(value) for value in task_dict.values())) + + def _dask_collections_to_task_dict(self, task_dict): + assert is_dask_collection is not None + from dask.highlevelgraph import HighLevelGraph, ensure_dict + + if not isinstance(task_dict, dict): + raise TypeError("Input must be a dict") + for key, value in task_dict.items(): + if not is_dask_collection(value): + raise TypeError(f"Input must be a dict of DaskCollection, but found {key} with type {type(value)}") + + if dts: + hlg = HighLevelGraph.merge(*(value.dask for value in task_dict.values())).to_dict() + else: + hlg = dask.base.collections_to_dsk(task_dict.values()).to_dict() + return ensure_dict(hlg) + + def _convert_to_workflow_tasks(self, task_dict): + if not task_dict: + return {} + + converted = {} + workflow_keys = set(task_dict.keys()) + task_spec = DaskTaskSpecConverter(dts) if dts else None + + for key, value in task_dict.items(): + if task_spec and task_spec.is_node(value): + converted[key] = task_spec.convert_node(key, value, workflow_keys) + else: + converted[key] = self._convert_legacy_task(value, workflow_keys) + + if task_spec: + while True: + pending = task_spec.pending_nodes(converted) + if not pending: + break + for key, node in pending: + converted[key] = task_spec.convert_node(key, node, workflow_keys) + + return converted + + def _convert_legacy_task(self, sexpr, workflow_keys): + try: + if not isinstance(sexpr, (list, tuple)) and sexpr in workflow_keys: + return _build_expr(identity, [TaskOutputRef(sexpr)], {}) + except TypeError: + pass + + if not isinstance(sexpr, (list, tuple)) or not sexpr: + raise TypeError(f"Task definition must be a non-empty tuple/list, got {type(sexpr)}") + + func = sexpr[0] + tail = sexpr[1:] + if tail and isinstance(tail[-1], Mapping): + raw_args, raw_kwargs = tail[:-1], tail[-1] + else: + raw_args, raw_kwargs = tail, {} + + args = tuple(self._wrap_dependency(arg, workflow_keys) for arg in raw_args) + kwargs = {key: self._wrap_dependency(value, workflow_keys) for key, value in raw_kwargs.items()} + return func, args, kwargs + + def _wrap_dependency(self, obj, workflow_keys): + if isinstance(obj, TaskOutputRef): + return obj + + key = _resolve_graph_key_if_task(obj, workflow_keys) + if key is not None: + return TaskOutputRef(key) + + if isinstance(obj, list): + return [self._wrap_dependency(value, workflow_keys) for value in obj] + if isinstance(obj, tuple): + return tuple(self._wrap_dependency(value, workflow_keys) for value in obj) + if isinstance(obj, Mapping): + return {key: self._wrap_dependency(value, workflow_keys) for key, value in obj.items()} + if isinstance(obj, set): + return {self._wrap_dependency(value, workflow_keys) for value in obj} + if isinstance(obj, frozenset): + return frozenset(self._wrap_dependency(value, workflow_keys) for value in obj) + return obj diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/dask_adaptor/legacy.py b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/dask_adaptor/legacy.py new file mode 100644 index 0000000000..0dee0316d9 --- /dev/null +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/dask_adaptor/legacy.py @@ -0,0 +1,64 @@ +import hashlib + + +def _legacy_subgraph_key(*parts): + """Stable short key for expanded legacy Dask subgraph tasks.""" + return hashlib.sha256("".join(str(p) for p in parts).encode("utf-8")).hexdigest()[:20] + + +def _rewrite_subgraph_expr(dsk_key, inner_dsk, expr, blockwise_args): + try: + if expr in inner_dsk: + return _legacy_subgraph_key(dsk_key, expr) + except Exception: + pass + + if hasattr(expr, "item") and callable(expr.item): + try: + item = expr.item() + if item in inner_dsk: + return _legacy_subgraph_key(dsk_key, item) + except Exception: + pass + + if isinstance(expr, list): + return [_rewrite_subgraph_expr(dsk_key, inner_dsk, item, blockwise_args) for item in expr] + if isinstance(expr, tuple): + return tuple(_rewrite_subgraph_expr(dsk_key, inner_dsk, item, blockwise_args) for item in expr) + if isinstance(expr, str) and expr.startswith("__dask_blockwise__"): + return blockwise_args[int(expr.split("__")[-1])] + return expr + + +def expand_legacy_subgraph_dsk(task_dict, dask_module): + """Inline legacy ``SubgraphCallable`` layers into a flat dsk.""" + if not task_dict or dask_module is None: + return task_dict + + try: + from dask.optimization import SubgraphCallable + except ImportError: + return task_dict + + expanded = {} + for key, sexpr in task_dict.items(): + if not isinstance(sexpr, (tuple, list)) or not sexpr: + expanded[key] = sexpr + continue + + head = sexpr[0] + tail = sexpr[1:] + if isinstance(head, SubgraphCallable): + expanded[key] = _legacy_subgraph_key(key, head.outkey) + for sub_key, sub_sexpr in head.dsk.items(): + rewritten_key = _legacy_subgraph_key(key, sub_key) + expanded[rewritten_key] = _rewrite_subgraph_expr(key, head.dsk, sub_sexpr, tail) + elif callable(head): + expanded[key] = sexpr + else: + raise TypeError( + f"Legacy dsk task {key!r} has non-callable head {type(head).__name__}; " + "expected SubgraphCallable or a callable." + ) + + return expanded diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/dask_adaptor/task_spec.py b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/dask_adaptor/task_spec.py new file mode 100644 index 0000000000..a76a8c5578 --- /dev/null +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/dask_adaptor/task_spec.py @@ -0,0 +1,361 @@ +import hashlib +from collections.abc import Mapping + +from ..workflow import TaskOutputRef + + +def identity(value): + """Return ``value`` unchanged.""" + return value + + +def _build_expr(func, args, kwargs): + return func, tuple(args), dict(kwargs) + + +def _safe_repr(value, limit=800): + """Best-effort repr that stays compact for large graphs.""" + try: + text = repr(value) + except Exception as exc: + text = f"" + if limit and len(text) > limit: + return text[:limit] + "..." + return text + + +def _resolve_graph_key_if_task(obj, workflow_keys): + """Return the matching graph key when ``obj`` denotes an existing Dask task.""" + if isinstance(obj, TaskOutputRef): + return None + try: + if obj in workflow_keys: + return obj + except TypeError: + pass + if hasattr(obj, "item") and callable(obj.item): + try: + item = obj.item() + if item in workflow_keys: + return item + except Exception: + pass + return None + + +def _extract_callable_from_task(node): + for attr in ("function", "op", "callable", "func", "operation", "callable_obj"): + if not hasattr(node, attr): + continue + value = getattr(node, attr) + if value is not None and (callable(value) or hasattr(value, "__call__")): + return value + if hasattr(node, "__call__") and callable(node): + return node + return None + + +def _is_pure_value_op(func): + return func in (dict, list, tuple, set, frozenset) + + +def _is_identity_cast_op(func): + name = getattr(func, "__name__", None) + module = getattr(func, "__module__", None) + return bool(name == "_identity_cast" and module and module.startswith("dask")) + + +def _is_too_large_inline_value(value, *, max_container_len=2000): + try: + return isinstance(value, (list, tuple, set, frozenset, dict)) and len(value) > max_container_len + except Exception: + return False + + +class DaskTaskSpecConverter: + """Convert modern Dask TaskSpec nodes into Workflow task expressions.""" + + def __init__(self, dts_module): + self.dts = dts_module + self.lifted_nodes = {} + self._lift_cache = {} + self._lift_counter = 0 + + def is_node(self, value): + if not self.dts: + return False + try: + return isinstance(value, self.dts.GraphNode) + except AttributeError: + return False + + def pending_nodes(self, converted): + return [(key, node) for key, node in self.lifted_nodes.items() if key not in converted] + + def convert_node(self, key, node, workflow_keys): + if not self.dts: + raise RuntimeError("Dask TaskSpec support unavailable: dask._task_spec is not installed") + + task_cls = getattr(self.dts, "Task", None) + alias_cls = getattr(self.dts, "Alias", None) + literal_cls = getattr(self.dts, "Literal", None) + datanode_cls = getattr(self.dts, "DataNode", None) + nested_cls = getattr(self.dts, "NestedContainer", None) + taskref_cls = getattr(self.dts, "TaskRef", None) + + if task_cls and isinstance(node, task_cls): + return self._convert_task_node(key, node, workflow_keys) + if alias_cls and isinstance(node, alias_cls): + alias_ref = self._extract_alias_target(node, workflow_keys) + if alias_ref is None: + raise ValueError(f"Alias {key} is missing a resolvable upstream task") + return _build_expr(identity, [alias_ref], {}) + if datanode_cls and isinstance(node, datanode_cls): + return _build_expr(identity, [node.value], {}) + if literal_cls and isinstance(node, literal_cls): + return _build_expr(identity, [node.value], {}) + if taskref_cls and isinstance(node, taskref_cls): + return _build_expr(identity, [TaskOutputRef(node.key, getattr(node, "path", ()) or ())], {}) + if nested_cls and isinstance(node, nested_cls): + payload = getattr(node, "value", None) + if payload is None: + payload = getattr(node, "data", None) + return _build_expr(identity, [self.unwrap_operand(payload, workflow_keys, parent_key=key)], {}) + return _build_expr(identity, [node], {}) + + def _convert_task_node(self, key, node, workflow_keys): + func = _extract_callable_from_task(node) + if func is None: + raise TypeError(f"Task {key} is missing a callable function/op attribute") + + raw_args = getattr(node, "args", ()) or () + raw_kwargs = getattr(node, "kwargs", {}) or {} + + args = [] + try: + for index, arg in enumerate(raw_args): + args.append(self.unwrap_operand(arg, workflow_keys, parent_key=key)) + except Exception as exc: + raise TypeError( + "Failed to adapt TaskSpec node argument while converting to Workflow.\n" + f"- parent_workflow_key: {key!r}\n" + f"- func: {_safe_repr(func)}\n" + f"- arg_index: {index}\n" + f"- arg_value: {_safe_repr(arg)}\n" + f"- raw_args: {_safe_repr(raw_args)}\n" + f"- raw_kwargs: {_safe_repr(raw_kwargs)}" + ) from exc + + kwargs = {} + try: + for kwarg_key, value in raw_kwargs.items(): + kwargs[kwarg_key] = self.unwrap_operand(value, workflow_keys, parent_key=key) + except Exception as exc: + raise TypeError( + "Failed to adapt TaskSpec node kwarg while converting to Workflow.\n" + f"- parent_workflow_key: {key!r}\n" + f"- func: {_safe_repr(func)}\n" + f"- kwarg_key: {kwarg_key!r}\n" + f"- kwarg_value: {_safe_repr(value)}\n" + f"- raw_args: {_safe_repr(raw_args)}\n" + f"- raw_kwargs: {_safe_repr(raw_kwargs)}" + ) from exc + + return _build_expr(func, args, kwargs) + + def unwrap_operand(self, operand, workflow_keys, *, parent_key=None): + taskref_cls = getattr(self.dts, "TaskRef", None) + if taskref_cls and isinstance(operand, taskref_cls): + return TaskOutputRef(getattr(operand, "key", None), getattr(operand, "path", ()) or ()) + + alias_cls = getattr(self.dts, "Alias", None) + if alias_cls and isinstance(operand, alias_cls): + alias_ref = self._extract_alias_target(operand, workflow_keys) + if alias_ref is None: + raise ValueError("Alias node is missing a valid upstream source") + return alias_ref + + literal_cls = getattr(self.dts, "Literal", None) + if literal_cls and isinstance(operand, literal_cls): + return getattr(operand, "value", None) + + datanode_cls = getattr(self.dts, "DataNode", None) + if datanode_cls and isinstance(operand, datanode_cls): + return operand.value + + nested_cls = getattr(self.dts, "NestedContainer", None) + if nested_cls and isinstance(operand, nested_cls): + payload = getattr(operand, "value", None) + if payload is None: + payload = getattr(operand, "data", None) + return self.unwrap_operand(payload, workflow_keys, parent_key=parent_key) + + task_cls = getattr(self.dts, "Task", None) + if task_cls and isinstance(operand, task_cls): + return self._unwrap_task_operand(operand, workflow_keys, parent_key=parent_key) + + if isinstance(operand, list): + return [self.unwrap_operand(value, workflow_keys, parent_key=parent_key) for value in operand] + if isinstance(operand, tuple): + return tuple(self.unwrap_operand(value, workflow_keys, parent_key=parent_key) for value in operand) + if isinstance(operand, Mapping): + return {key: self.unwrap_operand(value, workflow_keys, parent_key=parent_key) for key, value in operand.items()} + if isinstance(operand, set): + return {self.unwrap_operand(value, workflow_keys, parent_key=parent_key) for value in operand} + if isinstance(operand, frozenset): + return frozenset(self.unwrap_operand(value, workflow_keys, parent_key=parent_key) for value in operand) + return operand + + def _unwrap_task_operand(self, operand, workflow_keys, *, parent_key=None): + inline_key = getattr(operand, "key", None) + if inline_key is not None and inline_key in workflow_keys: + return TaskOutputRef(inline_key, ()) + + func = _extract_callable_from_task(operand) + if func is None: + return self._lift_inline_task(operand, workflow_keys, parent_key=parent_key) + if _is_identity_cast_op(func): + return self._unwrap_identity_cast(operand, workflow_keys, parent_key=parent_key) + if _is_pure_value_op(func): + reduced, used_lift = self._reduce_inline_task(operand, workflow_keys, parent_key=parent_key) + if used_lift or _is_too_large_inline_value(reduced): + return self._lift_inline_task(operand, workflow_keys, parent_key=parent_key) + return reduced + return self._lift_inline_task(operand, workflow_keys, parent_key=parent_key) + + def _unwrap_identity_cast(self, operand, workflow_keys, *, parent_key=None): + raw_args = getattr(operand, "args", ()) or () + raw_kwargs = getattr(operand, "kwargs", {}) or {} + typ = raw_kwargs.get("typ", None) + values = [self.unwrap_operand(arg, workflow_keys, parent_key=parent_key) for arg in raw_args] + + if typ in (list, tuple, set, frozenset, dict): + try: + return typ(values) + except Exception: + pass + return self._lift_inline_task(operand, workflow_keys, parent_key=parent_key) + + def _extract_alias_target(self, alias_node, workflow_keys): + fields = getattr(alias_node.__class__, "__dataclass_fields__", {}) if self.dts else {} + path = tuple(getattr(alias_node, "path", ()) or ()) + + for candidate in ("alias_of", "target", "source", "ref"): + if candidate not in fields: + continue + key = _resolve_graph_key_if_task(getattr(alias_node, candidate, None), workflow_keys) + if key is not None: + return TaskOutputRef(key, path) + + deps = getattr(alias_node, "dependencies", None) + if deps: + deps = list(deps) + if len(deps) == 1: + key = _resolve_graph_key_if_task(deps[0], workflow_keys) + return TaskOutputRef(key if key is not None else deps[0], path) + return None + + def _reduce_inline_task(self, task_node, workflow_keys, *, parent_key=None): + func = _extract_callable_from_task(task_node) + raw_args = getattr(task_node, "args", ()) or () + raw_kwargs = getattr(task_node, "kwargs", {}) or {} + used_lift = False + + args = [] + for arg in raw_args: + before = len(self.lifted_nodes) + args.append(self.unwrap_operand(arg, workflow_keys, parent_key=parent_key)) + used_lift = used_lift or (len(self.lifted_nodes) != before) + + kwargs = {} + for key, value in raw_kwargs.items(): + before = len(self.lifted_nodes) + kwargs[key] = self.unwrap_operand(value, workflow_keys, parent_key=parent_key) + used_lift = used_lift or (len(self.lifted_nodes) != before) + + try: + return func(*args, **kwargs), used_lift + except Exception: + return self._lift_inline_task(task_node, workflow_keys, parent_key=parent_key), True + + def _lift_inline_task(self, task_node, workflow_keys, *, parent_key=None): + inline_key = getattr(task_node, "key", None) + if parent_key is not None and inline_key == parent_key: + raise ValueError(f"Refusing to lift Task that would self-reference parent key {parent_key!r}") + + signature = self._structural_signature(task_node, workflow_keys) + cached = self._lift_cache.get(signature) + if cached is not None: + return TaskOutputRef(cached, ()) + + digest = hashlib.sha1(signature.encode("utf-8")).hexdigest()[:16] + base = f"__lift__{digest}" + new_key = base + while new_key in workflow_keys or new_key in self.lifted_nodes: + self._lift_counter += 1 + new_key = f"{base}_{self._lift_counter}" + + self._lift_cache[signature] = new_key + self.lifted_nodes[new_key] = task_node + workflow_keys.add(new_key) + return TaskOutputRef(new_key, ()) + + def _structural_signature(self, obj, workflow_keys): + try: + return self._structural_signature_impl(obj, workflow_keys) + except Exception: + return f"fallback({_safe_repr(obj)})" + + def _structural_signature_impl(self, obj, workflow_keys): + taskref_cls = getattr(self.dts, "TaskRef", None) + alias_cls = getattr(self.dts, "Alias", None) + literal_cls = getattr(self.dts, "Literal", None) + datanode_cls = getattr(self.dts, "DataNode", None) + nested_cls = getattr(self.dts, "NestedContainer", None) + task_cls = getattr(self.dts, "Task", None) + + if taskref_cls and isinstance(obj, taskref_cls): + return f"TaskRef({getattr(obj, 'key', None)!r},{tuple(getattr(obj, 'path', ()) or ())!r})" + if alias_cls and isinstance(obj, alias_cls): + ref = self._extract_alias_target(obj, workflow_keys) + return f"Alias({getattr(ref, 'workflow_key', None)!r},{getattr(ref, 'path', ())!r})" + if literal_cls and isinstance(obj, literal_cls): + return f"Literal({_safe_repr(getattr(obj, 'value', None))})" + if datanode_cls and isinstance(obj, datanode_cls): + return f"DataNode({_safe_repr(getattr(obj, 'value', None))})" + if nested_cls and isinstance(obj, nested_cls): + payload = getattr(obj, "value", None) + if payload is None: + payload = getattr(obj, "data", None) + return f"Nested({self._structural_signature(payload, workflow_keys)})" + if task_cls and isinstance(obj, task_cls): + key = getattr(obj, "key", None) + if key is not None and key in workflow_keys: + return f"TaskKey({key!r})" + func = _extract_callable_from_task(obj) + func_id = (getattr(func, "__module__", None), getattr(func, "__qualname__", None), getattr(func, "__name__", None)) + args = getattr(obj, "args", ()) or () + kwargs = getattr(obj, "kwargs", {}) or {} + arg_sigs = ",".join(self._structural_signature(arg, workflow_keys) for arg in args) + kw_sigs = ",".join(f"{key}={self._structural_signature(value, workflow_keys)}" for key, value in sorted(kwargs.items())) + return f"TaskInline(func={func_id!r},args=[{arg_sigs}],kwargs=[{kw_sigs}])" + if isinstance(obj, list): + return "list(" + ",".join(self._structural_signature(value, workflow_keys) for value in obj) + ")" + if isinstance(obj, tuple): + return "tuple(" + ",".join(self._structural_signature(value, workflow_keys) for value in obj) + ")" + if isinstance(obj, dict): + items = ",".join( + f"{_safe_repr(key)}:{self._structural_signature(value, workflow_keys)}" + for key, value in sorted(obj.items(), key=lambda item: repr(item[0])) + ) + return "dict(" + items + ")" + if isinstance(obj, (set, frozenset)): + items = ",".join(sorted(self._structural_signature(value, workflow_keys) for value in obj)) + return f"{type(obj).__name__}(" + items + ")" + return f"py({_safe_repr(obj)})" + + +build_expr = _build_expr +safe_repr = _safe_repr +resolve_graph_key_if_task = _resolve_graph_key_if_task +extract_callable_from_task = _extract_callable_from_task diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/executor/__init__.py b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/executor/__init__.py new file mode 100644 index 0000000000..db860c5f5c --- /dev/null +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/executor/__init__.py @@ -0,0 +1,27 @@ +# Copyright (C) 2025 The University of Notre Dame +# This software is distributed under the GNU General Public License. +# See the file COPYING for details. + +import importlib +import sys +from . import vine_graph_capi +from . import vine_graph +from .vine_graph import ( + ExecutorGraph, + VineGraphExecutor, + _format_scheduler_keys_runner_payload, + format_scheduler_keys_runner_payload, +) + +# SWIG-generated vine_graph_capi.py does "import cvine"; wire the top-level name to the real module. +sys.modules.setdefault("cvine", importlib.import_module("ndcctools.taskvine.cvine")) + +sys.modules.setdefault(__name__ + ".graph", vine_graph) + +__all__ = [ + "vine_graph_capi", + "VineGraphExecutor", + "ExecutorGraph", + "_format_scheduler_keys_runner_payload", + "format_scheduler_keys_runner_payload", +] diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/executor/vine_graph.py b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/executor/vine_graph.py new file mode 100644 index 0000000000..c20e7d2cf5 --- /dev/null +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/executor/vine_graph.py @@ -0,0 +1,132 @@ +# Copyright (C) 2025 The University of Notre Dame +# This software is distributed under the GNU General Public License. +# See the file COPYING for details. + +"""Python wrapper for the C executor graph API.""" + +import json + +from . import vine_graph_capi + + +def _format_scheduler_keys_runner_payload(scheduler_keys): + """ + JSON string for the task-runner ``infile`` buffer: comma-separated scheduler keys in ``fn_args[0]``. + Compatible with worker ``task_runner.task.run_scheduler_keys`` (parses string / list / int). + Workflow keys ↔ scheduler keys mapping stays on the executor graph (``VineGraphExecutor.add_node``). + + Use this helper when assembling a multi-key call from Python. + """ + keys_list = scheduler_keys if isinstance(scheduler_keys, (list, tuple)) else [scheduler_keys] + if not keys_list: + raise ValueError("scheduler_keys must be non-empty") + joined = ",".join(str(int(k)) for k in keys_list) + return json.dumps({"fn_args": [joined], "fn_kwargs": {}}) + + +class VineGraphExecutor: + """Thin wrapper around the SWIG bindings.""" + + def __init__(self, c_taskvine): + """Create the backing C executor graph.""" + self._c_graph = vine_graph_capi.vine_graph_executor_create_graph(c_taskvine) + self._c_executor = vine_graph_capi.vine_graph_executor_create(c_taskvine, self._c_graph) + self._workflow_key_to_scheduler_key = {} + self._scheduler_key_to_workflow_key = {} + + def tune(self, name, value): + """Forward a tuning parameter to the C executor.""" + if vine_graph_capi.vine_graph_executor_tune(self._c_executor, name, value) != 0: + raise RuntimeError(f"Failed to tune executor parameter {name!r}={value!r}") + + def add_node(self, workflow_key, is_target=None): + """Create a C node and record its workflow key.""" + node_id = vine_graph_capi.vine_graph_executor_add_node(self._c_executor) + self._workflow_key_to_scheduler_key[workflow_key] = node_id + self._scheduler_key_to_workflow_key[node_id] = workflow_key + if is_target is not None and bool(is_target): + vine_graph_capi.vine_graph_set_target(self._c_graph, node_id) + return node_id + + def set_target(self, workflow_key): + """Mark a node as a target.""" + node_id = self._workflow_key_to_scheduler_key.get(workflow_key) + if node_id is None: + raise KeyError(f"Workflow key not found: {workflow_key}") + vine_graph_capi.vine_graph_set_target(self._c_graph, node_id) + + def add_dependency(self, parent_workflow_key, child_workflow_key): + """Add an edge between two existing nodes.""" + wk2sk = self._workflow_key_to_scheduler_key + if parent_workflow_key not in wk2sk or child_workflow_key not in wk2sk: + raise KeyError("parent or child workflow_key missing in mapping; call add_node() first") + vine_graph_capi.vine_graph_add_dependency( + self._c_graph, wk2sk[parent_workflow_key], wk2sk[child_workflow_key] + ) + + def group_chain_like_tasks(self): + """Merge maximal singleton linear chains into supernodes (C vine_graph_group_chain_like_tasks).""" + return vine_graph_capi.vine_graph_group_chain_like_tasks(self._c_graph) + + def compute_topology_metrics(self): + """Finalize the C graph and compute topology metrics.""" + vine_graph_capi.vine_graph_executor_finalize(self._c_executor) + + def get_node_outfile_remote_name(self, workflow_key): + """Return the output path assigned by the C graph.""" + if workflow_key not in self._workflow_key_to_scheduler_key: + raise KeyError(f"Workflow key not found: {workflow_key}") + return vine_graph_capi.vine_graph_get_node_outfile_remote_name( + self._c_graph, self._workflow_key_to_scheduler_key[workflow_key] + ) + + def get_task_runner_library_name(self): + """Return the generated task runner library name.""" + return vine_graph_capi.vine_graph_get_task_runner_library_name(self._c_graph) + + def set_task_runner_function(self, task_runner_function): + """Set the worker-side task runner entry point.""" + vine_graph_capi.vine_graph_set_task_runner_function_name( + self._c_graph, task_runner_function.__name__ + ) + + def add_task_input(self, workflow_key, filename): + """Add an input file to a task.""" + task_id = self._workflow_key_to_scheduler_key.get(workflow_key) + if task_id is None: + raise KeyError(f"Workflow key not found: {workflow_key}") + vine_graph_capi.vine_graph_executor_add_task_input(self._c_executor, task_id, filename) + + def add_task_output(self, workflow_key, filename): + """Add an output file to a task.""" + task_id = self._workflow_key_to_scheduler_key.get(workflow_key) + if task_id is None: + raise KeyError(f"Workflow key not found: {workflow_key}") + vine_graph_capi.vine_graph_executor_add_task_output(self._c_executor, task_id, filename) + + def execute(self): + """Execute the graph.""" + vine_graph_capi.vine_graph_executor_execute(self._c_executor) + + def get_makespan_us(self): + """Return the graph makespan in microseconds.""" + return vine_graph_capi.vine_graph_executor_get_makespan_us(self._c_executor) + + def get_total_recovery_tasks(self): + """Return the total number of submitted recovery tasks.""" + return vine_graph_capi.vine_graph_executor_get_total_recovery_tasks(self._c_executor) + + def get_completed_recovery_tasks(self): + """Return the number of completed recovery tasks.""" + return vine_graph_capi.vine_graph_executor_get_completed_recovery_tasks(self._c_executor) + + def delete(self): + """Delete the backing C graph.""" + vine_graph_capi.vine_graph_executor_delete(self._c_executor) + self._c_executor = None + vine_graph_capi.vine_graph_delete(self._c_graph) + self._c_graph = None + + +ExecutorGraph = VineGraphExecutor +format_scheduler_keys_runner_payload = _format_scheduler_keys_runner_payload diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/task_runner/.gitignore b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/task_runner/.gitignore new file mode 100644 index 0000000000..c18dd8d83c --- /dev/null +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/task_runner/.gitignore @@ -0,0 +1 @@ +__pycache__/ diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/task_runner/__init__.py b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/task_runner/__init__.py new file mode 100644 index 0000000000..7c8bcb2dbc --- /dev/null +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/task_runner/__init__.py @@ -0,0 +1,18 @@ +from .library import TaskRunnerLibrary +from .task import ( + compute_task, + execute_scheduler_keys, + execute_workflow_task, + run_scheduler_keys, +) + +run_task_key = run_scheduler_keys + +__all__ = [ + "TaskRunnerLibrary", + "execute_workflow_task", + "execute_scheduler_keys", + "run_scheduler_keys", + "compute_task", + "run_task_key", +] diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/task_runner/library.py b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/task_runner/library.py new file mode 100644 index 0000000000..9142c17373 --- /dev/null +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/task_runner/library.py @@ -0,0 +1,89 @@ +# Copyright (C) 2025- The University of Notre Dame +# This software is distributed under the GNU General Public License. +# See the file COPYING for details. + +import os +import uuid +import cloudpickle +import types +import time +import random +import hashlib +import collections + +from ..workflow import Workflow, TaskOutputRef, TaskOutputWrapper +from .task import run_scheduler_keys +from ndcctools.taskvine.utils import load_variable_from_library + + +class TaskRunnerLibrary: + def __init__(self, py_manager): + self.py_manager = py_manager + + self.name = None + self.libcores = None + + self.libtask = None + + # these modules are always included in the preamble of the library task, so that function calls can execute directly + # using the loaded context without importing them over and over again + self.hoisting_modules = [ + os, cloudpickle, Workflow, TaskOutputRef, TaskOutputWrapper, uuid, hashlib, random, types, collections, time, + load_variable_from_library, run_scheduler_keys + ] + + # environment files serve as additional inputs to the library task, where each key is the local path and the value is the remote path + # those local files will be sent remotely to the workers so tasks can access them as appropriate + self.env_files = {} + + # context loader is a function that will be used to load the library context on remote nodes. + self.context_loader_func = None + self.context_loader_args = [] + self.context_loader_kwargs = {} + + self.local_path = None + self.remote_path = None + + def set_libcores(self, libcores): + self.libcores = libcores + + def set_name(self, name): + self.name = name + + def add_hoisting_modules(self, new_modules): + assert isinstance(new_modules, list), "new_modules must be a list of modules" + self.hoisting_modules.extend(new_modules) + + def add_env_files(self, new_env_files): + assert isinstance(new_env_files, dict), "new_env_files must be a dictionary" + self.env_files.update(new_env_files) + + def set_context_loader(self, context_loader_func, context_loader_args=[], context_loader_kwargs={}): + self.context_loader_func = context_loader_func + self.context_loader_args = context_loader_args + self.context_loader_kwargs = context_loader_kwargs + + def install(self): + assert self.name is not None, "Library name must be set before installing (use set_name method)" + assert self.libcores is not None, "Library cores must be set before installing (use set_libcores method)" + + self.libtask = self.py_manager.create_library_from_functions( + self.name, + run_scheduler_keys, + library_context_info=[self.context_loader_func, self.context_loader_args, self.context_loader_kwargs], + add_env=False, + function_infile_load_mode="json", + hoisting_modules=self.hoisting_modules, + ) + for local, remote in self.env_files.items(): + # check if the local file exists + if not os.path.exists(local): + raise FileNotFoundError(f"Local file {local} not found") + # attach as the input file to the library task + self.libtask.add_input(self.py_manager.declare_file(local, cache=True, peer_transfer=True), remote) + self.libtask.set_cores(self.libcores) + self.libtask.set_function_slots(self.libcores) + self.py_manager.install_library(self.libtask) + + def uninstall(self): + self.py_manager.remove_library(self.name) diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/task_runner/task.py b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/task_runner/task.py new file mode 100644 index 0000000000..ad5a1232ce --- /dev/null +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/task_runner/task.py @@ -0,0 +1,199 @@ +# Copyright (C) 2025- The University of Notre Dame +# This software is distributed under the GNU General Public License. +# See the file COPYING for details. + + +import sys +import time +from collections import defaultdict, deque + +from ndcctools.taskvine.utils import load_variable_from_library + + +def _resolve_nested_legacy_tasks(obj): + """Evaluate nested legacy Dask tasks encoded as plain ``(func, *args)`` tuples.""" + if type(obj) is tuple and obj and callable(obj[0]): + func = obj[0] + args = tuple(_resolve_nested_legacy_tasks(v) for v in obj[1:]) + return func(*args) + + if isinstance(obj, list): + return [_resolve_nested_legacy_tasks(v) for v in obj] + + if type(obj) is tuple: + return tuple(_resolve_nested_legacy_tasks(v) for v in obj) + + if isinstance(obj, dict): + return {k: _resolve_nested_legacy_tasks(v) for k, v in obj.items()} + + if isinstance(obj, set): + return {_resolve_nested_legacy_tasks(v) for v in obj} + + if isinstance(obj, frozenset): + return frozenset(_resolve_nested_legacy_tasks(v) for v in obj) + + return obj + + +def compute_task(workflow, task_expr): + func_id, args, kwargs = task_expr + func = workflow.callables[func_id] + cache = {} + + def _follow_path(value, path): + current = value + for token in path: + if isinstance(current, (list, tuple)): + current = current[token] + elif isinstance(current, dict): + current = current[token] + else: + current = getattr(current, token) + return current + + def on_ref(r): + x = cache.get(r.workflow_key) + if x is None: + x = workflow.load_task_output(r.workflow_key) + cache[r.workflow_key] = x + if r.path: + return _follow_path(x, r.path) + return x + + r_args = workflow._visit_task_output_refs(args, on_ref, rewrite=True) + r_kwargs = workflow._visit_task_output_refs(kwargs, on_ref, rewrite=True) + + r_args = _resolve_nested_legacy_tasks(r_args) + r_kwargs = _resolve_nested_legacy_tasks(r_kwargs) + + return func(*r_args, **r_kwargs) + + +def topo_sort_group_scheduler_keys(workflow, member_scheduler_keys): + """Return a topological order of scheduler keys for one batched library call. + + Edges count only when both endpoints lie in the batch. Ties are broken by the order keys + appear in the argument list which usually matches the CSV written by C with the leader first. + """ + # Kahn tie-break uses the order keys were listed because plain sets forget that order. + ordered = [] + seen = set() + for k in member_scheduler_keys: + kk = int(k) + if kk not in seen: + seen.add(kk) + ordered.append(kk) + + nodes = set(ordered) + order_index = {k: i for i, k in enumerate(ordered)} + adj = defaultdict(set) + indeg = {k: 0 for k in nodes} + + # Restrict Workflow to this batch: edge parent_sk -> child_sk only when both endpoints are members. + # Aligns with the C executor DAG via the same parents_of as Python Workflow. + for child_sk in nodes: + wk_child = workflow.scheduler_key_to_workflow_key[child_sk] + for wk_parent in workflow.parents_of.get(wk_child, ()): + parent_sk = workflow.workflow_key_to_scheduler_key[wk_parent] + if parent_sk in nodes and child_sk not in adj[parent_sk]: + adj[parent_sk].add(child_sk) + indeg[child_sk] += 1 + + # Kahn: serial execution order on the worker (intra-batch deps before dependents). + q = deque(k for k in ordered if indeg[k] == 0) + out = [] + while q: + u = q.popleft() + out.append(u) + ready_next = [] + for v in adj[u]: + indeg[v] -= 1 + if indeg[v] == 0: + ready_next.append(v) + ready_next.sort(key=lambda x: order_index[x]) + for v in ready_next: + q.append(v) + if len(out) != len(nodes): + raise ValueError("task_group: cycle among members per Workflow graph") + return out + + +def run_single_workflow_node(workflow, scheduler_key): + """Run one node: scheduler_key -> workflow_key, execute, write outfile for downstream refs.""" + workflow_key = workflow.scheduler_key_to_workflow_key[scheduler_key] + task_expr = workflow.task_dict[workflow_key] + + output = compute_task(workflow, task_expr) + + time.sleep(workflow.extra_task_sleep_time[workflow_key]) + + workflow.save_task_output(workflow_key, output) + + +def _scheduler_keys_spec_to_list(scheduler_keys_spec): + """ + Normalize library infile payload to a list of integer scheduler keys. + Primary wire form is one comma-separated string (e.g. ``\"1,2,3\"``). + Also accepts a bare int (legacy JSON) or a list of ints/strings from json. + """ + if isinstance(scheduler_keys_spec, str): + parts = [p.strip() for p in scheduler_keys_spec.split(",") if p.strip()] + if not parts: + raise ValueError("run_scheduler_keys: empty scheduler key list after parsing") + return [int(p, 10) for p in parts] + if isinstance(scheduler_keys_spec, int): + return [scheduler_keys_spec] + if isinstance(scheduler_keys_spec, list): + if not scheduler_keys_spec: + raise ValueError("run_scheduler_keys: empty list") + return [int(x) for x in scheduler_keys_spec] + raise TypeError( + f"run_scheduler_keys: expected str, int, or list of keys, got {type(scheduler_keys_spec).__name__}" + ) + + +def _workflow_from_task_runner_library(): + """ + Resolve the Workflow from the TaskVine library context. + + The normal path is ``ndcctools.taskvine.utils.load_variable_from_library``. + Keep a ``__main__`` lookup first so generated library scripts that inject + context directly into their own module namespace also work. + """ + main = sys.modules.get("__main__") + g = getattr(main, "graph", None) if main is not None else None + if g is not None: + return g + return load_variable_from_library("graph") + + +def run_scheduler_keys(scheduler_keys_spec): + """TaskVine library entry that parses keys, orders them, runs each node, and writes outfiles.""" + workflow = _workflow_from_task_runner_library() + keys = _scheduler_keys_spec_to_list(scheduler_keys_spec) + ordered = topo_sort_group_scheduler_keys(workflow, keys) + leader_sk = keys[0] + # The infile from C lists the leader first. Kahn might pick another indeg-zero key first which + # would reorder writes and confuse the manager's validation, so move the leader to the front + # when it has no parent that is also inside this batch. + if ordered[0] != leader_sk: + batch = set(ordered) + wk_leader = workflow.scheduler_key_to_workflow_key[leader_sk] + for wkp in workflow.parents_of.get(wk_leader, ()): + psk = workflow.workflow_key_to_scheduler_key[wkp] + if psk in batch: + raise ValueError( + f"run_scheduler_keys: leader {leader_sk} must run first but has intra-batch parent {psk}" + ) + ordered = [leader_sk] + [sk for sk in ordered if sk != leader_sk] + + for sk in ordered: + run_single_workflow_node(workflow, sk) + + +execute_workflow_task = compute_task +topologically_sort_scheduler_keys = topo_sort_group_scheduler_keys +execute_workflow_node = run_single_workflow_node +_parse_scheduler_keys = _scheduler_keys_spec_to_list +_load_workflow_from_task_runner_library = _workflow_from_task_runner_library +execute_scheduler_keys = run_scheduler_keys diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/test/.gitignore b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/test/.gitignore new file mode 100644 index 0000000000..c18dd8d83c --- /dev/null +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/test/.gitignore @@ -0,0 +1 @@ +__pycache__/ diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/test/Makefile b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/test/Makefile new file mode 100644 index 0000000000..49f953f698 --- /dev/null +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/test/Makefile @@ -0,0 +1,17 @@ +include ../../../../../../../../config.mk + +.PHONY: all install lint format test clean + +all: + +install: all + +lint: + +format: + +test: + cd $(CURDIR) && $(CCTOOLS_PYTHON3) task_group_parity.py + +clean: + rm -rf __pycache__ diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/test/graph_templates.py b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/test/graph_templates.py new file mode 100644 index 0000000000..7358d01348 --- /dev/null +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/test/graph_templates.py @@ -0,0 +1,157 @@ +import math + +from ndcctools.taskvine.dagvine.workflow import TaskOutputRef, Workflow + +MERGE_FANIN = 32 # cap add() fan-in for wide merges + + +def add(*args): + return sum(args) + + +def _merge(bg, keys, prefix="M"): + if len(keys) == 1: + return keys[0] + cur = list(keys) + level = idx = 0 + while len(cur) > 1: + nxt = [] + for i in range(0, len(cur), MERGE_FANIN): + grp = cur[i:i + MERGE_FANIN] + if len(grp) == 1: + nxt.append(grp[0]) + else: + k = f"{prefix}{level}_{idx}" + idx += 1 + bg.add_task(k, add, *[TaskOutputRef(x) for x in grp]) + nxt.append(k) + cur = nxt + level += 1 + return cur[0] + + +def make_simple_graph(): + bg = Workflow() + bg.add_task("t0", add, 1, 5) + bg.add_task("output", add, TaskOutputRef("t0")) + return bg + + +def make_chain_graph(chain_len=1, branches=1): + chain_len = max(1, int(chain_len)) + branches = max(1, int(branches)) + bg = Workflow() + finals = [] + for b in range(branches): + head = f"C{b}_0" + bg.add_task(head, add, 1) + prev = head + for i in range(1, chain_len): + node = f"C{b}_{i}" + bg.add_task(node, add, TaskOutputRef(prev)) + prev = node + finals.append(prev) + root = finals[0] if len(finals) == 1 else _merge(bg, finals, "CH_") + bg.add_task("output", add, TaskOutputRef(root)) + return bg + + +def make_chain_rich(n=1): + n = max(1, int(n)) + bg = Workflow() + if n == 1: + bg.add_task("output", add, 1) + return bg + branch_count = min(MERGE_FANIN, max(1, (n - 1) // 8)) + base, extra = divmod(n - 1, branch_count) + finals = [] + for b in range(branch_count): + size = base + (1 if b < extra else 0) + k = f"B{b}_0" + bg.add_task(k, add, 1) + prev = k + for i in range(1, size): + nk = f"B{b}_{i}" + bg.add_task(nk, add, TaskOutputRef(prev)) + prev = nk + finals.append(prev) + bg.add_task("output", add, TaskOutputRef(_merge(bg, finals, "CR_"))) + return bg + + +def make_individuals(n=1): + n = max(1, int(n)) + bg = Workflow() + keys = [f"I{i}" for i in range(n)] + for k in keys: + bg.add_task(k, add, 1) + bg.add_task("output", add, TaskOutputRef(_merge(bg, keys, "IND_"))) + return bg + + +def make_trivial(n=1): + bg = Workflow() + for i in range(max(1, int(n))): + bg.add_task(f"T{i}", add, 1) + return bg + + +def make_binary_tree(n=1): + n = max(1, int(n)) + bg = Workflow() + last = (n - 2) // 2 + for i in range(last + 1, n): + bg.add_task(f"BT{i}", add, 1) + for i in range(last, -1, -1): + deps = [TaskOutputRef(f"BT{2 * i + 1}")] + if 2 * i + 2 < n: + deps.append(TaskOutputRef(f"BT{2 * i + 2}")) + bg.add_task(f"BT{i}", add, *deps) + return bg + + +def make_binary_forest(n=None, *, branches=5, level=8): + if n is not None: + n = max(1, int(n)) + branches = max(1, min(n, MERGE_FANIN)) + level = max(1, math.ceil(math.log2(max(1, n // branches) + 1))) + else: + branches, level = max(1, branches), max(1, level) + + bg = Workflow() + finals = [] + for b in range(branches): + leaves = [f"F{b}_L{i}" for i in range(2 ** (level - 1))] + for leaf in leaves: + bg.add_task(leaf, add, 1) + cur = leaves + for d in range(level - 1): + nxt = [] + for i in range(0, len(cur), 2): + p = f"F{b}_N{d}_{i // 2}" + bg.add_task(p, add, TaskOutputRef(cur[i]), TaskOutputRef(cur[i + 1])) + nxt.append(p) + cur = nxt + finals.append(cur[0]) + bg.add_task("output", add, TaskOutputRef(_merge(bg, finals, "BF_"))) + return bg + + +def build(name, n=None): + if name == "simple": + return make_simple_graph() + if name == "chain": + return make_chain_graph(max(1, n or 8)) + if name == "chain-branches": + return make_chain_graph(max(1, n or 8), branches=4) + if name == "chain-rich": + return make_chain_rich(max(1, n or 1000)) + if name == "binary-forest": + return make_binary_forest(n) + if name == "individuals": + return make_individuals(max(1, n or 1000)) + if name == "trivial": + return make_trivial(max(1, n or 1000)) + if name == "binary-tree": + return make_binary_tree(max(1, n or 1000)) + raise ValueError(name) diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/test/run_graph.py b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/test/run_graph.py new file mode 100644 index 0000000000..89ca59a759 --- /dev/null +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/test/run_graph.py @@ -0,0 +1,58 @@ +import argparse +import shutil +import tempfile +from pathlib import Path + +from runner import parse_cases, run_graph + + +def main(): + p = argparse.ArgumentParser() + p.add_argument("-G", "--graph", nargs="+") + p.add_argument("--case", action="append", dest="cases") + p.add_argument("--task-group", type=int, default=0) + p.add_argument("--task-priority-mode", default="random") + p.add_argument("--port", type=int, default=9100) + p.add_argument("--timeout", type=float, default=120.0) + p.add_argument("--no-print-results", action="store_true") + args = p.parse_args() + + if args.cases: + cases = parse_cases(args.cases) + elif args.graph: + if len(args.graph) > 2: + p.error("-G takes GRAPH [N]") + n = int(args.graph[1]) if len(args.graph) == 2 else None + cases = [(args.graph[0], n)] + else: + p.error("need -G or --case") + + root = Path(tempfile.mkdtemp(prefix="vine_graph-run-")) if len(cases) > 1 else None + rc = 0 + try: + for i, (g, n) in enumerate(cases): + try: + res = run_graph( + g, + n, + task_group=args.task_group, + port=args.port + i, + work_root=root, + tag=f"{i:02d}-{g}-{n or 'na'}", + timeout_s=args.timeout, + priority=args.task_priority_mode, + ) + except Exception as e: + rc = 1 + print(g, n, "fail:", e) + continue + if not args.no_print_results: + print(g, n, res) + finally: + if root: + shutil.rmtree(root, ignore_errors=True) + return rc + + +if __name__ == "__main__": + raise SystemExit(main()) diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/test/runner.py b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/test/runner.py new file mode 100644 index 0000000000..66907b206c --- /dev/null +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/test/runner.py @@ -0,0 +1,127 @@ +import os +import shutil +import signal +import subprocess +import sys +import tempfile +import time +from pathlib import Path + +import cloudpickle +import graph_templates as gt +import ndcctools.taskvine.dagvine.vine_graph as vine_graph_mod +from ndcctools.taskvine.dagvine import VineGraph + +TEST_DIR = Path(__file__).resolve().parent + + +def parse_cases(specs): + out = [] + for s in specs: + name, _, n = s.strip().partition(":") + out.append((name, None if not n else int(n))) + return out + + +def _vine_worker(): + w = Path(sys.executable).parent / "vine_worker" + return str(w) if w.is_file() else shutil.which("vine_worker") + + +def _run_vine_graph(graph, n, task_group, port, logs, tag, out_dir, ckpt_dir, priority): + run_info = logs / tag + if run_info.exists(): + shutil.rmtree(run_info) + + wf = gt.build(graph, n) + targets = ["output"] if "output" in wf.task_dict else [] + + def context_loader(graph_pkl): + cwd = os.getcwd() + if cwd not in sys.path: + sys.path.insert(0, cwd) + return {"graph": cloudpickle.loads(graph_pkl)} + + vine_graph_mod.context_loader_func = context_loader + try: + cloudpickle.register_pickle_by_value(gt) + except Exception: + pass + + m = VineGraph([port, port], run_info_path=str(logs), run_info_template=tag) + m.update_params( + { + "checkpoint-dir": str(ckpt_dir), + "extra-task-output-size-mb": [0.0, 0.0], + "extra-task-sleep-time": [0.0, 0.0], + "output-dir": str(out_dir), + "task-group": task_group, + "task-priority-mode": priority, + "wait-for-workers": 1, + } + ) + return m.run( + wf, + target_keys=targets, + hoisting_modules=[gt], + env_files={"./graph_templates.py": "graph_templates.py"}, + ) or {} + + +def run_graph( + graph, + n=None, + task_group=0, + port=9100, + work_root=None, + tag="run", + timeout_s=120.0, + priority="random", +): + worker = _vine_worker() + if not worker: + raise RuntimeError("vine_worker not found") + + os.environ.setdefault("CATALOG_UPDATE_PROTOCOL", "udp") + + root = work_root or Path(tempfile.mkdtemp(prefix="vine_graph-run-")) + delete_root = work_root is None + ws = root / f"ws-{tag}" + shutil.rmtree(ws, ignore_errors=True) + ws.mkdir(parents=True) + logs = root / "logs" + out_d = root / "out" / tag + ckpt = root / "ckpt" / tag + for d in (logs, out_d, ckpt): + d.mkdir(parents=True, exist_ok=True) + + env = os.environ.copy() + proc = subprocess.Popen( + [worker, "--idle-timeout", str(max(120, int(timeout_s) + 60)), "-s", str(ws), "127.0.0.1", str(port)], + stdout=subprocess.DEVNULL, + stderr=subprocess.PIPE, + env=env, + ) + time.sleep(1.2) + + def on_alarm(signum, frame): + raise TimeoutError(timeout_s) + + try: + old = signal.signal(signal.SIGALRM, on_alarm) + signal.setitimer(signal.ITIMER_REAL, timeout_s) + try: + os.chdir(TEST_DIR) + return _run_vine_graph(graph, n, task_group, port, logs, tag, out_d, ckpt, priority) + finally: + signal.setitimer(signal.ITIMER_REAL, 0) + signal.signal(signal.SIGALRM, old) + finally: + proc.terminate() + try: + proc.wait(timeout=8) + except subprocess.TimeoutExpired: + proc.kill() + shutil.rmtree(ws, ignore_errors=True) + if delete_root: + shutil.rmtree(root, ignore_errors=True) diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/test/task_group_parity.py b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/test/task_group_parity.py new file mode 100644 index 0000000000..2e78835d6a --- /dev/null +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/test/task_group_parity.py @@ -0,0 +1,65 @@ +import argparse +import shutil +import tempfile +from pathlib import Path + +from runner import parse_cases, run_graph + +CASES = [ + ("simple", None), + ("chain", 1), + ("chain", 2), + ("chain", 3), + ("chain", 8), + ("chain", 32), + ("chain", 128), + ("chain-branches", 6), + ("chain-branches", 16), + ("individuals", 8), + ("trivial", 24), + ("binary-tree", 7), + ("binary-tree", 31), + ("binary-tree", 127), + ("chain-rich", 30), + ("chain-rich", 120), + ("chain-rich", 500), +] + + +def main(): + p = argparse.ArgumentParser() + p.add_argument("--timeout", type=float, default=120.0) + p.add_argument("--base-port", type=int, default=22100) + p.add_argument("--case", action="append", dest="cases") + args = p.parse_args() + + cases = parse_cases(args.cases) if args.cases else CASES + root = Path(tempfile.mkdtemp(prefix="vine_graph-parity-")) + rc = 0 + try: + for i, (g, n) in enumerate(cases): + tag = f"{i:02d}-{g}-{n or 'na'}" + kw = dict(graph=g, n=n, work_root=root, timeout_s=args.timeout, priority="fifo") + try: + r0 = run_graph(**kw, port=args.base_port + i * 2, task_group=0, tag="tg0-" + tag) + r1 = run_graph(**kw, port=args.base_port + i * 2 + 1, task_group=1, tag="tg1-" + tag) + except TimeoutError: + rc = 1 + print(g, n, "timeout") + continue + except Exception as e: + rc = 1 + print(g, n, "fail:", e) + continue + if r0 != r1: + rc = 1 + print(g, n, "mismatch", r0, r1) + else: + print(g, n, "ok", r0) + finally: + shutil.rmtree(root, ignore_errors=True) + return rc + + +if __name__ == "__main__": + raise SystemExit(main()) diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/utils.py b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/utils.py new file mode 100644 index 0000000000..63dede4ed0 --- /dev/null +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/utils.py @@ -0,0 +1,29 @@ +import cloudpickle +import os + + +def context_loader_func(graph_pkl): + graph = cloudpickle.loads(graph_pkl) + return {"graph": graph} + + +def remove_tree_contents(root_dir): + """Remove files under the run-info template directory.""" + if not os.path.exists(root_dir): + return + for dirpath, dirnames, filenames in os.walk(root_dir): + for filename in filenames: + file_path = os.path.join(dirpath, filename) + try: + os.remove(file_path) + except FileNotFoundError: + print(f"Failed to delete file {file_path}") + + +def color_text(text, color_code): + """Return text wrapped in an ANSI color code.""" + return f"\033[{color_code}m{text}\033[0m" + + +load_vine_graph_context = context_loader_func +delete_all_files = remove_tree_contents diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/vine_graph.py b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/vine_graph.py new file mode 100644 index 0000000000..d082061227 --- /dev/null +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/vine_graph.py @@ -0,0 +1,360 @@ +# Copyright (C) 2025 The University of Notre Dame +# This software is distributed under the GNU General Public License. +# See the file COPYING for details. + +from ndcctools.taskvine.manager import Manager + +from .dask_adaptor import VineGraphDaskAdaptor +from .task_runner import TaskRunnerLibrary, execute_workflow_task, run_scheduler_keys +from .workflow import Workflow, TaskOutputRef, TaskOutputWrapper +from .executor.vine_graph import VineGraphExecutor +from .utils import color_text, context_loader_func, remove_tree_contents + +from rich.progress import BarColumn, MofNCompleteColumn, Progress, TextColumn, TimeRemainingColumn + +import cloudpickle +import os +import random +import signal +import time + + +class VineGraphConfig: + def __init__(self): + """Store VineGraph configuration by the layer that consumes it.""" + self.manager_tuning = { + "worker-source-max-transfers": 100, + "max-retrievals": -1, + "prefer-dispatch": 1, + "transient-error-interval": 1, + "attempt-schedule-depth": 10000, + "temp-replica-count": 1, + "enforce-worker-eviction-interval": -1, + "shift-disk-load": 0, + "clean-redundant-replicas": 0, + } + self.executor_tuning = { + "failure-injection-step-percent": -1, + "task-priority-mode": "largest-input-first", + "prune-depth": 1, + "output-dir": "./outputs", + "checkpoint-dir": "./checkpoints", + "checkpoint-fraction": 0, + "progress-bar-update-interval-sec": 0.1, + "enable-debug-log": 1, + "print-graph-details": 0, + } + self.task_runner = { + "libcores": 16, + } + self.execution = { + "schedule": "worst", + "extra-task-output-size-mb": [0, 0], + "extra-task-sleep-time": [0, 0], + # 1 = run Workflow in-process (topological order), no workers / no task runner library; stdout stays on frontend. + "local-execute": 0, + # 1 = merge maximal linear chains into supernodes before finalize (vine_graph_group_chain_like_tasks). + "task-group": 0, + } + + def _sections(self): + return ( + self.manager_tuning, + self.executor_tuning, + self.task_runner, + self.execution, + ) + + def update_param(self, param_name, new_value): + """Update one parameter.""" + for section in self._sections(): + if param_name in section: + section[param_name] = new_value + return + # Unknown parameters are assumed to be TaskVine manager tuning knobs. + self.manager_tuning[param_name] = new_value + + def get_value_of(self, param_name): + """Return the current value for a parameter.""" + for section in self._sections(): + if param_name in section: + return section[param_name] + raise ValueError(f"Invalid param name: {param_name}") + + +class VineGraph(Manager): + def __init__(self, + *args, + **kwargs): + """Create a VineGraph manager.""" + + signal.signal(signal.SIGINT, self._on_sigint) + + self.params = VineGraphConfig() + + run_info_path = kwargs.get("run_info_path", None) + run_info_template = kwargs.get("run_info_template", None) + + self.run_info_template_path = os.path.join(run_info_path, run_info_template) + if self.run_info_template_path: + remove_tree_contents(self.run_info_template_path) + + # Manager lifetime is tied to this object. + super().__init__(*args, **kwargs) + + print(f"=== Manager name: {color_text(self.name, 92)}") + print(f"=== Manager port: {color_text(self.port, 92)}") + print(f"=== Runtime directory: {color_text(self.runtime_directory, 92)}") + self._sigint_received = False + + def get_param(self, param_name): + """Return a parameter value.""" + return self.params.get_value_of(param_name) + + def set_params(self, new_params): + """Apply a batch of parameter overrides.""" + assert isinstance(new_params, dict), "new_params must be a dict" + for k, new_v in new_params.items(): + self.params.update_param(k, new_v) + + def tune_manager(self): + """Apply manager-side tuning.""" + for k, v in self.params.manager_tuning.items(): + try: + self.tune(k, v) + except Exception: + raise ValueError(f"Unrecognized parameter: {k}") + + def tune_executor(self, executor): + """Apply executor-graph tuning.""" + for k, v in self.params.executor_tuning.items(): + executor.tune(k, str(v)) + + def _rep_key(self, k, r): + return k if r == 0 else ("__rep", r, k) + + def _replicate_graph(self, task_dict, target_keys, repeats): + if repeats <= 1: + return task_dict, target_keys + if isinstance(task_dict, Workflow): + old_workflow = task_dict + new_workflow = Workflow() + new_workflow.callables = list(old_workflow.callables) + new_workflow._callable_index = dict(old_workflow._callable_index) + for r in range(repeats): + def rewriter(ref): + return TaskOutputRef(self._rep_key(ref.workflow_key, r), ref.path) + + def _rewrite(obj): + return old_workflow._visit_task_output_refs(obj, rewriter, rewrite=True) + + for k, (func_id, args, kwargs) in old_workflow.task_dict.items(): + new_workflow.add_task(self._rep_key(k, r), old_workflow.callables[func_id], *_rewrite(args), **_rewrite(kwargs)) + new_workflow.finalize() + executor_targets = list(target_keys) + for r in range(1, repeats): + executor_targets.extend(self._rep_key(k, r) for k in target_keys if k in old_workflow.task_dict) + return new_workflow, executor_targets + else: + temp_workflow = Workflow() + expanded = {} + for r in range(repeats): + def rewriter(ref): + return TaskOutputRef(self._rep_key(ref.workflow_key, r), ref.path) + + def _rewrite(obj): + return temp_workflow._visit_task_output_refs(obj, rewriter, rewrite=True) + + for k, v in task_dict.items(): + func, args, kwargs = v + expanded[self._rep_key(k, r)] = (func, _rewrite(args), _rewrite(kwargs)) + executor_targets = list(target_keys) + for r in range(1, repeats): + executor_targets.extend(self._rep_key(k, r) for k in target_keys if k in task_dict) + return expanded, executor_targets + + def build_workflow(self, task_dict): + if isinstance(task_dict, Workflow): + workflow = task_dict + else: + workflow = Workflow() + + for k, v in task_dict.items(): + func, args, kwargs = v + assert callable(func), f"Task {k} does not have a callable" + workflow.add_task(k, func, *args, **kwargs) + + workflow.finalize() + + return workflow + + def build_executor(self, py_graph, target_keys): + """Build the C executor graph from the Python graph.""" + assert py_graph is not None, "Python graph must be built before building the VineGraphExecutor" + + executor = VineGraphExecutor(self._taskvine) + + executor.set_task_runner_function(run_scheduler_keys) + + self.tune_manager() + self.tune_executor(executor) + + topo_order = py_graph.get_topological_order() + + for k in topo_order: + node_id = executor.add_node(k) + py_graph.workflow_key_to_scheduler_key[k] = node_id + py_graph.scheduler_key_to_workflow_key[node_id] = k + for pk in py_graph.parents_of[k]: + executor.add_dependency(pk, k) + + for k in target_keys: + executor.set_target(k) + + return executor + + def build_workflow_and_executor(self, task_dict, target_keys): + """Build the Python graph and its C mirror.""" + py_graph = self.build_workflow(task_dict) + + # Ignore requested targets that are not in the graph. + missing_keys = [k for k in target_keys if k not in py_graph.task_dict] + if missing_keys: + print(f"=== Warning: the following target keys are not in the graph: {','.join(map(str, missing_keys))}") + target_keys = list(set(target_keys) - set(missing_keys)) + + executor = self.build_executor(py_graph, target_keys) + + # Declare graph-level file dependencies in the C graph. + for filename in py_graph.producer_of: + workflow_key = py_graph.producer_of[filename] + executor.add_task_output(workflow_key, filename) + for filename in py_graph.consumers_of: + for workflow_key in py_graph.consumers_of[filename]: + executor.add_task_input(workflow_key, filename) + + # Matches --task-group on the Python side so the C executor knows whether merging is allowed. + executor.tune( + "chain-grouping-enabled", + "1" if int(self.get_param("task-group")) else "0", + ) + + if int(self.get_param("task-group")): + executor.group_chain_like_tasks() + + executor.compute_topology_metrics() + + # Save output locations back into the Python graph after finalize may adjust checkpoint paths. + for k in py_graph.workflow_key_to_scheduler_key: + outfile_remote_name = executor.get_node_outfile_remote_name(k) + py_graph.outfile_remote_name[k] = outfile_remote_name + + return py_graph, executor + + def build_task_runner_library(self, py_graph, executor, hoisting_modules, env_files): + """Build the TaskVine task runner library.""" + task_runner_library = TaskRunnerLibrary(self) + task_runner_library.add_hoisting_modules(hoisting_modules) + task_runner_library.add_env_files(env_files) + task_runner_library.set_context_loader(context_loader_func, context_loader_args=[cloudpickle.dumps(py_graph)]) + task_runner_library.set_libcores(self.get_param("libcores")) + task_runner_library.set_name(executor.get_task_runner_library_name()) + + return task_runner_library + + def _execute_workflow_local(self, py_graph): + """Run the workflow locally in topological order.""" + out_dir = os.path.abspath(self.get_param("output-dir")) + os.makedirs(out_dir, exist_ok=True) + prev_cwd = os.getcwd() + os.chdir(out_dir) + t0 = time.time() + try: + order = py_graph.get_topological_order() + interval = float(self.get_param("progress-bar-update-interval-sec")) + if interval <= 0: + interval = 0.1 + refresh_per_second = min(30.0, max(1.0, 1.0 / interval)) + + n = len(order) + if n == 0: + return time.time() - t0 + + with Progress( + TextColumn("[bold]Executing Tasks"), + TextColumn("•"), + TextColumn("[cyan]User"), + BarColumn(), + MofNCompleteColumn(), + TimeRemainingColumn(), + refresh_per_second=refresh_per_second, + transient=False, + ) as progress: + bar_id = progress.add_task("User", total=n) + for k in order: + out = execute_workflow_task(py_graph, py_graph.task_dict[k]) + py_graph.save_task_output(k, out) + progress.advance(bar_id) + finally: + os.chdir(prev_cwd) + return time.time() - t0 + + def run(self, task_dict, target_keys=[], params={}, hoisting_modules=[], env_files={}, from_dask=False, expand_subgraphs=False, repeats=1): + """Build the graph, run it, and return the requested results.""" + self.set_params(params) + + if from_dask: + task_dict = VineGraphDaskAdaptor(task_dict, expand_subgraphs=expand_subgraphs).converted + + result_keys = list(target_keys) + task_dict, target_keys = self._replicate_graph(task_dict, target_keys, repeats) + + py_graph, executor = self.build_workflow_and_executor(task_dict, target_keys) + # Optional synthetic output size / sleep for testing. + for k in py_graph.task_dict: + py_graph.extra_task_output_size_mb[k] = random.uniform(*self.get_param("extra-task-output-size-mb")) + py_graph.extra_task_sleep_time[k] = random.uniform(*self.get_param("extra-task-sleep-time")) + + local_execute = bool(self.get_param("local-execute")) + task_runner_library = None + + try: + if local_execute: + print("=== local-execute: running Workflow in process (no workers)", flush=True) + makespan_s = self._execute_workflow_local(py_graph) + completed_recovery_tasks = 0 + else: + task_runner_library = self.build_task_runner_library(py_graph, executor, hoisting_modules, env_files) + task_runner_library.install() + executor.execute() + makespan_s = round(executor.get_makespan_us() / 1e6, 6) + completed_recovery_tasks = executor.get_completed_recovery_tasks() + + total_tasks_completed = len(py_graph.task_dict) + completed_recovery_tasks + throughput_tps = round(total_tasks_completed / makespan_s, 6) if makespan_s > 0 else 0.0 + print(f"=== Makespan: {makespan_s:.6f} seconds") + print(f"=== Total tasks completed: {total_tasks_completed}") + print(f"=== Throughput: {throughput_tps:.6f} tasks/s") + + results = {} + for k in result_keys: + if k not in py_graph.task_dict: + continue + outfile_path = os.path.join(self.get_param("output-dir"), py_graph.outfile_remote_name[k]) + results[k] = TaskOutputWrapper.load_from_path(outfile_path) + return results + finally: + try: + if task_runner_library is not None: + task_runner_library.uninstall() + finally: + executor.delete() + + param = get_param + update_params = set_params + build_graphs = build_workflow_and_executor + create_task_runner_library = build_task_runner_library + + def _on_sigint(self, signum, frame): + self._sigint_received = True + raise KeyboardInterrupt diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/workflow.py b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/workflow.py new file mode 100644 index 0000000000..fe8e5d5203 --- /dev/null +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/workflow.py @@ -0,0 +1,210 @@ +# Copyright (C) 2025- The University of Notre Dame +# This software is distributed under the GNU General Public License. +# See the file COPYING for details. + +from collections import defaultdict, deque +from collections.abc import Mapping +from dataclasses import is_dataclass, fields, replace +import cloudpickle + + +# Lightweight wrapper around task results that optionally pads the payload. The +# padding lets tests model large outputs without altering the logical result. +class TaskOutputWrapper: + def __init__(self, result, extra_size_mb=None): + self.result = result + self.extra_obj = bytearray(int(extra_size_mb * 1024 * 1024)) if extra_size_mb and extra_size_mb > 0 else None + + @staticmethod + def load_from_path(path): + try: + with open(path, "rb") as f: + result_obj = cloudpickle.load(f) + assert isinstance(result_obj, TaskOutputWrapper), "Loaded object is not of type TaskOutputWrapper" + return result_obj.result + except FileNotFoundError: + raise FileNotFoundError(f"Task result file not found at {path}") + + +# A reference to a task output. This is used to represent the output of a task as a dependency of another task. +class TaskOutputRef: + __slots__ = ("workflow_key", "path") + + def __init__(self, workflow_key, path=()): + self.workflow_key = workflow_key + self.path = tuple(path) + + def __getitem__(self, key): + if isinstance(key, tuple): + return TaskOutputRef(self.workflow_key, self.path + key) + return TaskOutputRef(self.workflow_key, self.path + (key,)) + + +# The Workflow is a directed acyclic graph (DAG) that represents the logical dependencies between tasks. +# It is used to build the C executor graph. +class Workflow: + + _LEAF_TYPES = (str, bytes, bytearray, memoryview, int, float, bool, type(None)) + + def __init__(self): + self.callables = [] + self._callable_index = {} + + self.task_dict = {} + + self.parents_of = defaultdict(set) # workflow_key -> set of workflow_keys + self.children_of = defaultdict(set) # workflow_key -> set of workflow_keys + + self.producer_of = {} # filename -> workflow_key + self.consumers_of = defaultdict(set) # filename -> set of workflow_keys + + self.outfile_remote_name = defaultdict(lambda: None) # workflow_key -> remote outfile name, will be set by the executor graph + + self.workflow_key_to_scheduler_key = {} # workflow_key -> scheduler key (C node id) + self.scheduler_key_to_workflow_key = {} # scheduler key -> workflow_key + + self.extra_task_output_size_mb = {} # workflow_key -> extra size in MB + self.extra_task_sleep_time = {} # workflow_key -> extra sleep time in seconds + + def _intern_callable(self, func): + idx = self._callable_index.get(func) + if idx is None: + idx = len(self.callables) + self.callables.append(func) + self._callable_index[func] = idx + return idx + + def _visit_task_output_refs(self, obj, on_ref, *, rewrite: bool): + seen = set() + + def rec(x): + if isinstance(x, TaskOutputRef): + return on_ref(x) + + if x is None or isinstance(x, self._LEAF_TYPES): + return x if rewrite else None + + oid = id(x) + if oid in seen: + return x if rewrite else None + seen.add(oid) + + if isinstance(x, Mapping): + for k in x.keys(): + if isinstance(k, TaskOutputRef): + raise ValueError("TaskOutputRef cannot be used as dict key") + if not rewrite: + for v in x.values(): + rec(v) + return None + return {k: rec(v) for k, v in x.items()} + + if is_dataclass(x) and not isinstance(x, type): + if not rewrite: + for f in fields(x): + rec(getattr(x, f.name)) + return None + updates = {f.name: rec(getattr(x, f.name)) for f in fields(x)} + try: + return replace(x, **updates) + except Exception: + return x.__class__(**updates) + + if isinstance(x, tuple) and hasattr(x, "_fields"): # namedtuple + if not rewrite: + for v in x: + rec(v) + return None + return x.__class__(*(rec(v) for v in x)) + + if isinstance(x, (list, tuple, set, frozenset, deque)): + if not rewrite: + for v in x: + rec(v) + return None + it = (rec(v) for v in x) + if isinstance(x, list): + return list(it) + if isinstance(x, tuple): + return tuple(it) + if isinstance(x, set): + return set(it) + if isinstance(x, frozenset): + return frozenset(it) + return deque(it) + + return x if rewrite else None + + return rec(obj) + + def _find_parents(self, obj): + parents = set() + + def on_ref(r): + parents.add(r.workflow_key) + return None + + self._visit_task_output_refs(obj, on_ref, rewrite=False) + return parents + + def add_task(self, workflow_key, func, *args, **kwargs): + if workflow_key in self.task_dict: + raise ValueError(f"Task {workflow_key} already exists") + + func_id = self._intern_callable(func) + self.task_dict[workflow_key] = (func_id, args, kwargs) + + parents = self._find_parents(args) | self._find_parents(kwargs) + + for parent in parents: + self.parents_of[workflow_key].add(parent) + self.children_of[parent].add(workflow_key) + + def task_produces(self, workflow_key, *filenames): + for filename in filenames: + # a file can only be produced by one task + if filename in self.producer_of: + raise ValueError(f"File {filename} already produced by task {self.producer_of[filename]}") + self.producer_of[filename] = workflow_key + + def task_consumes(self, workflow_key, *filenames): + for filename in filenames: + # a file can be consumed by multiple tasks + self.consumers_of[filename].add(workflow_key) + + def save_task_output(self, workflow_key, output): + with open(self.outfile_remote_name[workflow_key], "wb") as f: + wrapped_output = TaskOutputWrapper(output, extra_size_mb=self.extra_task_output_size_mb[workflow_key]) + cloudpickle.dump(wrapped_output, f) + + def load_task_output(self, workflow_key): + return TaskOutputWrapper.load_from_path(self.outfile_remote_name[workflow_key]) + + def get_topological_order(self): + indegree = {} + for workflow_key in self.task_dict: + indegree[workflow_key] = len(self.parents_of.get(workflow_key, ())) + + q = deque(t for t, d in indegree.items() if d == 0) + order = [] + + while q: + u = q.popleft() + order.append(u) + + for v in self.children_of.get(u, ()): + indegree[v] -= 1 + if indegree[v] == 0: + q.append(v) + + if len(order) != len(self.task_dict): + raise ValueError("Graph has a cycle or missing dependencies") + + return order + + def finalize(self): + # build the dependencies determined by files produced and consumed + for file, producer in self.producer_of.items(): + for consumer in self.consumers_of.get(file, ()): + self.parents_of[consumer].add(producer) + self.children_of[producer].add(consumer) diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/manager.py b/taskvine/src/bindings/python3/ndcctools/taskvine/manager.py index 3fca684785..97aa7af2b8 100644 --- a/taskvine/src/bindings/python3/ndcctools/taskvine/manager.py +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/manager.py @@ -240,6 +240,12 @@ def port(self): def using_ssl(self): return self._using_ssl + ## + # Absolute path to this workflow run's runtime directory (parent of vine-logs, staging, etc.). + @property + def runtime_directory(self): + return cvine.vine_get_runtime_directory(self._taskvine) + ## # Get the logs directory of the manager @property @@ -1031,8 +1037,9 @@ def check_library_exists(self, library_name): # @param hoisting_modules A list of modules imported at the preamble of library, including packages, functions and classes. # @param exec_mode Execution mode that the library should use to run function calls. Either 'direct' or 'fork' # @param library_context_info A list containing [library_context_func, library_context_args, library_context_kwargs]. Used to create the library context on remote nodes. + # @param function_infile_load_mode The mode to load infile for function tasks inside this library. # @returns A task to be used with @ref ndcctools.taskvine.manager.Manager.install_library. - def create_library_from_functions(self, library_name, *function_list, poncho_env=None, init_command=None, add_env=True, hoisting_modules=None, exec_mode='fork', library_context_info=None): + def create_library_from_functions(self, library_name, *function_list, poncho_env=None, init_command=None, add_env=True, hoisting_modules=None, exec_mode='fork', library_context_info=None, function_infile_load_mode='cloudpickle'): # Delay loading of poncho until here, to avoid bringing in poncho dependencies unless needed. # Ensure poncho python library is available. from ndcctools.poncho import package_serverize @@ -1054,7 +1061,8 @@ def create_library_from_functions(self, library_name, *function_list, poncho_env add_env=add_env, exec_mode=exec_mode, hoisting_modules=hoisting_modules, - library_context_info=library_context_info) + library_context_info=library_context_info, + function_infile_load_mode=function_infile_load_mode) # Create path for caching library code and environment based on function hash. library_cache_dir_name = "vine-library-cache" @@ -1102,7 +1110,8 @@ def create_library_from_functions(self, library_name, *function_list, poncho_env need_pack=need_pack, exec_mode=exec_mode, hoisting_modules=hoisting_modules, - library_context_info=library_context_info) + library_context_info=library_context_info, + function_infile_load_mode=function_infile_load_mode) # enable correct permissions for library code os.chmod(library_code_path, 0o775) diff --git a/taskvine/src/graph/.gitignore b/taskvine/src/graph/.gitignore new file mode 100644 index 0000000000..1576058b30 --- /dev/null +++ b/taskvine/src/graph/.gitignore @@ -0,0 +1,6 @@ +*.o +*.py[cod] +*.so +__pycache__/ +vine_graph_capi.py +vine_graph_wrap.c diff --git a/taskvine/src/graph/Makefile b/taskvine/src/graph/Makefile new file mode 100644 index 0000000000..48b9a10cb4 --- /dev/null +++ b/taskvine/src/graph/Makefile @@ -0,0 +1,62 @@ +include ../../../config.mk +include ../../../rules.mk + +PROJECT_NAME := vine_graph +SOURCE_DIR := $(CCTOOLS_HOME)/taskvine/src/graph +PY_EXECUTOR_DIR := $(CCTOOLS_HOME)/taskvine/src/bindings/python3/ndcctools/taskvine/dagvine/executor +MODULE_DIR := $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/dagvine/executor +MODULE_NAME := vine_graph_capi + +SOURCES := vine_graph_node.c vine_graph.c vine_graph_executor.c +OBJECTS := $(SOURCES:%.c=%.o) +SWIG_WRAP := vine_graph_wrap.c +WRAP_OBJ := vine_graph_wrap.o +PYMODULE := _$(MODULE_NAME).$(CCTOOLS_DYNAMIC_SUFFIX) + +LOCAL_LINKAGE += $(CCTOOLS_HOME)/taskvine/src/manager/libtaskvine.a $(CCTOOLS_HOME)/dttools/src/libdttools.a +LOCAL_CCFLAGS += -I $(CCTOOLS_HOME)/taskvine/src/manager -I $(SOURCE_DIR) + +.PHONY: all install clean lint format test + +all: $(PYMODULE) + +$(SWIG_WRAP): vine_graph.i vine_graph.h vine_graph_executor.h + $(CCTOOLS_SWIG) -python -threads -relativeimport \ + -I$(CCTOOLS_HOME)/taskvine/src/manager \ + -I$(CCTOOLS_HOME)/dttools/src \ + -I$(SOURCE_DIR) \ + -outdir $(SOURCE_DIR) -o $@ $< + +$(WRAP_OBJ): $(SWIG_WRAP) + $(CCTOOLS_CC) -o $@ -c $(CCTOOLS_INTERNAL_CCFLAGS) $(LOCAL_CCFLAGS) $(CCTOOLS_PYTHON3_CCFLAGS) -w -fPIC -DNDEBUG $< + +%.o: %.c vine_graph.h vine_graph_executor.h vine_graph_node.h + $(CCTOOLS_CC) -o $@ -c $(CCTOOLS_INTERNAL_CCFLAGS) $(LOCAL_CCFLAGS) -fPIC -DNDEBUG $< + +$(PYMODULE): $(WRAP_OBJ) $(OBJECTS) + $(CCTOOLS_LD) -o $@ $(CCTOOLS_DYNAMIC_FLAG) $(CCTOOLS_INTERNAL_LDFLAGS) $(LOCAL_LDFLAGS) $^ $(LOCAL_LINKAGE) $(CCTOOLS_PYTHON3_LDFLAGS) $(CCTOOLS_EXTERNAL_LINKAGE) + +install: all + mkdir -p $(CCTOOLS_INSTALL_DIR)/$(PROJECT_NAME)/include + cp $(CCTOOLS_HOME)/taskvine/src/manager/taskvine.h $(CCTOOLS_INSTALL_DIR)/$(PROJECT_NAME)/include/ + rm -rf $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/vine_graph/executor + rm -rf $(MODULE_DIR) + mkdir -p $(MODULE_DIR) + rm -f $(MODULE_DIR)/graph.py $(MODULE_DIR)/graph_capi.py $(MODULE_DIR)/_graph_capi.$(CCTOOLS_DYNAMIC_SUFFIX) + cp $(PYMODULE) $(MODULE_DIR) + cp $(MODULE_NAME).py $(MODULE_DIR) + cp $(PY_EXECUTOR_DIR)/__init__.py $(MODULE_DIR) + cp $(PY_EXECUTOR_DIR)/vine_graph.py $(MODULE_DIR) + +clean: + rm -f *.o + rm -rf __pycache__ + rm -f $(SWIG_WRAP) $(WRAP_OBJ) $(PYMODULE) $(MODULE_NAME).py + +lint: + clang-format -Werror --dry-run --style='file:$(CCTOOLS_HOME)/.clang-format' $(SOURCE_DIR)/*.[ch] + +format: + clang-format -i $(SOURCE_DIR)/*.[ch] + +test: diff --git a/taskvine/src/graph/vine_graph.c b/taskvine/src/graph/vine_graph.c new file mode 100644 index 0000000000..673512d7de --- /dev/null +++ b/taskvine/src/graph/vine_graph.c @@ -0,0 +1,1023 @@ +#include +#include +#include +#include +#include +#include + +#include "debug.h" +#include "vine_graph.h" +#include "priority_queue.h" +#include "set.h" +#include "stringtools.h" +#include "uuid.h" +#include "xxmalloc.h" + +/*************************************************************/ +/* Private Functions */ +/*************************************************************/ + +/** + * Compute a topological ordering of the executor graph. + * Call only after all nodes, edges, and metrics have been populated. + * @param g Reference to the executor graph. + * @return Nodes in topological order. + */ +static struct list *vine_graph_compute_topological_order(struct vine_graph *g) +{ + if (!g) { + return NULL; + } + + int total_nodes = itable_size(g->nodes); + struct list *topo_order = list_create(); + struct itable *in_degree_map = itable_create(0); + struct priority_queue *pq = priority_queue_create(total_nodes); + + uint64_t nid; + struct vine_graph_node *node; + ITABLE_ITERATE(g->nodes, nid, node) + { + int deg = list_size(node->parents); + itable_insert(in_degree_map, nid, (void *)(intptr_t)deg); + if (deg == 0) { + priority_queue_push(pq, node, -(double)node->node_id); + } + } + + while (priority_queue_size(pq) > 0) { + struct vine_graph_node *current = priority_queue_pop(pq); + list_push_tail(topo_order, current); + + struct vine_graph_node *child; + LIST_ITERATE(current->children, child) + { + intptr_t raw_deg = (intptr_t)itable_lookup(in_degree_map, child->node_id); + int deg = (int)raw_deg - 1; + itable_insert(in_degree_map, child->node_id, (void *)(intptr_t)deg); + + if (deg == 0) { + priority_queue_push(pq, child, -(double)child->node_id); + } + } + } + + if (list_size(topo_order) != total_nodes) { + debug(D_ERROR, "Error: executor graph contains cycles or is malformed."); + debug(D_ERROR, "Expected %d nodes, but only sorted %d.", total_nodes, list_size(topo_order)); + + uint64_t id; + ITABLE_ITERATE(g->nodes, id, node) + { + intptr_t raw_deg = (intptr_t)itable_lookup(in_degree_map, id); + int deg = (int)raw_deg; + if (deg > 0) { + debug(D_ERROR, " Node %" PRIu64 " has in-degree %d. Parents:", id, deg); + struct vine_graph_node *p; + LIST_ITERATE(node->parents, p) + { + debug(D_ERROR, " -> %" PRIu64, p->node_id); + } + } + } + + list_delete(topo_order); + itable_delete(in_degree_map); + priority_queue_delete(pq); + exit(1); + } + + itable_delete(in_degree_map); + priority_queue_delete(pq); + return topo_order; +} + +/** + * Extract weakly connected components of the executor graph. + * Currently used for debugging and instrumentation only. + * @param g Reference to the executor graph. + * @return List of weakly connected components. + */ +static struct list *vine_graph_extract_weak_components(struct vine_graph *g) +{ + if (!g) { + return NULL; + } + + struct set *visited = set_create(0); + struct list *components = list_create(); + + uint64_t nid; + struct vine_graph_node *node; + ITABLE_ITERATE(g->nodes, nid, node) + { + if (set_lookup(visited, node)) { + continue; + } + + struct list *component = list_create(); + struct list *queue = list_create(); + + list_push_tail(queue, node); + set_insert(visited, node); + list_push_tail(component, node); + + while (list_size(queue) > 0) { + struct vine_graph_node *curr = list_pop_head(queue); + + struct vine_graph_node *p; + LIST_ITERATE(curr->parents, p) + { + if (!set_lookup(visited, p)) { + list_push_tail(queue, p); + set_insert(visited, p); + list_push_tail(component, p); + } + } + + struct vine_graph_node *c; + LIST_ITERATE(curr->children, c) + { + if (!set_lookup(visited, c)) { + list_push_tail(queue, c); + set_insert(visited, c); + list_push_tail(component, c); + } + } + } + + list_push_tail(components, component); + list_delete(queue); + } + + set_delete(visited); + return components; +} + +/** + * Compute the heavy score of a node in the executor graph. + * @param node Reference to the node. + * @return Heavy score. + */ +static double vine_graph_node_compute_heavy_score(struct vine_graph_node *node) +{ + if (!node) { + return 0; + } + + double up_score = node->depth * node->upstream_subgraph_size * node->fan_in; + double down_score = node->height * node->downstream_subgraph_size * node->fan_out; + + return up_score / (down_score + 1); +} + +static void vine_graph_node_set_local_output_file(struct vine_graph_node *node) +{ + node->outfile_type = VINE_GRAPH_NODE_OUTFILE_TYPE_LOCAL; +} + +static void vine_graph_node_set_temp_output_file(struct vine_graph_node *node) +{ + node->outfile_type = VINE_GRAPH_NODE_OUTFILE_TYPE_TEMP; +} + +/** + * Compute upstream/downstream subgraph sizes and heavy scores for each node. + * This is expensive (can approach transitive-closure cost) and should only be + * invoked when heavy-score-based checkpoint selection is enabled. + */ +static void vine_graph_compute_topology_scores(struct vine_graph *g, struct list *topo_order) +{ + if (!g || !topo_order) { + return; + } + + struct vine_graph_node *node; + struct vine_graph_node *parent_node; + struct vine_graph_node *child_node; + + struct itable *upstream_map = itable_create(0); // reachable ancestors per node + struct itable *downstream_map = itable_create(0); // reachable descendants per node + uint64_t nid_tmp; + ITABLE_ITERATE(g->nodes, nid_tmp, node) + { + struct set *upstream = set_create(0); + struct set *downstream = set_create(0); + itable_insert(upstream_map, node->node_id, upstream); + itable_insert(downstream_map, node->node_id, downstream); + } + + LIST_ITERATE(topo_order, node) + { + struct set *upstream = itable_lookup(upstream_map, node->node_id); + LIST_ITERATE(node->parents, parent_node) + { + struct set *parent_upstream = itable_lookup(upstream_map, parent_node->node_id); + set_insert_set(upstream, parent_upstream); // in-place union, not set_union + set_insert(upstream, parent_node); + } + } + + LIST_ITERATE_REVERSE(topo_order, node) + { + struct set *downstream = itable_lookup(downstream_map, node->node_id); + LIST_ITERATE(node->children, child_node) + { + struct set *child_downstream = itable_lookup(downstream_map, child_node->node_id); + set_insert_set(downstream, child_downstream); // in-place union, not set_union + set_insert(downstream, child_node); + } + } + + LIST_ITERATE(topo_order, node) + { + node->upstream_subgraph_size = set_size(itable_lookup(upstream_map, node->node_id)); + node->downstream_subgraph_size = set_size(itable_lookup(downstream_map, node->node_id)); + node->fan_in = list_size(node->parents); + node->fan_out = list_size(node->children); + set_delete(itable_lookup(upstream_map, node->node_id)); + set_delete(itable_lookup(downstream_map, node->node_id)); + } + + itable_delete(upstream_map); + itable_delete(downstream_map); + + LIST_ITERATE(topo_order, node) + { + node->heavy_score = vine_graph_node_compute_heavy_score(node); // ranks checkpoint candidates + } +} + +/*************************************************************/ +/* Public APIs */ +/*************************************************************/ + +/** Tune the executor graph. + * @param g Reference to the executor graph. + * @param name Reference to the name of the parameter to tune. + * @param value Reference to the value of the parameter to tune. + * @return 0 on success, -1 on failure. + */ +int vine_graph_tune(struct vine_graph *g, const char *name, const char *value) +{ + if (!g || !name || !value) { + return -1; + } + + if (strcmp(name, "output-dir") == 0) { + if (mkdir(value, 0777) != 0 && errno != EEXIST) { + debug(D_ERROR, "failed to mkdir %s (errno=%d)", value, errno); + return -1; + } + free(g->output_dir); + g->output_dir = xxstrdup(value); + + } else if (strcmp(name, "prune-depth") == 0) { + int k = atoi(value); + if (k < 0) { + debug(D_ERROR, "invalid prune-depth: %s (must be >= 0; 0 disables prune-depth release)", value); + return -1; + } + g->prune_depth = k; + + } else if (strcmp(name, "checkpoint-fraction") == 0) { + double fraction = atof(value); + if (fraction < 0.0 || fraction > 1.0) { + debug(D_ERROR, "invalid checkpoint fraction: %s (must be between 0.0 and 1.0)", value); + return -1; + } + g->checkpoint_fraction = fraction; + + } else if (strcmp(name, "checkpoint-dir") == 0) { + if (mkdir(value, 0777) != 0 && errno != EEXIST) { + debug(D_ERROR, "failed to mkdir %s (errno=%d)", value, errno); + return -1; + } + free(g->checkpoint_dir); + g->checkpoint_dir = xxstrdup(value); + + } else if (strcmp(name, "print-graph-details") == 0) { + g->print_graph_details = (atoi(value) == 1) ? 1 : 0; + } else if (strcmp(name, "chain-grouping-enabled") == 0) { + /* Stays aligned with Python's --task-group. When zero the executor never merges chain members. */ + g->chain_grouping_enabled = (atoi(value) != 0) ? 1 : 0; + } else { + debug(D_ERROR, "invalid parameter name: %s", name); + return -1; + } + + return 0; +} + +/** + * Get the outfile remote name of a node in the executor graph. + * @param g Reference to the executor graph. + * @param node_id Reference to the node id. + * @return The outfile remote name. + */ +const char *vine_graph_get_node_outfile_remote_name(const struct vine_graph *g, uint64_t node_id) +{ + if (!g) { + return NULL; + } + + struct vine_graph_node *node = itable_lookup(g->nodes, node_id); + if (!node) { + return NULL; + } + + return node->outfile_remote_name; +} + +/** + * Get the task runner library name of the executor graph. + * @param g Reference to the executor graph. + * @return The task runner library name. + */ +const char *vine_graph_get_task_runner_library_name(const struct vine_graph *g) +{ + if (!g) { + return NULL; + } + + return g->task_runner_library_name; +} + +/** + * Set the task runner function name of the executor graph. + * @param g Reference to the executor graph. + * @param task_runner_function_name Reference to the task runner function name. + */ +void vine_graph_set_task_runner_function_name(struct vine_graph *g, const char *task_runner_function_name) +{ + if (!g || !task_runner_function_name) { + return; + } + + if (g->task_runner_function_name) { + free(g->task_runner_function_name); + } + + g->task_runner_function_name = xxstrdup(task_runner_function_name); +} + +/** + * Get the heavy score of a node in the executor graph. + * @param g Reference to the executor graph. + * @param node_id Reference to the node id. + * @return The heavy score. + */ +double vine_graph_get_node_heavy_score(const struct vine_graph *g, uint64_t node_id) +{ + if (!g) { + return -1; + } + + struct vine_graph_node *node = itable_lookup(g->nodes, node_id); + if (!node) { + return -1; + } + + return node->heavy_score; +} + +/** + * Compute the topology metrics of the executor graph, including depth, height, upstream and downstream counts, + * heavy scores, and weakly connected components. Must be called after all nodes and dependencies are added. + * @param g Reference to the executor graph. + */ +void vine_graph_finalize(struct vine_graph *g) +{ + if (!g) { + return; + } + + struct list *topo_order = vine_graph_compute_topological_order(g); // required for all metric passes + if (!topo_order) { + return; + } + + struct vine_graph_node *node; + struct vine_graph_node *parent_node; + struct vine_graph_node *child_node; + + /* Longest path from any source in topo order. */ + LIST_ITERATE(topo_order, node) + { + node->depth = 0; + LIST_ITERATE(node->parents, parent_node) + { + if (node->depth < parent_node->depth + 1) { + node->depth = parent_node->depth + 1; + } + } + } + + /* Longest path to any sink in reverse topo order. */ + LIST_ITERATE_REVERSE(topo_order, node) + { + node->height = 0; + LIST_ITERATE(node->children, child_node) + { + if (node->height < child_node->height + 1) { + node->height = child_node->height + 1; + } + } + } + + int total_nodes = list_size(topo_order); + int total_target_nodes = 0; + LIST_ITERATE(topo_order, node) + { + if (node->is_target) { + total_target_nodes++; + } + } + + /* + * Pick how many non-target nodes become shared-filesystem checkpoints. + * If zero, skip heavy-score passes entirely. + */ + int checkpoint_count = (int)((total_nodes - total_target_nodes) * g->checkpoint_fraction); + if (checkpoint_count < 0) { + checkpoint_count = 0; + } + + if (checkpoint_count > 0) { + vine_graph_compute_topology_scores(g, topo_order); // expensive, only if ranking needed + + struct priority_queue *sorted_nodes = priority_queue_create(total_nodes); + LIST_ITERATE(topo_order, node) + { + priority_queue_push(sorted_nodes, node, node->heavy_score); + } + + int assigned_checkpoint_count = 0; + while ((node = priority_queue_pop(sorted_nodes))) { + if (node->is_target) { + vine_graph_node_set_local_output_file(node); // targets keep managed local returns + continue; + } + if (assigned_checkpoint_count < checkpoint_count) { + /* + * Top heavy_score nodes checkpoint to shared storage under checkpoint_dir. + * No vine_file handle for that mode. + */ + node->outfile_type = VINE_GRAPH_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM; + char *shared_file_system_outfile_path = string_format("%s/%s", g->checkpoint_dir, node->outfile_remote_name); + free(node->outfile_remote_name); + node->outfile_remote_name = shared_file_system_outfile_path; + assigned_checkpoint_count++; + } else { + vine_graph_node_set_temp_output_file(node); // remaining nodes use temp storage + } + } + priority_queue_delete(sorted_nodes); + } else { + LIST_ITERATE(topo_order, node) + { + if (node->is_target) { + vine_graph_node_set_local_output_file(node); + } else { + vine_graph_node_set_temp_output_file(node); // no checkpoint budget, all non-targets are temp + } + } + } + + if (g->print_graph_details) { + // weakly connected components and vine_graph_node_debug_print, debug only + struct list *weakly_connected_components = vine_graph_extract_weak_components(g); + struct list *component; + int component_index = 0; + debug(D_VINE, "graph has %d weakly connected components\n", list_size(weakly_connected_components)); + LIST_ITERATE(weakly_connected_components, component) + { + debug(D_VINE, "component %d size: %d\n", component_index, list_size(component)); + list_delete(component); + component_index++; + } + list_delete(weakly_connected_components); + + LIST_ITERATE(topo_order, node) + { + vine_graph_node_debug_print(node); + } + } + + list_delete(topo_order); + + return; +} + +/** + * Create a new node and track it in the executor graph. + * @param g Reference to the executor graph. + * @return The auto-assigned node id. + */ +uint64_t vine_graph_add_node(struct vine_graph *g) +{ + if (!g) { + return 0; + } + + uint64_t candidate_id = itable_size(g->nodes); + candidate_id += 1; // skip zero, search upward until unused + while (itable_lookup(g->nodes, candidate_id)) { + candidate_id++; + } + uint64_t node_id = candidate_id; + + struct vine_graph_node *node = vine_graph_node_create(node_id); // defaults to non-target + + if (!node) { + debug(D_ERROR, "failed to create node %" PRIu64, node_id); + vine_graph_delete(g); + exit(1); + } + + itable_insert(g->nodes, node_id, node); + + return node_id; +} + +/** + * Mark a node as a retrieval target. + */ +void vine_graph_set_target(struct vine_graph *g, uint64_t node_id) +{ + if (!g) { + return; + } + struct vine_graph_node *node = itable_lookup(g->nodes, node_id); + if (!node) { + debug(D_ERROR, "node %" PRIu64 " not found", node_id); + exit(1); + } + + node->is_target = 1; +} + +/** + * Create a new executor graph using graph-owned path configuration. + * @param runtime_dir Runtime directory used as the default path root. + * @return A new executor graph instance. + */ +struct vine_graph *vine_graph_create(const char *runtime_dir) +{ + if (!runtime_dir) { + return NULL; + } + + struct vine_graph *g = xxmalloc(sizeof(struct vine_graph)); + + g->checkpoint_dir = xxstrdup(runtime_dir); // default to current working directory + g->output_dir = xxstrdup(runtime_dir); // default to current working directory + + g->nodes = itable_create(0); + g->super_leader_to_members = itable_create(0); + g->outfile_cachename_to_node = hash_table_create(0, 0); + g->inout_filename_to_cached_name = hash_table_create(0, 0); + g->supernode_leader_child_to_input_source = hash_table_create(0, 0); + + cctools_uuid_t task_runner_library_name_id; + cctools_uuid_create(&task_runner_library_name_id); + g->task_runner_library_name = xxstrdup(task_runner_library_name_id.str); + + g->task_runner_function_name = NULL; + g->checkpoint_fraction = 0.0; + + /* Default prune-depth: release a TEMP node as soon as all of its + * direct children have completed. Set to 0 via tune("prune-depth") to + * disable and rely exclusively on cut-propagation. */ + g->prune_depth = 1; + + g->print_graph_details = 0; + g->chain_grouping_enabled = 0; + + return g; +} + +/** + * Add a dependency between two nodes in the executor graph. Note that the input-output file relationship + * is not handled here, because their file names might not have been determined yet. + * @param g Reference to the executor graph. + * @param parent_id Reference to the parent node id. + * @param child_id Reference to the child node id. + */ +void vine_graph_add_dependency(struct vine_graph *g, uint64_t parent_id, uint64_t child_id) +{ + if (!g) { + return; + } + + struct vine_graph_node *parent_node = itable_lookup(g->nodes, parent_id); + struct vine_graph_node *child_node = itable_lookup(g->nodes, child_id); + if (!parent_node) { + debug(D_ERROR, "parent node %" PRIu64 " not found", parent_id); + exit(1); + } + if (!child_node) { + debug(D_ERROR, "child node %" PRIu64 " not found", child_id); + exit(1); + } + + vine_graph_node_ensure_dependency(parent_node, child_node); + + return; +} + +/** + * Copy a list of struct vine_graph_node * into a new list in the same order (pointer values only, not deep copy). + * Snapshots parents/children before mutating edges during supernode rewire. Caller frees the new list. + */ +static struct list *vine_graph_node_list_copy(struct list *src) +{ + struct list *dst = list_create(); + if (!src) { + return dst; + } + struct vine_graph_node *x; + LIST_ITERATE(src, x) + { + list_push_tail(dst, x); + } + return dst; +} + +/** + * After supernode registration: for each node in mset, remove internal edges (both ends in mset). + * Rewire edges that cross the group boundary through leader. mset contains leader plus all non-leader members. + */ +static void vine_graph_supernode_rewire(struct vine_graph *g, struct vine_graph_node *leader, struct set *mset) +{ + struct vine_graph_node *m; + uint64_t nid; + ITABLE_ITERATE(g->nodes, nid, m) + { + if (!set_lookup(mset, m)) { + continue; + } + + struct list *psnap = vine_graph_node_list_copy(m->parents); + struct vine_graph_node *p; + while ((p = list_pop_head(psnap))) { + if (set_lookup(mset, p)) { + vine_graph_node_remove_dependency(p, m); + } else { + vine_graph_node_remove_dependency(p, m); + vine_graph_node_ensure_dependency(p, leader); + } + } + list_delete(psnap); + + struct list *csnap = vine_graph_node_list_copy(m->children); + struct vine_graph_node *c; + while ((c = list_pop_head(csnap))) { + if (set_lookup(mset, c)) { + vine_graph_node_remove_dependency(m, c); + } else { + vine_graph_node_remove_dependency(m, c); + vine_graph_node_ensure_dependency(leader, c); + /* + * Scheduling sees leader->c; data still flows from member m (e.g. chain tail). + * Record so materialize mounts m->outfile, not the leader's primary outfile. + */ + if (m != leader) { + char *lckey = string_format("%" PRIu64 ",%" PRIu64, leader->node_id, c->node_id); + /* + * The table keeps the first insert if the same key is stored twice. Drop the + * old mapping explicitly so the node that truly produces the file for c wins. + */ + hash_table_remove(g->supernode_leader_child_to_input_source, lckey); + hash_table_insert(g->supernode_leader_child_to_input_source, lckey, m); + free(lckey); + } + } + } + list_delete(csnap); + } +} + +/** + * Resolve the leader struct vine_graph_node * for any node in a supernode (singleton maps to itself). + * Returns NULL if g or n is NULL or the leader id is missing from g. + */ +struct vine_graph_node *vine_graph_supernode_leader_node(struct vine_graph *g, struct vine_graph_node *n) +{ + if (!g || !n) { + return NULL; + } + return itable_lookup(g->nodes, n->super_leader_id); +} + +/** + * List of non-leader member nodes for leader_id. Owned by the graph; do not free. + * Items are struct vine_graph_node *. NULL if g is NULL or there is no entry. + */ +struct list *vine_graph_supernode_nonleader_members(struct vine_graph *g, uint64_t leader_id) +{ + if (!g) { + return NULL; + } + return itable_lookup(g->super_leader_to_members, leader_id); +} + +/** + * Register a supernode: validate members, rewire externals to leader_id, set super_leader_id on all members, + * clear fired_parents, and store non-leaders in super_leader_to_members. Nodes and plain deps must exist first. + * Returns 0 on success, -1 on error (see debug log). + */ +int vine_graph_supernode_register(struct vine_graph *g, uint64_t leader_id, const uint64_t *member_ids, int n_member_ids) +{ + if (!g || member_ids == NULL || n_member_ids < 1) { + debug(D_ERROR, "vine_graph_supernode_register: invalid arguments"); + return -1; + } + + struct vine_graph_node *leader = itable_lookup(g->nodes, leader_id); + if (!leader) { + debug(D_ERROR, "vine_graph_supernode_register: leader %" PRIu64 " not found", leader_id); + return -1; + } + + struct set *seen = set_create(0); + struct set *mset = set_create(0); + set_insert(mset, leader); + + for (int i = 0; i < n_member_ids; i++) { + uint64_t mid = member_ids[i]; + if (mid == leader_id) { + debug(D_ERROR, "vine_graph_supernode_register: member list must not contain leader %" PRIu64, leader_id); + set_delete(seen); + set_delete(mset); + return -1; + } + if (set_lookup(seen, (void *)(uintptr_t)mid)) { + debug(D_ERROR, "vine_graph_supernode_register: duplicate member %" PRIu64, mid); + set_delete(seen); + set_delete(mset); + return -1; + } + + struct vine_graph_node *mn = itable_lookup(g->nodes, mid); + if (!mn) { + debug(D_ERROR, "vine_graph_supernode_register: member node %" PRIu64 " not found", mid); + set_delete(seen); + set_delete(mset); + return -1; + } + if (mn->super_leader_id != mn->node_id) { + debug(D_ERROR, + "vine_graph_supernode_register: node %" PRIu64 " already belongs to leader %" PRIu64, + mid, + mn->super_leader_id); + set_delete(seen); + set_delete(mset); + return -1; + } + set_insert(seen, (void *)(uintptr_t)mid); + set_insert(mset, mn); + } + set_delete(seen); + + if (leader->super_leader_id != leader->node_id) { + debug(D_ERROR, + "vine_graph_supernode_register: leader %" PRIu64 " already belongs to group %" PRIu64, + leader_id, + leader->super_leader_id); + set_delete(mset); + return -1; + } + + if (itable_lookup(g->super_leader_to_members, leader_id)) { + debug(D_ERROR, "vine_graph_supernode_register: leader %" PRIu64 " already registered", leader_id); + set_delete(mset); + return -1; + } + + vine_graph_supernode_rewire(g, leader, mset); + + struct vine_graph_node *x; + uint64_t xid; + ITABLE_ITERATE(g->nodes, xid, x) + { + if (set_lookup(mset, x)) { + x->super_leader_id = leader_id; + vine_graph_node_clear_fired_parents(x); + } + } + + struct list *nonleaders = list_create(); + for (int i = 0; i < n_member_ids; i++) { + struct vine_graph_node *mn = itable_lookup(g->nodes, member_ids[i]); + list_push_tail(nonleaders, mn); + } + itable_insert(g->super_leader_to_members, leader_id, nonleaders); + + set_delete(mset); + return 0; +} + +/** Exactly one child pointer, or NULL if not exactly one. */ +static struct vine_graph_node *vine_graph_node_only_child(struct vine_graph_node *n) +{ + if (!n || list_size(n->children) != 1) { + return NULL; + } + struct vine_graph_node *c; + LIST_ITERATE(n->children, c) + { + return c; + } + return NULL; +} + +/** Exactly one parent pointer, or NULL if not exactly one. */ +static struct vine_graph_node *vine_graph_node_only_parent(struct vine_graph_node *n) +{ + if (!n || list_size(n->parents) != 1) { + return NULL; + } + struct vine_graph_node *p; + LIST_ITERATE(n->parents, p) + { + return p; + } + return NULL; +} + +/** + * Head of a maximal linear chain: singleton, and not strictly inside such a chain from the left + * (no parent, fan-in > 1, or unique parent has fan-out > 1). + */ +static int vine_graph_node_is_chain_head(struct vine_graph_node *n) +{ + if (!vine_graph_node_is_supernode_leader(n)) { + return 0; + } + if (list_size(n->parents) == 0) { + return 1; + } + if (list_size(n->parents) > 1) { + return 1; + } + struct vine_graph_node *p = vine_graph_node_only_parent(n); + return p && list_size(p->children) > 1; +} + +struct pending_chain_group { + uint64_t leader_id; + uint64_t *member_ids; + int n_members; +}; + +int vine_graph_group_chain_like_tasks(struct vine_graph *g) +{ + if (!g) { + return -1; + } + if (!g->chain_grouping_enabled) { + return 0; + } + + /* + * Registering a supernode rewires the graph. Find every chain on the original adjacency + * first, then register them, so a half-updated graph does not confuse later chain walks. + */ + struct list *pending = list_create(); + + uint64_t nid; + struct vine_graph_node *n; + ITABLE_ITERATE(g->nodes, nid, n) + { + if (!vine_graph_node_is_chain_head(n)) { + continue; + } + if (list_size(n->children) != 1) { + continue; + } + + struct list *chain = list_create(); + struct vine_graph_node *cur = n; + for (;;) { + list_push_tail(chain, cur); + if (list_size(cur->children) != 1) { + break; + } + struct vine_graph_node *c = vine_graph_node_only_child(cur); + if (!c || !vine_graph_node_is_supernode_leader(c)) { + break; + } + if (c->is_target) { + /* Leave the retrieval or output consumer as its own task outside the merged chain. */ + break; + } + if (list_size(c->parents) != 1) { + break; + } + if (vine_graph_node_only_parent(c) != cur) { + break; + } + cur = c; + } + + int L = list_size(chain); + if (L < 2) { + list_delete(chain); + continue; + } + + struct pending_chain_group *pc = xxmalloc(sizeof(*pc)); + pc->leader_id = n->node_id; + pc->n_members = L - 1; + pc->member_ids = xxmalloc((size_t)(L - 1) * sizeof(uint64_t)); + int mi = 0; + int first = 1; + struct vine_graph_node *x; + LIST_ITERATE(chain, x) + { + if (first) { + first = 0; + continue; + } + pc->member_ids[mi++] = x->node_id; + } + list_push_tail(pending, pc); + list_delete(chain); + } + + int groups = 0; + struct pending_chain_group *pc; + while ((pc = (struct pending_chain_group *)list_pop_head(pending))) { + if (vine_graph_supernode_register(g, pc->leader_id, pc->member_ids, pc->n_members) == 0) { + groups++; + } + free(pc->member_ids); + free(pc); + } + list_delete(pending); + + return groups; +} + +struct vine_graph_node *vine_graph_input_producer_node(struct vine_graph *g, struct vine_graph_node *parent, struct vine_graph_node *child) +{ + if (!g || !parent || !child) { + return parent; + } + if (!g->chain_grouping_enabled || !g->supernode_leader_child_to_input_source) { + return parent; + } + + char *k = string_format("%" PRIu64 ",%" PRIu64, parent->node_id, child->node_id); + struct vine_graph_node *src = (struct vine_graph_node *)hash_table_lookup(g->supernode_leader_child_to_input_source, k); + free(k); + return src ? src : parent; +} + +/** + * Delete an executor graph instance. + * @param g Reference to the executor graph. + */ +void vine_graph_delete(struct vine_graph *g) +{ + if (!g) { + return; + } + + uint64_t lid; + struct list *mems; + if (g->super_leader_to_members) { + ITABLE_ITERATE(g->super_leader_to_members, lid, mems) + { + (void)lid; + if (mems) { + list_delete(mems); + } + } + itable_delete(g->super_leader_to_members); + g->super_leader_to_members = NULL; + } + + uint64_t nid; + struct vine_graph_node *node; + ITABLE_ITERATE(g->nodes, nid, node) + { + vine_graph_node_delete(node); + } + + free(g->task_runner_library_name); + free(g->task_runner_function_name); + free(g->checkpoint_dir); + free(g->output_dir); + + itable_delete(g->nodes); + hash_table_delete(g->outfile_cachename_to_node); + + hash_table_delete(g->supernode_leader_child_to_input_source); + + hash_table_clear(g->inout_filename_to_cached_name, (void *)free); + hash_table_delete(g->inout_filename_to_cached_name); + + free(g); +} diff --git a/taskvine/src/graph/vine_graph.h b/taskvine/src/graph/vine_graph.h new file mode 100644 index 0000000000..5f4566d648 --- /dev/null +++ b/taskvine/src/graph/vine_graph.h @@ -0,0 +1,157 @@ +#ifndef VINE_GRAPH_H +#define VINE_GRAPH_H + +#include "hash_table.h" +#include "itable.h" + +#include "vine_graph_node.h" + +struct vine_graph { + struct itable *nodes; + /** Maps each supernode leader id to the list of non-leader member nodes. Empty when no groups exist. */ + struct itable *super_leader_to_members; + struct hash_table *outfile_cachename_to_node; + struct hash_table *inout_filename_to_cached_name; + /** + * Maps leader id and downstream child id to the member that actually wrote the outfile the + * child should read. After a rewire the DAG edge may show the leader while the tail member + * still owns the bytes. Keys are decimal node ids joined with a comma. + */ + struct hash_table *supernode_leader_child_to_input_source; + + char *checkpoint_dir; + char *output_dir; + char *task_runner_library_name; + char *task_runner_function_name; + + double checkpoint_fraction; + int prune_depth; + + int print_graph_details; + /* + * Zero unless the user turned on chain grouping. When zero the executor does not merge + * members onto one task, does not list several scheduler keys in one runner infile, and does + * not remap which node's outfile a consumer should mount. + */ + int chain_grouping_enabled; +}; + +// Public graph API (declarations below) + +/** + * Resolve which producer node's outfile a child should consume for a scheduling parent edge. + * After supernode rewire, parent may be the leader while the file is produced by a member (e.g. tail). + * @return The node that owns the vine_file inputs should mount, or @p parent if there is no override. + */ +struct vine_graph_node *vine_graph_input_producer_node(struct vine_graph *g, struct vine_graph_node *parent, struct vine_graph_node *child); + +/** Create an executor graph and return it. +@param runtime_dir Runtime directory used for default graph output paths. +@return A new executor graph. +*/ +struct vine_graph *vine_graph_create(const char *runtime_dir); + +/** Create a new node in the executor graph. +@param g Reference to the executor graph. +@return The auto-assigned node id. +*/ +uint64_t vine_graph_add_node(struct vine_graph *g); + +/** Mark a node as a retrieval target. +@param g Reference to the executor graph. +@param node_id Identifier of the node to mark as target. +*/ +void vine_graph_set_target(struct vine_graph *g, uint64_t node_id); + +/** Add a dependency between two nodes in the executor graph. +@param g Reference to the executor graph. +@param parent_id Identifier of the parent node. +@param child_id Identifier of the child node. +*/ +void vine_graph_add_dependency(struct vine_graph *g, uint64_t parent_id, uint64_t child_id); + +/** Finalize the metrics of the executor graph. +@param g Reference to the executor graph. +*/ +void vine_graph_finalize(struct vine_graph *g); + +/** Get the heavy score of a node in the executor graph. +@param g Reference to the executor graph. +@param node_id Identifier of the node. +@return The heavy score. +*/ +double vine_graph_get_node_heavy_score(const struct vine_graph *g, uint64_t node_id); + +/** Get the outfile remote name of a node in the executor graph. +@param g Reference to the executor graph. +@param node_id Identifier of the node. +@return The outfile remote name. +*/ +const char *vine_graph_get_node_outfile_remote_name(const struct vine_graph *g, uint64_t node_id); + +/** Delete an executor graph. +@param g Reference to the executor graph. +*/ +void vine_graph_delete(struct vine_graph *g); + +/** Get the task runner library name of the executor graph. +@param g Reference to the executor graph. +@return The task runner library name. +*/ +const char *vine_graph_get_task_runner_library_name(const struct vine_graph *g); + +/** Set the task runner function name of the executor graph. +@param g Reference to the executor graph. +@param task_runner_function_name Reference to the task runner function name. +*/ +void vine_graph_set_task_runner_function_name(struct vine_graph *g, const char *task_runner_function_name); + +/** Tune the executor graph. +@param g Reference to the executor graph. +@param name Reference to the name of the parameter to tune. +@param value Reference to the value of the parameter to tune. +@return 0 on success, -1 on failure. +*/ +int vine_graph_tune(struct vine_graph *g, const char *name, const char *value); + +/** + * True if this node may submit a TaskVine task: singletons and supernode leaders only. + * Non-leader members run inside the leader's runner and must not be scheduled separately. + * Returns non-zero when n is non-NULL and node_id equals super_leader_id. + */ +static inline int vine_graph_node_is_supernode_leader(const struct vine_graph_node *n) +{ + return n && n->node_id == n->super_leader_id; +} + +/** + * Look up the leader node for n using n->super_leader_id (identity when n is already the leader). + * Returns the leader struct vine_graph_node *, or NULL on invalid input or if the leader id is missing from g. + */ +struct vine_graph_node *vine_graph_supernode_leader_node(struct vine_graph *g, struct vine_graph_node *n); + +/** + * Merge nodes into one supernode: rewire external edges to leader_id and set super_leader_id on all members. + * Call after all plain vine_graph_add_dependency edges exist and before vine_graph_executor_finalize. + * Every involved node must currently be its own group (super_leader_id == node_id); member_ids must list + * only non-leader members (not leader_id) without duplicates. Returns 0 on success, -1 on error. + */ +int vine_graph_supernode_register(struct vine_graph *g, uint64_t leader_id, const uint64_t *member_ids, int n_member_ids); + +/** + * Non-leader members for a registered supernode as a list of struct vine_graph_node *. + * Do not free the list; it is owned by g. Returns NULL if g is NULL or no group was registered for that leader. + */ +struct list *vine_graph_supernode_nonleader_members(struct vine_graph *g, uint64_t leader_id); + +/** + * Collapse each maximal singleton linear chain into one supernode (leader = chain head). + * A chain is a path n0->n1->... where each n_i (i > 0) has exactly one parent (n_{i-1}) and + * each n_i (i < last) has exactly one child (n_{i+1}); n0 is not preceded by such an edge from a + * single-child parent (head: no parent, multiple parents, or parent with multiple children). + * Call after all vine_graph_add_dependency edges and before vine_graph_executor_finalize. + * Returns the number of supernodes registered, or -1 if g is NULL. + */ +int vine_graph_group_chain_like_tasks(struct vine_graph *g); + +#endif // VINE_GRAPH_H diff --git a/taskvine/src/graph/vine_graph.i b/taskvine/src/graph/vine_graph.i new file mode 100644 index 0000000000..e546ec57c4 --- /dev/null +++ b/taskvine/src/graph/vine_graph.i @@ -0,0 +1,21 @@ +/* SWIG interface for local executor graph API bindings */ +%module vine_graph_capi + +%{ +#include "int_sizes.h" +#include "vine_graph.h" +#include "vine_graph_executor.h" +%} + +%include "stdint.i" +%include "int_sizes.h" + +/* uint64_t[] + length: not mapped yet; expose via VineGraphExecutor when needed. */ +%ignore vine_graph_supernode_register; +%ignore vine_graph_supernode_nonleader_members; + +/* Import existing SWIG interface for type information (do not wrap again) */ +%import "../bindings/python3/taskvine.i" + +%include "vine_graph.h" +%include "vine_graph_executor.h" diff --git a/taskvine/src/graph/vine_graph_executor.c b/taskvine/src/graph/vine_graph_executor.c new file mode 100644 index 0000000000..cb95969644 --- /dev/null +++ b/taskvine/src/graph/vine_graph_executor.c @@ -0,0 +1,1473 @@ +#include +#include +#include +#include +#include +#include + +#include "buffer.h" +#include "debug.h" +#include "vine_graph_executor.h" +#include "macros.h" +#include "progress_bar.h" +#include "random.h" +#include "set.h" +#include "stringtools.h" +#include "xxmalloc.h" + +#include "vine_file.h" +#include "vine_mount.h" +#include "vine_task.h" +#include "vine_temp.h" + +static volatile sig_atomic_t interrupted = 0; + +static void vine_graph_executor_submit_node(struct vine_graph_executor *e, struct vine_graph_node *node); +static struct vine_task *vine_graph_executor_make_vine_task(struct vine_graph_executor *e); +static void vine_graph_executor_materialize_node(struct vine_graph_executor *e, struct vine_graph_node *node); +static void vine_graph_executor_run_completion_postprocess(struct vine_graph_executor *e, struct vine_graph_node *node); + +static uint64_t vine_graph_executor_count_completed_user_nodes(const struct vine_graph *g) +{ + uint64_t n = 0; + uint64_t nid; + struct vine_graph_node *nd; + + if (!g) { + return 0; + } + ITABLE_ITERATE(g->nodes, nid, nd) + { + if (nd->completed) { + n++; + } + } + return n; +} + +/* + * The leader's task has passed validation. Mark that node completed and, when chain grouping + * applies, mark every non-leader in the same group. Those members did not receive their own + * vine tasks because they ran inside the leader's single submission. + */ +static void vine_graph_executor_mark_user_node_completed_after_success(struct vine_graph *g, struct vine_graph_node *leader) +{ + if (!g || !leader) { + return; + } + + leader->completed = 1; + + if (!g->chain_grouping_enabled) { + return; + } + + struct list *smems = vine_graph_supernode_nonleader_members(g, leader->node_id); + if (!smems) { + return; + } + + struct vine_graph_node *m; + LIST_ITERATE(smems, m) + { + if (!m->completed) { + m->completed = 1; + } + } +} + +/* + * Build the JSON payload for the runner infile argument. Field fn_args[0] names the scheduler + * keys that run_scheduler_keys should execute. A merged chain passes one comma-separated string + * such as "1,2,3". A singleton passes a single id. The payload must be one JSON string in that + * slot instead of an array that mixes strings and bare numbers. + */ +static char *vine_graph_executor_format_runner_infile_json(struct vine_graph *g, struct vine_graph_node *node) +{ + if (!g || !node) { + return NULL; + } + + if (!g->chain_grouping_enabled) { + return string_format("{\"fn_args\":[\"%" PRIu64 "\"],\"fn_kwargs\":{}}", node->node_id); + } + + struct list *mems = vine_graph_supernode_nonleader_members(g, node->node_id); + if (!mems || list_size(mems) == 0) { + return string_format("{\"fn_args\":[\"%" PRIu64 "\"],\"fn_kwargs\":{}}", node->node_id); + } + + buffer_t buf; + buffer_init(&buf); + buffer_printf(&buf, "{\"fn_args\":[\"%" PRIu64 "", node->node_id); + struct vine_graph_node *m; + LIST_ITERATE(mems, m) + { + buffer_printf(&buf, ",%" PRIu64 "", m->node_id); + } + buffer_printf(&buf, "\"],\"fn_kwargs\":{}}"); + + char *s = xxstrdup(buffer_tostring(&buf)); + buffer_free(&buf); + return s; +} + +static void vine_graph_io_mount_add(struct list *lst, struct vine_file *f, const char *remote_name) +{ + struct vine_graph_io_mount *m = xxmalloc(sizeof(*m)); + m->file = f; + m->remote_name = xxstrdup(remote_name); + list_push_tail(lst, m); +} + +/* Undeclare runner infile buffer (before discarding the vine_task). */ +static void vine_graph_executor_clear_node_runner_arg(struct vine_graph_executor *e, struct vine_graph_node *node) +{ + if (!e || !node || !node->task_runner_arg_file) { + return; + } + vine_undeclare_file(e->manager, node->task_runner_arg_file); + node->task_runner_arg_file = NULL; +} + +static void vine_graph_executor_undeclare_extra_io_mounts(struct vine_graph_executor *e, struct list *mounts, struct set *seen_files) +{ + if (!e || !mounts || !seen_files) { + return; + } + + struct vine_graph_io_mount *m; + LIST_ITERATE(mounts, m) + { + if (!m || !m->file) { + continue; + } + if (!set_lookup(seen_files, m->file)) { + set_insert(seen_files, m->file); + vine_undeclare_file(e->manager, m->file); + } + m->file = NULL; + } +} + +/* Initialize runtime fields and default tuning values for a new executor. */ +static void vine_graph_executor_init_runtime(struct vine_graph_executor *e) +{ + if (!e) { + return; + } + + e->task_id_to_node = itable_create(0); + e->resubmit_queue = list_create(); + e->time_first_task_dispatched = UINT64_MAX; // sentinel until first task commit time + e->time_last_task_retrieved = 0; + e->makespan_us = 0; + e->completed_recovery_tasks = 0; + e->time_spent_on_cut_propagation = 0; + e->pfs_usage_bytes = 0; + e->total_preprocessing_time_us = 0; + e->total_postprocessing_time_us = 0; + e->task_priority_mode = TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST; + e->failure_injection_step_percent = -1.0; + e->progress_bar_update_interval_sec = 0.1; + e->enable_debug_log = 1; +} + +/* Release the task-id lookup table and the resubmit queue. */ +static void vine_graph_executor_clear_runtime(struct vine_graph_executor *e) +{ + if (!e) { + return; + } + if (e->task_id_to_node) { + itable_delete(e->task_id_to_node); + e->task_id_to_node = NULL; + } + if (e->resubmit_queue) { + list_delete(e->resubmit_queue); + e->resubmit_queue = NULL; + } +} + +/* Allocate an executor bound to the given manager and graph. */ +struct vine_graph_executor *vine_graph_executor_create(struct vine_manager *manager, struct vine_graph *graph) +{ + if (!manager || !graph) { + return NULL; + } + + struct vine_graph_executor *e = malloc(sizeof(*e)); + if (!e) { + return NULL; + } + + e->graph = graph; + e->manager = manager; + vine_graph_executor_init_runtime(e); + return e; +} + +/* Create a new graph for the manager and enable debug logging under the runtime directory. */ +struct vine_graph *vine_graph_executor_create_graph(struct vine_manager *manager) +{ + if (!manager) { + return NULL; + } + + char *debug_tmp = string_format("%s/vine-logs/debug", manager->runtime_directory); + vine_enable_debug_log(debug_tmp); + free(debug_tmp); + + return vine_graph_create(manager->runtime_directory); +} + +/* Undeclare managed files, remove local outputs, and free the executor. */ +void vine_graph_executor_delete(struct vine_graph_executor *e) +{ + struct vine_graph *g = e ? e->graph : NULL; + if (g && e->manager) { + struct set *extra_files = set_create(0); + uint64_t nid; + struct vine_graph_node *node; + ITABLE_ITERATE(g->nodes, nid, node) + { + if (node->task_runner_arg_file) { + vine_undeclare_file(e->manager, node->task_runner_arg_file); // before graph free to avoid double free + node->task_runner_arg_file = NULL; + } + switch (node->outfile_type) { + case VINE_GRAPH_NODE_OUTFILE_TYPE_TEMP: + break; + case VINE_GRAPH_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: + if (node->outfile_remote_name) { + unlink(node->outfile_remote_name); + } + break; + case VINE_GRAPH_NODE_OUTFILE_TYPE_LOCAL: + if (node->outfile && node->outfile->source) { + unlink(node->outfile->source); + } + break; + } + if (node->outfile) { + hash_table_remove(g->outfile_cachename_to_node, node->outfile->cached_name); + vine_undeclare_file(e->manager, node->outfile); + node->outfile = NULL; + } + vine_graph_executor_undeclare_extra_io_mounts(e, node->extra_inputs, extra_files); + vine_graph_executor_undeclare_extra_io_mounts(e, node->extra_outputs, extra_files); + } + set_delete(extra_files); + } + vine_graph_executor_clear_runtime(e); + free(e); +} + +/* + * Create a new library task (not yet published on the node). The caller attaches IO, then sets + * node->task only when the task is fully configured (atomic materialize). + */ +static struct vine_task *vine_graph_executor_make_vine_task(struct vine_graph_executor *e) +{ + struct vine_graph *g = e ? e->graph : NULL; + if (!g) { + return NULL; + } + + if (!g->task_runner_function_name) { + debug(D_ERROR, "task runner function name is not set"); + vine_graph_delete(g); + exit(1); + } + if (!g->task_runner_library_name) { + debug(D_ERROR, "task runner library name is not set"); + vine_graph_delete(g); + exit(1); + } + + struct vine_task *t = vine_task_create(g->task_runner_function_name); + vine_task_set_library_required(t, g->task_runner_library_name); + vine_task_addref(t); // keep alive across vine_submit and vine_wait + return t; +} + +/* + * Attach inputs, outputs, and the infile buffer to a new vine_task at submit time. The node's + * task pointer stays NULL until that bundle is complete and ready for vine_submit. + */ +static void vine_graph_executor_materialize_node(struct vine_graph_executor *e, struct vine_graph_node *node) +{ + struct vine_graph *g = e ? e->graph : NULL; + if (!g || !node) { + return; + } + + if (node->task && node->task->state != VINE_TASK_INITIAL) { + return; + } + /* INITIAL task implies mounts + runner infile are already complete. */ + if (node->task) { + return; + } + + vine_graph_executor_clear_node_runner_arg(e, node); + + struct vine_task *t = vine_graph_executor_make_vine_task(e); + if (!t) { + return; + } + + if (node->outfile) { + vine_task_add_output(t, node->outfile, node->outfile_remote_name, VINE_TRANSFER_ALWAYS); + } + + void *item; + LIST_ITERATE(node->extra_outputs, item) + { + struct vine_graph_io_mount *m = (struct vine_graph_io_mount *)item; + vine_task_add_output(t, m->file, m->remote_name, VINE_TRANSFER_ALWAYS); + } + + /* + * When chains are merged, only the leader submits a task, but that task must list every + * member's declared outputs so the manager can track each outfile by node id. + */ + if (g->chain_grouping_enabled && vine_graph_node_is_supernode_leader(node)) { + struct list *mems = vine_graph_supernode_nonleader_members(g, node->node_id); + if (mems) { + struct vine_graph_node *m; + LIST_ITERATE(mems, m) + { + if (m->outfile) { + vine_task_add_output(t, m->outfile, m->outfile_remote_name, VINE_TRANSFER_ALWAYS); + } + LIST_ITERATE(m->extra_outputs, item) + { + struct vine_graph_io_mount *em = (struct vine_graph_io_mount *)item; + vine_task_add_output(t, em->file, em->remote_name, VINE_TRANSFER_ALWAYS); + } + } + } + } + + struct vine_graph_node *parent_node; + LIST_ITERATE(node->parents, parent_node) + { + struct vine_graph_node *src = vine_graph_input_producer_node(g, parent_node, node); + if (src && src->outfile) { + vine_task_add_input(t, src->outfile, src->outfile_remote_name, VINE_TRANSFER_ALWAYS); + } + } + + LIST_ITERATE(node->extra_inputs, item) + { + struct vine_graph_io_mount *m = (struct vine_graph_io_mount *)item; + vine_task_add_input(t, m->file, m->remote_name, VINE_TRANSFER_ALWAYS); + } + + char *task_arguments = vine_graph_executor_format_runner_infile_json(g, node); + if (!task_arguments) { + goto fail_task; + } + struct vine_file *arg_file = + vine_declare_buffer(e->manager, task_arguments, strlen(task_arguments), VINE_CACHE_LEVEL_TASK, VINE_UNLINK_WHEN_DONE); + free(task_arguments); + if (!arg_file) { + goto fail_task; + } + vine_task_add_input(t, arg_file, "infile", VINE_TRANSFER_ALWAYS); + + node->task = t; + node->task_runner_arg_file = arg_file; + return; + +fail_task: + vine_task_delete(t); +} + +/* + * Declare the output vine_file for this node from outfile_type. + * Shared filesystem outputs may leave outfile unset. + */ +static void vine_graph_executor_declare_node_outfile(struct vine_graph_executor *e, struct vine_graph_node *node) +{ + struct vine_graph *g = e ? e->graph : NULL; + if (!g || !node || node->outfile) { + return; + } + + switch (node->outfile_type) { + case VINE_GRAPH_NODE_OUTFILE_TYPE_LOCAL: { + char *local_outfile_path = string_format("%s/%s", g->output_dir, node->outfile_remote_name); + node->outfile = vine_declare_file(e->manager, local_outfile_path, VINE_CACHE_LEVEL_WORKFLOW, 0); + free(local_outfile_path); + break; + } + case VINE_GRAPH_NODE_OUTFILE_TYPE_TEMP: + node->outfile = vine_declare_temp(e->manager); + break; + case VINE_GRAPH_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: + break; + } +} + +/* + * Allocate the next graph node. Per-node vine_task and I/O mounts appear later during + * vine_graph_executor_materialize_node at submit time. + */ +uint64_t vine_graph_executor_add_node(struct vine_graph_executor *e) +{ + if (!e || !e->graph) { + return 0; + } + + uint64_t node_id = vine_graph_add_node(e->graph); + return node_id; +} + +/* + * Finalize the graph: declare outputs with cached_name registration, + * attach parent inputs, and set remaining parent counts for scheduling. + */ +void vine_graph_executor_finalize(struct vine_graph_executor *e) +{ + struct vine_graph *g = e ? e->graph : NULL; + if (!g) { + return; + } + + vine_graph_finalize(g); + + /* + * Two passes. Declare outputs and cached_name map first so parent + * vine_file objects exist. Task-level input/output mounts are applied + * in vine_graph_executor_materialize_node at submit time. + */ + uint64_t nid; + struct vine_graph_node *node; + ITABLE_ITERATE(g->nodes, nid, node) + { + vine_graph_executor_declare_node_outfile(e, node); + if (node->outfile) { + hash_table_insert(g->outfile_cachename_to_node, node->outfile->cached_name, node); + } + } + + ITABLE_ITERATE(g->nodes, nid, node) + { + node->remaining_parents_count = list_size(node->parents); + } +} + +/* Add a named input, reusing a declared file when the logical filename was already mapped. */ +void vine_graph_executor_add_task_input(struct vine_graph_executor *e, uint64_t task_id, const char *filename) +{ + struct vine_graph *g = e ? e->graph : NULL; + if (!g || !task_id || !filename) { + return; + } + + struct vine_graph_node *node = itable_lookup(g->nodes, task_id); + if (!node) { + return; + } + + struct vine_file *f = NULL; + const char *cached_name = hash_table_lookup(g->inout_filename_to_cached_name, filename); + + if (cached_name) { + f = vine_manager_lookup_file(e->manager, cached_name); + } else { + f = vine_declare_temp(e->manager); // first use of logical name, record cache key for paired mounts + hash_table_insert(g->inout_filename_to_cached_name, filename, xxstrdup(f->cached_name)); + } + + vine_graph_io_mount_add(node->extra_inputs, f, filename); +} + +/* Add a named output, linking logical filename to a cache name for paired producer and consumer tasks. */ +void vine_graph_executor_add_task_output(struct vine_graph_executor *e, uint64_t task_id, const char *filename) +{ + struct vine_graph *g = e ? e->graph : NULL; + if (!g || !task_id || !filename) { + return; + } + + struct vine_graph_node *node = itable_lookup(g->nodes, task_id); + if (!node) { + return; + } + + struct vine_file *f = NULL; + const char *cached_name = hash_table_lookup(g->inout_filename_to_cached_name, filename); + + if (cached_name) { + f = vine_manager_lookup_file(e->manager, cached_name); + } else { + f = vine_declare_temp(e->manager); // pair with matching input on consumer tasks + hash_table_insert(g->inout_filename_to_cached_name, filename, xxstrdup(f->cached_name)); + } + + vine_graph_io_mount_add(node->extra_outputs, f, filename); +} + +/* Apply executor-level tuning. Unknown keys are forwarded to vine_graph_tune. */ +int vine_graph_executor_tune(struct vine_graph_executor *e, const char *name, const char *value) +{ + if (!e || !name || !value) { + return -1; + } + + if (strcmp(name, "failure-injection-step-percent") == 0) { + e->failure_injection_step_percent = atof(value); + + } else if (strcmp(name, "task-priority-mode") == 0) { + if (strcmp(value, "random") == 0) { + e->task_priority_mode = TASK_PRIORITY_MODE_RANDOM; + } else if (strcmp(value, "depth-first") == 0) { + e->task_priority_mode = TASK_PRIORITY_MODE_DEPTH_FIRST; + } else if (strcmp(value, "breadth-first") == 0) { + e->task_priority_mode = TASK_PRIORITY_MODE_BREADTH_FIRST; + } else if (strcmp(value, "fifo") == 0) { + e->task_priority_mode = TASK_PRIORITY_MODE_FIFO; + } else if (strcmp(value, "lifo") == 0) { + e->task_priority_mode = TASK_PRIORITY_MODE_LIFO; + } else if (strcmp(value, "largest-input-first") == 0) { + e->task_priority_mode = TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST; + } else if (strcmp(value, "largest-storage-footprint-first") == 0) { + e->task_priority_mode = TASK_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST; + } else { + debug(D_ERROR, "invalid priority mode: %s", value); + return -1; + } + + } else if (strcmp(name, "progress-bar-update-interval-sec") == 0) { + double val = atof(value); + e->progress_bar_update_interval_sec = (val > 0.0) ? val : 0.1; + + } else if (strcmp(name, "enable-debug-log") == 0) { + if (e->enable_debug_log == 0) { + return -1; + } + e->enable_debug_log = (atoi(value) == 1) ? 1 : 0; + if (e->enable_debug_log == 0) { + debug_flags_clear(); + debug_close(); + } + + } else { + return vine_graph_tune(e->graph, name, value); + } + + return 0; +} + +/* Set the interrupted flag when SIGINT is received. */ +static void vine_graph_executor_handle_sigint(int signal) +{ + interrupted = 1; +} + +/* Compute submission priority for a node using the configured scheduling policy. */ +static double vine_graph_executor_calculate_task_priority(struct vine_graph_executor *e, struct vine_graph_node *node) +{ + struct vine_graph *g = e ? e->graph : NULL; + if (!node || !g) { + return 0; + } + + double priority = 0; + timestamp_t current_time = timestamp_get(); + struct vine_graph_node *parent_node; + + switch (e->task_priority_mode) { + case TASK_PRIORITY_MODE_RANDOM: + priority = random_double(); + break; + case TASK_PRIORITY_MODE_DEPTH_FIRST: + priority = (double)node->depth; + break; + case TASK_PRIORITY_MODE_BREADTH_FIRST: + priority = -(double)node->depth; + break; + case TASK_PRIORITY_MODE_FIFO: + priority = -(double)current_time; // earlier time yields higher priority + break; + case TASK_PRIORITY_MODE_LIFO: + priority = (double)current_time; + break; + case TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST: + LIST_ITERATE(node->parents, parent_node) + { + struct vine_graph_node *src = vine_graph_input_producer_node(g, parent_node, node); + if (!src || !src->outfile) { + continue; + } + priority += (double)vine_file_size(src->outfile); + } + break; + case TASK_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST: + LIST_ITERATE(node->parents, parent_node) + { + struct vine_graph_node *src = vine_graph_input_producer_node(g, parent_node, node); + if (!src || !src->outfile) { + continue; + } + if (!parent_node->task) { + continue; + } + timestamp_t parent_task_completion_time = parent_node->task->time_workers_execute_last; + // weight by parent input size and last observed worker runtime + priority += (double)vine_file_size(src->outfile) * (double)parent_task_completion_time; + } + break; + } + + return priority; +} + +/* Submit the node task if it is still initial, and record the manager task id for later lookup. */ +static void vine_graph_executor_submit_node(struct vine_graph_executor *e, struct vine_graph_node *node) +{ + struct vine_graph *g = e ? e->graph : NULL; + if (!g || !node) { + return; + } + + timestamp_t t_pre = timestamp_get(); + + vine_graph_executor_materialize_node(e, node); + + if (!node->task) { + debug(D_ERROR, "vine_graph_executor_submit_node: node %" PRIu64 " has no task after materialize", node->node_id); + goto record_preprocessing; + } + + if (node->task->state != VINE_TASK_INITIAL) { + debug(D_VINE, "vine_graph_executor_submit_node: skipping node %" PRIu64 " (task already submitted, state=%d, task_id=%d)", node->node_id, node->task->state, node->task->task_id); + goto record_preprocessing; + } + + double priority = vine_graph_executor_calculate_task_priority(e, node); + vine_task_set_priority(node->task, priority); + + int task_id = vine_submit(e->manager, node->task); + + if (task_id <= 0) { + debug(D_ERROR, "vine_graph_executor_submit_node: failed to submit node %" PRIu64 " (returned task_id=%d)", node->node_id, task_id); + goto record_preprocessing; + } + + itable_insert(e->task_id_to_node, (uint64_t)task_id, node); // reverse lookup from vine_wait + debug(D_VINE, "submitted node %" PRIu64 " with task id %d", node->node_id, task_id); + +record_preprocessing: { + uint64_t dt = (uint64_t)(timestamp_get() - t_pre); + node->preprocessing_time_us = dt; + e->total_preprocessing_time_us += dt; + debug(D_VINE, + "node %" PRIu64 " preprocessing %" PRIu64 " us, graph cumulative %" PRIu64 " us", + node->node_id, + dt, + e->total_preprocessing_time_us); +} +} + +/* + * Return true when this node is allowed to submit. If chain grouping is active, only the + * supernode leader may submit because other members are executed inside the leader's task. + */ +static int vine_graph_node_ready_for_submission(struct vine_graph_executor *e, struct vine_graph_node *node) +{ + struct vine_graph *g = e ? e->graph : NULL; + if (!e || !node || node->remaining_parents_count != 0 || node->completed) { + return 0; + } + if (g && g->chain_grouping_enabled && !vine_graph_node_is_supernode_leader(node)) { + return 0; + } + if (node->in_resubmit_queue) { + return 0; + } + if (node->task && node->task->state != VINE_TASK_INITIAL) { + return 0; + } + return 1; +} + +/* Submit ready source nodes and enable delivery of recovery tasks to the application. */ +static void vine_graph_executor_submit_initial_ready_nodes(struct vine_graph_executor *e) +{ + struct vine_graph *g = e ? e->graph : NULL; + if (!g || !e->manager) { + return; + } + + uint64_t nid; + struct vine_graph_node *node; + ITABLE_ITERATE(g->nodes, nid, node) + { + if (vine_graph_node_ready_for_submission(e, node)) { + vine_graph_executor_submit_node(e, node); + } + } + + vine_enable_return_recovery_tasks(e->manager); // driver must observe recovery completions for cut and prune +} + +/* After one parent completes, decrement remaining parents and submit children that become ready. */ +static void vine_graph_executor_submit_unblocked_children(struct vine_graph_executor *e, struct vine_graph_node *node) +{ + struct vine_graph *g = e ? e->graph : NULL; + if (!g || !node) { + return; + } + + struct vine_graph_node *child_node; + LIST_ITERATE(node->children, child_node) + { + if (!child_node) { + continue; + } + + if (!child_node->fired_parents) { + child_node->fired_parents = set_create(0); + } + if (set_lookup(child_node->fired_parents, node)) { + continue; + } + set_insert(child_node->fired_parents, node); + + if (child_node->remaining_parents_count > 0) { + child_node->remaining_parents_count--; + } + + if (vine_graph_node_ready_for_submission(e, child_node)) { + vine_graph_executor_submit_node(e, child_node); + } + } +} + +/* Map a completed vine_task to the corresponding graph node, including recovery tasks. */ +static struct vine_graph_node *vine_graph_executor_node_from_task(struct vine_graph_executor *e, struct vine_task *task) +{ + struct vine_graph *g = e ? e->graph : NULL; + if (!g || !task) { + return NULL; + } + + if (task->type == VINE_TASK_TYPE_STANDARD) { + return itable_lookup(e->task_id_to_node, (uint64_t)task->task_id); + } else if (task->type == VINE_TASK_TYPE_RECOVERY) { + /* + * Recovery tasks are not stored in task_id_to_node. + * Resolve the logical node via the original producer id on output mounts. + */ + struct vine_mount *mount; + LIST_ITERATE(task->output_mounts, mount) + { + if (mount->file && mount->file->cached_name) { + struct vine_graph_node *producer = hash_table_lookup(g->outfile_cachename_to_node, mount->file->cached_name); + if (producer) { + return producer; + } + } + if (!mount->file) { + continue; + } + uint64_t original_producer_task_id = mount->file->original_producer_task_id; + if (original_producer_task_id > 0) { + return itable_lookup(e->task_id_to_node, original_producer_task_id); + } + } + } + + debug(D_ERROR, "task %d has no original producer task id", task->task_id); + return NULL; +} + +/* Update shared-filesystem byte counters when a node's credited output size changes. */ +static void vine_graph_executor_account_pfs_write(struct vine_graph_executor *e, struct vine_graph_node *n, size_t new_size) +{ + struct vine_graph *g = e ? e->graph : NULL; + if (!g || !n) { + return; + } + + size_t prev = n->pfs_credited_bytes; + if (new_size == prev) { + return; + } + + if (new_size > prev) { + e->pfs_usage_bytes += (new_size - prev); + } else { + e->pfs_usage_bytes -= (prev - new_size); // retry may produce a smaller file + } + n->pfs_credited_bytes = new_size; + + debug(D_VINE, + "pfs write: node %" PRIu64 " size=%zu (prev=%zu) usage=%" PRIu64, + n->node_id, + new_size, + prev, + e->pfs_usage_bytes); +} + +/* Remove a node's credited bytes from the shared-filesystem usage total. */ +static void vine_graph_executor_account_pfs_delete(struct vine_graph_executor *e, struct vine_graph_node *n) +{ + struct vine_graph *g = e ? e->graph : NULL; + if (!g || !n) { + return; + } + + size_t credited = n->pfs_credited_bytes; + if (credited == 0) { + return; + } + + e->pfs_usage_bytes -= credited; + n->pfs_credited_bytes = 0; + + debug(D_VINE, + "pfs delete: node %" PRIu64 " size=%zu usage=%" PRIu64, + n->node_id, + credited, + e->pfs_usage_bytes); +} + +/* Return non-zero if the completed node retains output on local disk or a shared filesystem path. */ +static int vine_graph_node_is_anchored(const struct vine_graph_node *n) +{ + if (!n || !n->completed) { + return 0; + } + return n->outfile_type == VINE_GRAPH_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM || n->outfile_type == VINE_GRAPH_NODE_OUTFILE_TYPE_LOCAL; +} + +/* Return non-zero when a temporary output file has a recovery task that is neither initial nor finished. */ +static int vine_graph_node_is_mid_recovery(const struct vine_graph_node *n) +{ + if (!n || !n->outfile || n->outfile->type != VINE_TEMP) { + return 0; + } + struct vine_task *rt = n->outfile->recovery_task; + if (!rt) { + return 0; + } + return rt->state != VINE_TASK_INITIAL && rt->state != VINE_TASK_DONE; // recovery in flight +} + +/* Remove or prune the node's result file according to its output storage mode. */ +static void vine_graph_executor_delete_node_output(struct vine_graph_executor *e, struct vine_graph_node *n) +{ + struct vine_graph *g = e ? e->graph : NULL; + if (!g || !n) { + return; + } + + switch (n->outfile_type) { + case VINE_GRAPH_NODE_OUTFILE_TYPE_TEMP: + if (n->outfile) { + vine_prune_file(e->manager, n->outfile); + } + break; + case VINE_GRAPH_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: + vine_graph_executor_account_pfs_delete(e, n); + if (n->outfile_remote_name) { + unlink(n->outfile_remote_name); + } + break; + case VINE_GRAPH_NODE_OUTFILE_TYPE_LOCAL: + if (n->outfile && n->outfile->source) { + unlink(n->outfile->source); + } + break; + } +} + +/* Attempt to mark a completed node as cut and delete its return file when all children permit release. */ +static int vine_graph_executor_try_cut_node(struct vine_graph_executor *e, struct vine_graph_node *n) +{ + struct vine_graph *g = e ? e->graph : NULL; + if (!g || !n || n->cut || !n->completed) { + return 0; + } + + struct vine_graph_node *c; + LIST_ITERATE(n->children, c) + { + if ((!vine_graph_node_is_anchored(c) && !c->cut) || vine_graph_node_is_mid_recovery(c)) { + return 0; // wait for anchored, cut, or non-recovery children + } + } + + n->cut = 1; + debug(D_VINE, "cut: node %" PRIu64 " outfile_type=%d is_target=%d", n->node_id, n->outfile_type, n->is_target); + + if (!n->is_target) { + vine_graph_executor_delete_node_output(e, n); // targets keep data for retrieval + } + + return 1; +} + +/* Walk upstream from a completed node and apply cut propagation along the worklist. */ +static void vine_graph_executor_propagate_cut_from(struct vine_graph_executor *e, struct vine_graph_node *start) +{ + struct vine_graph *g = e ? e->graph : NULL; + if (!g || !start || !start->completed) { + return; + } + + timestamp_t t0 = timestamp_get(); + vine_graph_executor_try_cut_node(e, start); + + /* Upstream BFS: when a node is cut, enqueue its parents for the same check. */ + struct list *worklist = list_create(); + struct vine_graph_node *p; + LIST_ITERATE(start->parents, p) + { + list_push_tail(worklist, p); + } + + while (list_size(worklist) > 0) { + struct vine_graph_node *m = list_pop_head(worklist); + if (vine_graph_executor_try_cut_node(e, m)) { + LIST_ITERATE(m->parents, p) + { + list_push_tail(worklist, p); + } + } + } + + list_delete(worklist); + e->time_spent_on_cut_propagation += timestamp_get() - t0; +} + +/* Return non-zero if every descendant within the given depth bound is complete and not mid-recovery. */ +static int vine_graph_node_descendants_completed_within_depth(struct vine_graph_node *a, int depth) +{ + if (!a || depth <= 0) { + return 1; + } + + struct set *visited = set_create(0); + struct list *current = list_create(); + list_push_tail(current, a); + set_insert(visited, a); + + int ok = 1; + /* Expand one child frontier per iteration up to depth hops from a. */ + for (int d = 0; d < depth && ok; d++) { + struct list *next = list_create(); + struct vine_graph_node *n; + LIST_ITERATE(current, n) + { + struct vine_graph_node *c; + LIST_ITERATE(n->children, c) + { + if (set_lookup(visited, c)) { + continue; + } + set_insert(visited, c); + if (!c->completed || vine_graph_node_is_mid_recovery(c)) { + ok = 0; + break; + } + list_push_tail(next, c); + } + if (!ok) { + break; + } + } + list_delete(current); + current = next; + } + + list_delete(current); + set_delete(visited); + return ok; +} + +/* Release a temporary output when prune-depth constraints and descendant completion are satisfied. */ +static void vine_graph_executor_try_prune_depth_release(struct vine_graph_executor *e, struct vine_graph_node *a) +{ + struct vine_graph *g = e ? e->graph : NULL; + if (!g || !a) { + return; + } + if (a->released_by_prune_depth) { + return; + } + if (a->outfile_type != VINE_GRAPH_NODE_OUTFILE_TYPE_TEMP) { + return; + } + if (a->is_target) { + return; + } + if (!a->completed || !a->outfile) { + return; + } + if (!vine_graph_node_descendants_completed_within_depth(a, g->prune_depth)) { + return; // wait until descendants within prune_depth layers are settled + } + + vine_graph_executor_delete_node_output(e, a); + a->released_by_prune_depth = 1; + + debug(D_VINE, "prune-depth release: node %" PRIu64 " depth=%d", a->node_id, g->prune_depth); +} + +/* Apply prune-depth release starting at a node and extending up to k ancestor levels. */ +static void vine_graph_executor_apply_prune_depth_from(struct vine_graph_executor *e, struct vine_graph_node *node) +{ + struct vine_graph *g = e ? e->graph : NULL; + if (!g || !node) { + return; + } + int k = g->prune_depth; + if (k <= 0) { + return; + } + + vine_graph_executor_try_prune_depth_release(e, node); + + struct set *visited = set_create(0); + struct list *current = list_create(); + list_push_tail(current, node); + set_insert(visited, node); + + /* Visit new parents up to k levels, trying prune release on each. */ + for (int d = 1; d <= k; d++) { + struct list *next = list_create(); + struct vine_graph_node *n; + LIST_ITERATE(current, n) + { + struct vine_graph_node *p; + LIST_ITERATE(n->parents, p) + { + if (set_lookup(visited, p)) { + continue; + } + set_insert(visited, p); + list_push_tail(next, p); + vine_graph_executor_try_prune_depth_release(e, p); + } + } + list_delete(current); + current = next; + } + + list_delete(current); + set_delete(visited); +} + +/* + * Completion hook after a node finishes: cut propagation, prune-depth handling, and timing. + * Postprocessing wall time is charged to the node that triggered this call. + */ +static void vine_graph_executor_run_completion_postprocess(struct vine_graph_executor *e, struct vine_graph_node *node) +{ + if (!e || !node) { + return; + } + + timestamp_t t0 = timestamp_get(); + vine_graph_executor_propagate_cut_from(e, node); + vine_graph_executor_apply_prune_depth_from(e, node); + uint64_t dt = (uint64_t)(timestamp_get() - t0); + node->postprocessing_time_us = dt; + e->total_postprocessing_time_us += dt; + debug(D_VINE, + "node %" PRIu64 " postprocessing %" PRIu64 " us, graph cumulative %" PRIu64 " us", + node->node_id, + dt, + e->total_postprocessing_time_us); +} + +#define RESUBMIT_SCAN_LIMIT 100 +#define RESUBMIT_COOLDOWN_USECS ((timestamp_t)1000000) + +/* + * Queue a retry after failure. With chain grouping the queue stores the leader so one retry + * resubmits the whole merged task. Without grouping the failing node itself is the leader. + */ +static void vine_graph_executor_queue_node_retry(struct vine_graph_executor *e, struct vine_graph_node *node) +{ + struct vine_graph *g = e ? e->graph : NULL; + if (!g || !node) { + return; + } + + struct vine_graph_node *leader = node; + if (g->chain_grouping_enabled) { + struct vine_graph_node *mapped = vine_graph_supernode_leader_node(g, node); + if (mapped) { + leader = mapped; + } + } + if (!leader) { + return; + } + + if (leader->in_resubmit_queue) { + return; + } + + leader->last_failure_time = timestamp_get(); + list_push_tail(e->resubmit_queue, leader); + leader->in_resubmit_queue = 1; +} + +/* + * Process the resubmit queue after cooldown. A ready head is popped, its failed task is torn down, + * and vine_graph_executor_submit_node runs again. If the head is still cooling off, rotate it to the tail so + * other leaders behind it are not stuck forever while the driver waits. + */ +static void vine_graph_executor_drain_resubmit_queue(struct vine_graph_executor *e) +{ + struct vine_graph *g = e ? e->graph : NULL; + if (!g) { + return; + } + + timestamp_t now = timestamp_get(); + int queued = list_size(e->resubmit_queue); + if (queued == 0) { + return; + } + + int budget = queued < RESUBMIT_SCAN_LIMIT ? queued : RESUBMIT_SCAN_LIMIT; + int resubmits = 0; + int rotations_without_resubmit = 0; + + while (resubmits < budget && list_size(e->resubmit_queue) > 0) { + struct vine_graph_node *node = list_peek_head(e->resubmit_queue); + if (!node) { + break; + } + if (now - node->last_failure_time >= RESUBMIT_COOLDOWN_USECS) { + list_pop_head(e->resubmit_queue); + node->in_resubmit_queue = 0; + + debug(D_VINE, "Resubmitting node %" PRIu64, node->node_id); + vine_graph_executor_clear_node_runner_arg(e, node); + if (node->task) { + vine_task_delete(node->task); + node->task = NULL; + } + vine_graph_executor_submit_node(e, node); + resubmits++; + rotations_without_resubmit = 0; + } else { + list_pop_head(e->resubmit_queue); + list_push_tail(e->resubmit_queue, node); + rotations_without_resubmit++; + if (rotations_without_resubmit >= list_size(e->resubmit_queue)) { + break; + } + } + } +} + +/* + * Verify status and on-disk outputs (primary outfile only). + * On failure enqueue a retry for the node. + */ +static int vine_graph_executor_validate_node_outputs_or_retry(struct vine_graph_executor *e, struct vine_graph_node *retry_node, struct vine_graph_node *output_node, struct vine_task *task) +{ + switch (output_node->outfile_type) { + case VINE_GRAPH_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: { + struct stat info; + // shared path may lag behind a successful task result code + if (stat(output_node->outfile_remote_name, &info) < 0) { + debug(D_VINE, "Task %d succeeded but missing sharedfs output %s", task->task_id, output_node->outfile_remote_name); + vine_graph_executor_queue_node_retry(e, retry_node); + return 0; + } + output_node->outfile_size_bytes = info.st_size; + vine_graph_executor_account_pfs_write(e, output_node, (size_t)info.st_size); + break; + } + case VINE_GRAPH_NODE_OUTFILE_TYPE_LOCAL: + case VINE_GRAPH_NODE_OUTFILE_TYPE_TEMP: + if (output_node->outfile) { + if (output_node->outfile->type == VINE_TEMP || output_node->outfile->type == VINE_BUFFER) { + if (output_node->outfile->state != VINE_FILE_STATE_CREATED) { + debug(D_VINE, + "Task %d succeeded but primary output for node %" PRIu64 " is not available (state=%d)", + task->task_id, + output_node->node_id, + (int)output_node->outfile->state); + vine_graph_executor_queue_node_retry(e, retry_node); + return 0; + } + } + output_node->outfile_size_bytes = output_node->outfile->size; + } + break; + } + + return 1; +} + +/* + * Verify one extra output mount after a successful task (temp/buffer must be CREATED; VINE_FILE paths must exist). + */ +static int vine_graph_executor_validate_io_mount_or_retry( + struct vine_graph_executor *e, struct vine_graph_node *retry_node, struct vine_graph_io_mount *mount, struct vine_task *task) +{ + if (!mount || !mount->file) { + return 1; + } + struct vine_file *f = mount->file; + + switch (f->type) { + case VINE_TEMP: + case VINE_BUFFER: + if (f->state != VINE_FILE_STATE_CREATED) { + debug(D_VINE, + "Task %d succeeded but extra output \"%s\" is not available (state=%d)", + task->task_id, + mount->remote_name ? mount->remote_name : "?", + (int)f->state); + vine_graph_executor_queue_node_retry(e, retry_node); + return 0; + } + break; + case VINE_FILE: + if (f->source) { + struct stat info; + if (stat(f->source, &info) < 0) { + debug(D_VINE, + "Task %d succeeded but missing extra output file %s (%s)", + task->task_id, + mount->remote_name ? mount->remote_name : "?", + f->source); + vine_graph_executor_queue_node_retry(e, retry_node); + return 0; + } + } + break; + default: + break; + } + return 1; +} + +/* + * Primary outfile (per node outfile_type) plus every extra_outputs mount declared for that graph node. + */ +static int vine_graph_executor_validate_all_declared_outputs_or_retry( + struct vine_graph_executor *e, struct vine_graph_node *retry_node, struct vine_graph_node *output_node, struct vine_task *task) +{ + if (!vine_graph_executor_validate_node_outputs_or_retry(e, retry_node, output_node, task)) { + return 0; + } + void *item; + LIST_ITERATE(output_node->extra_outputs, item) + { + if (!vine_graph_executor_validate_io_mount_or_retry(e, retry_node, (struct vine_graph_io_mount *)item, task)) { + return 0; + } + } + return 1; +} + +static int vine_graph_executor_validate_task_or_retry(struct vine_graph_executor *e, struct vine_graph_node *node, struct vine_task *task) +{ + struct vine_graph *g = e ? e->graph : NULL; + /* + * Returning zero means the completion is rejected, a retry is enqueued, and the progress + * bar must not advance because the batch did not truly succeed yet. + */ + if (task->result != VINE_RESULT_SUCCESS || task->exit_code != 0) { + debug(D_VINE, "Task %d failed (result=%d, exit=%d)", task->task_id, task->result, task->exit_code); + vine_graph_executor_queue_node_retry(e, node); + return 0; + } + + if (!vine_graph_executor_validate_all_declared_outputs_or_retry(e, node, node, task)) { + return 0; + } + + /* + * One vine task may carry outputs for the whole supernode. Validate every grouped member's + * declared files against that same task, not only the leader's primary outfile. + */ + if (g && g->chain_grouping_enabled) { + struct list *mems = vine_graph_supernode_nonleader_members(g, node->node_id); + if (mems) { + struct vine_graph_node *m; + LIST_ITERATE(mems, m) + { + if (!vine_graph_executor_validate_all_declared_outputs_or_retry(e, node, m, task)) { + return 0; + } + } + } + } + + return 1; +} + +/* Return the recorded user-task makespan in microseconds. */ +uint64_t vine_graph_executor_get_makespan_us(const struct vine_graph_executor *e) +{ + if (!e) { + return 0; + } + + return (uint64_t)e->makespan_us; +} + +/* Return the manager's cumulative count of recovery tasks submitted. */ +uint64_t vine_graph_executor_get_total_recovery_tasks(const struct vine_graph_executor *e) +{ + if (!e || !e->manager || !e->manager->stats) { + return 0; + } + + return (uint64_t)e->manager->stats->tasks_recovery; +} + +/* Return how many recovery tasks have completed in the current executor run. */ +uint64_t vine_graph_executor_get_completed_recovery_tasks(const struct vine_graph_executor *e) +{ + if (!e) { + return 0; + } + + return e->completed_recovery_tasks; +} + +/* Main loop: submit work, wait, handle recovery, update graph state. */ +void vine_graph_executor_execute(struct vine_graph_executor *e) +{ + struct vine_graph *g = e ? e->graph : NULL; + if (!g) { + return; + } + + interrupted = 0; + void (*previous_sigint_handler)(int) = signal(SIGINT, vine_graph_executor_handle_sigint); + + debug(D_VINE, "start executing executor graph"); + + vine_graph_executor_submit_initial_ready_nodes(e); + + struct ProgressBar *pbar = progress_bar_init("Executing Tasks"); + progress_bar_set_update_interval(pbar, e->progress_bar_update_interval_sec); + e->completed_recovery_tasks = 0; + + struct ProgressBarPart *user_tasks_part = progress_bar_create_part("User", itable_size(g->nodes)); + struct ProgressBarPart *recovery_tasks_part = progress_bar_create_part("Recovery", 0); + progress_bar_bind_part(pbar, user_tasks_part); + progress_bar_bind_part(pbar, recovery_tasks_part); + + const uint64_t user_node_total = itable_size(g->nodes); + + double next_failure_threshold = -1.0; + if (e->failure_injection_step_percent > 0) { + next_failure_threshold = e->failure_injection_step_percent / 100.0; + } + + int wait_timeout = 1; // short timeout after a result, longer when idle + + /* + * Stop the main loop once every graph node reports completed. Count completed nodes directly + * instead of relying on progress bar ticks because grouped members can flip to completed in + * a single completion event and the bar would miss intermediate steps. + */ + while (vine_graph_executor_count_completed_user_nodes(g) < user_node_total) { + if (interrupted) { + break; + } + + vine_graph_executor_drain_resubmit_queue(e); + progress_bar_set_part_total(pbar, recovery_tasks_part, vine_graph_executor_get_total_recovery_tasks(e)); + + struct vine_task *task = vine_wait(e->manager, wait_timeout); + if (task) { + wait_timeout = 0; + + struct vine_graph_node *node = vine_graph_executor_node_from_task(e, task); + if (!node) { + debug(D_ERROR, "fatal: task %d could not be mapped to a task node, this indicates a serious bug.", task->task_id); + exit(1); + } + + if (task->time_when_commit_end > 0) { + e->time_first_task_dispatched = MIN(e->time_first_task_dispatched, task->time_when_commit_end); // makespan start + } + + /* + * User and recovery progress advances only after outputs validate. Failed tasks enter + * the retry path and leave the bar unchanged until a later successful completion. + */ + if (!vine_graph_executor_validate_task_or_retry(e, node, task)) { + continue; + } + + int first_completion = 0; + + if (task->type == VINE_TASK_TYPE_RECOVERY) { + e->completed_recovery_tasks++; + progress_bar_update_part( + pbar, + recovery_tasks_part, + e->completed_recovery_tasks - recovery_tasks_part->current); + + /* Reset cut and prune-depth flags for recovery tasks. */ + node->cut = 0; + node->released_by_prune_depth = 0; + + /* Only postprocess recovery tasks. */ + vine_graph_executor_run_completion_postprocess(e, node); + } else { + e->time_last_task_retrieved = MAX(e->time_last_task_retrieved, task->time_when_retrieval); + e->makespan_us = e->time_last_task_retrieved - e->time_first_task_dispatched; + + first_completion = !node->completed; + vine_graph_executor_mark_user_node_completed_after_success(g, node); + + if (first_completion) { + if (user_tasks_part->current == 0) { + progress_bar_set_start_time(pbar, task->time_when_commit_start); + } + + vine_graph_node_update_critical_path_time(node, task->time_workers_execute_last); + } + + progress_bar_set_part_current(pbar, user_tasks_part, vine_graph_executor_count_completed_user_nodes(g)); + + if (e->failure_injection_step_percent > 0) { + // test hook, drop workers at stepped progress thresholds + double progress = (double)user_tasks_part->current / (double)user_tasks_part->total; + if (progress >= next_failure_threshold && release_random_worker(e->manager)) { + debug(D_VINE, "released a random worker at %.2f%% (threshold %.2f%%)", progress * 100, next_failure_threshold * 100); + next_failure_threshold += e->failure_injection_step_percent / 100.0; + } + } + + /* Postprocess the node and submit its children. */ + vine_graph_executor_run_completion_postprocess(e, node); + vine_graph_executor_submit_unblocked_children(e, node); + } + } else { + wait_timeout = 1; // no task ready, wait with default blocking timeout + } + } + + progress_bar_finish(pbar); + progress_bar_delete(pbar); + + debug(D_VINE, "total time spent on cut propagation: %.6f seconds\n", e->time_spent_on_cut_propagation / 1e6); + + signal(SIGINT, previous_sigint_handler); + if (interrupted) { + raise(SIGINT); // restore handler first, then honor prior interrupt + } +} diff --git a/taskvine/src/graph/vine_graph_executor.h b/taskvine/src/graph/vine_graph_executor.h new file mode 100644 index 0000000000..475daaea57 --- /dev/null +++ b/taskvine/src/graph/vine_graph_executor.h @@ -0,0 +1,56 @@ +#ifndef VINE_GRAPH_EXECUTOR_H +#define VINE_GRAPH_EXECUTOR_H + +#include "vine_graph.h" + +#include "vine_manager.h" + +typedef enum { + TASK_PRIORITY_MODE_RANDOM = 0, + TASK_PRIORITY_MODE_DEPTH_FIRST, + TASK_PRIORITY_MODE_BREADTH_FIRST, + TASK_PRIORITY_MODE_FIFO, + TASK_PRIORITY_MODE_LIFO, + TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST, + TASK_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST +} task_priority_mode_t; + +struct vine_graph_executor { + struct vine_graph *graph; // DAG executed by this executor + struct vine_manager *manager; // TaskVine runtime + + struct itable *task_id_to_node; // maps vine task id to graph node after submit + struct list *resubmit_queue; // nodes waiting for retry + + timestamp_t time_first_task_dispatched; // earliest dispatch time among user tasks + timestamp_t time_last_task_retrieved; // latest user task retrieval time + timestamp_t makespan_us; // workflow span in microseconds + timestamp_t time_spent_on_cut_propagation; // time spent in cut propagation + uint64_t completed_recovery_tasks; // recovery completions seen this run + uint64_t pfs_usage_bytes; // bytes credited for shared filesystem outputs + /** Sum of @c vine_graph_executor_submit_node preprocessing intervals across all nodes (microseconds). */ + uint64_t total_preprocessing_time_us; + /** Sum of @c vine_graph_executor_run_completion_postprocess intervals across all completions (microseconds). */ + uint64_t total_postprocessing_time_us; + + task_priority_mode_t task_priority_mode; // schedule order before submit + double failure_injection_step_percent; // optional worker release steps for tests + double progress_bar_update_interval_sec; + int enable_debug_log; +}; + +struct vine_graph_executor *vine_graph_executor_create(struct vine_manager *manager, struct vine_graph *graph); +struct vine_graph *vine_graph_executor_create_graph(struct vine_manager *manager); +void vine_graph_executor_delete(struct vine_graph_executor *e); +uint64_t vine_graph_executor_add_node(struct vine_graph_executor *e); +void vine_graph_executor_finalize(struct vine_graph_executor *e); +void vine_graph_executor_add_task_input(struct vine_graph_executor *e, uint64_t task_id, const char *filename); +void vine_graph_executor_add_task_output(struct vine_graph_executor *e, uint64_t task_id, const char *filename); + +int vine_graph_executor_tune(struct vine_graph_executor *e, const char *name, const char *value); +void vine_graph_executor_execute(struct vine_graph_executor *e); +uint64_t vine_graph_executor_get_makespan_us(const struct vine_graph_executor *e); +uint64_t vine_graph_executor_get_total_recovery_tasks(const struct vine_graph_executor *e); +uint64_t vine_graph_executor_get_completed_recovery_tasks(const struct vine_graph_executor *e); + +#endif // VINE_GRAPH_EXECUTOR_H diff --git a/taskvine/src/graph/vine_graph_node.c b/taskvine/src/graph/vine_graph_node.c new file mode 100644 index 0000000000..4545d68d27 --- /dev/null +++ b/taskvine/src/graph/vine_graph_node.c @@ -0,0 +1,275 @@ +#include +#include + +#include "debug.h" +#include "list.h" +#include "stringtools.h" +#include "xxmalloc.h" + +#include "vine_graph_node.h" + +/*************************************************************/ +/* Public APIs */ +/*************************************************************/ + +/** + * Update the critical path time of a node. + * @param node Reference to the node object. + * @param execution_time Reference to the execution time of the node. + */ +void vine_graph_node_update_critical_path_time(struct vine_graph_node *node, timestamp_t execution_time) +{ + timestamp_t max_parent_critical_path_time = 0; + struct vine_graph_node *parent_node; + LIST_ITERATE(node->parents, parent_node) + { + if (parent_node->critical_path_time > max_parent_critical_path_time) { + max_parent_critical_path_time = parent_node->critical_path_time; + } + } + node->critical_path_time = max_parent_critical_path_time + execution_time; +} + +/** + * Create a new node owned by the C-side graph. + * @param node_id Graph-assigned identifier that keeps C and Python in sync. + * @return Newly allocated node. + */ +struct vine_graph_node *vine_graph_node_create(uint64_t node_id) +{ + struct vine_graph_node *node = xxmalloc(sizeof(struct vine_graph_node)); + + node->is_target = 0; + node->node_id = node_id; + node->super_leader_id = node_id; + + node->task = NULL; + node->task_runner_arg_file = NULL; + node->outfile = NULL; + node->outfile_remote_name = string_format("outfile_node_%" PRIu64, node->node_id); + node->outfile_type = VINE_GRAPH_NODE_OUTFILE_TYPE_TEMP; + + node->parents = list_create(); + node->children = list_create(); + node->extra_outputs = list_create(); + node->extra_inputs = list_create(); + node->remaining_parents_count = 0; + node->fired_parents = NULL; + node->completed = 0; + node->cut = 0; + node->released_by_prune_depth = 0; + node->outfile_size_bytes = 0; + node->pfs_credited_bytes = 0; + node->in_resubmit_queue = 0; + node->last_failure_time = 0; + + node->depth = -1; + node->height = -1; + node->upstream_subgraph_size = -1; + node->downstream_subgraph_size = -1; + node->fan_in = -1; + node->fan_out = -1; + node->heavy_score = -1; + + node->critical_path_time = -1; + node->preprocessing_time_us = 0; + node->postprocessing_time_us = 0; + + return node; +} + +/** Non-zero if parent->child is already in the adjacency lists (checked via parent's children). */ +static int vine_graph_node_dependency_exists(struct vine_graph_node *parent, struct vine_graph_node *child) +{ + struct vine_graph_node *x; + if (!parent || !child) { + return 0; + } + LIST_ITERATE(parent->children, x) + { + if (x == child) { + return 1; + } + } + return 0; +} + +void vine_graph_node_remove_dependency(struct vine_graph_node *parent, struct vine_graph_node *child) +{ + if (!parent || !child) { + return; + } + list_remove(child->parents, parent); + list_remove(parent->children, child); +} + +void vine_graph_node_ensure_dependency(struct vine_graph_node *parent, struct vine_graph_node *child) +{ + if (!parent || !child || vine_graph_node_dependency_exists(parent, child)) { + return; + } + list_push_tail(child->parents, parent); + list_push_tail(parent->children, child); +} + +/** + * Drop fired_parents so executor scheduling can recount parents (e.g. after supernode merge rewires edges). + */ +void vine_graph_node_clear_fired_parents(struct vine_graph_node *n) +{ + if (!n || !n->fired_parents) { + return; + } + set_delete(n->fired_parents); + n->fired_parents = NULL; +} + +/** + * Construct the task arguments for the node. + * @param node Reference to the node object. + * @return The task arguments in JSON format: {"fn_args": ["node_id"], "fn_kwargs": {}} (string for run_scheduler_keys). + */ +char *vine_graph_node_construct_task_arguments(struct vine_graph_node *node) +{ + if (!node) { + return NULL; + } + return string_format("{\"fn_args\":[\"%" PRIu64 "\"],\"fn_kwargs\":{}}", node->node_id); +} + +/** + * Print the info of the node. + * @param node Reference to the node object. + */ +void vine_graph_node_debug_print(struct vine_graph_node *node) +{ + if (!node) { + return; + } + + debug(D_VINE, "---------------- Node Info ----------------"); + debug(D_VINE, "node_id: %" PRIu64, node->node_id); + debug(D_VINE, "preprocessing_time_us (last): %" PRIu64, node->preprocessing_time_us); + debug(D_VINE, "postprocessing_time_us (last): %" PRIu64, node->postprocessing_time_us); + + if (!node->task) { + debug(D_VINE, "task: (none yet)"); + debug(D_VINE, "-------------------------------------------"); + return; + } + + debug(D_VINE, "task_id: %d", node->task->task_id); + debug(D_VINE, "depth: %d", node->depth); + debug(D_VINE, "height: %d", node->height); + + if (node->outfile_remote_name) { + debug(D_VINE, "outfile_remote_name: %s", node->outfile_remote_name); + } + + if (node->outfile) { + const char *type_str = "UNKNOWN"; + switch (node->outfile->type) { + case VINE_FILE: + type_str = "VINE_FILE"; + break; + case VINE_TEMP: + type_str = "VINE_TEMP"; + break; + case VINE_URL: + type_str = "VINE_URL"; + break; + case VINE_BUFFER: + type_str = "VINE_BUFFER"; + break; + case VINE_MINI_TASK: + type_str = "VINE_MINI_TASK"; + break; + } + debug(D_VINE, "outfile_type: %s", type_str); + debug(D_VINE, "outfile_cached_name: %s", node->outfile->cached_name ? node->outfile->cached_name : "(null)"); + } else { + debug(D_VINE, "outfile_type: SHARED_FILE_SYSTEM or none"); + } + + char *parent_ids = NULL; // comma separated parent ids for logging + struct vine_graph_node *p; + LIST_ITERATE(node->parents, p) + { + if (!parent_ids) { + parent_ids = string_format("%" PRIu64, p->node_id); + } else { + char *tmp = string_format("%s, %" PRIu64, parent_ids, p->node_id); + free(parent_ids); + parent_ids = tmp; + } + } + + char *child_ids = NULL; // comma separated child ids for logging + struct vine_graph_node *c; + LIST_ITERATE(node->children, c) + { + if (!child_ids) { + child_ids = string_format("%" PRIu64, c->node_id); + } else { + char *tmp = string_format("%s, %" PRIu64, child_ids, c->node_id); + free(child_ids); + child_ids = tmp; + } + } + + debug(D_VINE, "parents: %s", parent_ids ? parent_ids : "(none)"); + debug(D_VINE, "children: %s", child_ids ? child_ids : "(none)"); + + free(parent_ids); + free(child_ids); + + debug(D_VINE, "-------------------------------------------"); +} + +/** + * Delete the node and all of its associated resources. + * @param node Reference to the node object. + */ +void vine_graph_node_delete(struct vine_graph_node *node) +{ + if (!node) { + return; + } + + if (node->outfile_remote_name) { + free(node->outfile_remote_name); + } + + vine_task_delete(node->task); + node->task = NULL; + + if (node->task_runner_arg_file) { + vine_file_delete(node->task_runner_arg_file); + node->task_runner_arg_file = NULL; + } + if (node->outfile) { + vine_file_delete(node->outfile); + node->outfile = NULL; + } + + list_delete(node->parents); + list_delete(node->children); + + while (list_size(node->extra_inputs) > 0) { + struct vine_graph_io_mount *m = list_pop_head(node->extra_inputs); + free(m->remote_name); + free(m); + } + list_delete(node->extra_inputs); + while (list_size(node->extra_outputs) > 0) { + struct vine_graph_io_mount *m = list_pop_head(node->extra_outputs); + free(m->remote_name); + free(m); + } + list_delete(node->extra_outputs); + + if (node->fired_parents) { + set_delete(node->fired_parents); + } + free(node); +} \ No newline at end of file diff --git a/taskvine/src/graph/vine_graph_node.h b/taskvine/src/graph/vine_graph_node.h new file mode 100644 index 0000000000..cb411de882 --- /dev/null +++ b/taskvine/src/graph/vine_graph_node.h @@ -0,0 +1,131 @@ +#ifndef VINE_GRAPH_NODE_H +#define VINE_GRAPH_NODE_H + +#include + +#include "set.h" +#include "timestamp.h" + +#include "list.h" +#include "vine_file.h" +#include "vine_task.h" + +/** + * One element of @c extra_outputs or @c extra_inputs: a logical filename plus its @c vine_file + * (declared during graph build; attached to @c vine_task in @c vine_graph_executor_materialize_node). + */ +struct vine_graph_io_mount { + struct vine_file *file; + char *remote_name; +}; + +/** The storage type of the node's output file. */ +typedef enum { + VINE_GRAPH_NODE_OUTFILE_TYPE_LOCAL = 0, // staged file under graph output_dir + VINE_GRAPH_NODE_OUTFILE_TYPE_TEMP, // TaskVine temp blob + VINE_GRAPH_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM, // path on shared storage, no vine_file +} vine_graph_node_outfile_type_t; + +/** The node object. */ +struct vine_graph_node { + uint64_t node_id; // graph assigned id + /** + * Supernode leader id for scheduling: equals @c node_id for a single-node group. + * After @c vine_graph_supernode_register, every member shares the same leader id. + */ + uint64_t super_leader_id; + int is_target; // if set, output is retrieved when the task completes + + struct vine_task *task; + struct vine_file *task_runner_arg_file; // JSON args buffer for the runner + struct vine_file *outfile; // NULL when output is PFS only + char *outfile_remote_name; + size_t outfile_size_bytes; + vine_graph_node_outfile_type_t outfile_type; + size_t pfs_credited_bytes; // contribution to executor pfs_usage_bytes + + struct list *parents; + struct list *children; + /** + * @c Workflow.task_produces / @c vine_graph_executor_add_task_output: extra outputs beyond this node’s primary + * @c outfile (the runner’s standard @c outfile_node_* result). Each list item is a declared + * @c vine_file keyed by the same logical path string used by consumers. Filled before + * @c node->task exists; consumed when building the task at submit / materialize time. + */ + struct list *extra_outputs; + /** + * @c Workflow.task_consumes / @c vine_graph_executor_add_task_input: extra inputs beyond those implied by + * the DAG via parent @c outfile edges (paired logical files shared with producers). Same lifecycle + * as @c extra_outputs: queued at graph build, wired on @c vine_task at materialize. + */ + struct list *extra_inputs; + + int remaining_parents_count; // parents not yet satisfied for scheduling + struct set *fired_parents; // parents already counted toward that count + int completed; + int cut; // return released by cut, cleared if recovery restores file + /** Non-zero after this node's temp output was released under @c graph->prune_depth; cleared on recovery. */ + int released_by_prune_depth; + int in_resubmit_queue; + timestamp_t last_failure_time; // last enqueue to resubmit queue + + int depth; + int height; + int upstream_subgraph_size; + int downstream_subgraph_size; + int fan_in; + int fan_out; + double heavy_score; + + timestamp_t critical_path_time; + /** Latest @c vine_graph_executor_submit_node interval for this node (microseconds); graph total is on @c struct vine_graph_executor. */ + uint64_t preprocessing_time_us; + /** Latest @c vine_graph_executor_run_completion_postprocess interval for this node (microseconds); graph total on executor. */ + uint64_t postprocessing_time_us; +}; + +/** Create a new node. +@param node_id Unique node identifier supplied by the owning graph. +@return Newly allocated node instance. +*/ +struct vine_graph_node *vine_graph_node_create(uint64_t node_id); + +/** + * Remove parent->child from both endpoints' parents/children lists (no-op if NULL). + * Used when rewiring supernodes so stale edges do not corrupt fan-in/out. + */ +void vine_graph_node_remove_dependency(struct vine_graph_node *parent, struct vine_graph_node *child); + +/** + * Add parent->child if that edge is not already present (idempotent). + */ +void vine_graph_node_ensure_dependency(struct vine_graph_node *parent, struct vine_graph_node *child); + +/** + * Drop fired_parents so executor scheduling can recount parents (e.g. after supernode merge rewires edges). + */ +void vine_graph_node_clear_fired_parents(struct vine_graph_node *n); + +/** Create the task arguments for a node. +@param node Reference to the node. +@return The task arguments in JSON format: {"fn_args": ["node_id"], "fn_kwargs": {}} (string id for run_scheduler_keys). +*/ +char *vine_graph_node_construct_task_arguments(struct vine_graph_node *node); + +/** Delete a node and release owned resources. +@param node Reference to the node. +*/ +void vine_graph_node_delete(struct vine_graph_node *node); + +/** Print information about a node. +@param node Reference to the node. +*/ +void vine_graph_node_debug_print(struct vine_graph_node *node); + +/** Update the critical path time of a node. +@param node Reference to the node. +@param execution_time Reference to the execution time of the node. +*/ +void vine_graph_node_update_critical_path_time(struct vine_graph_node *node, timestamp_t execution_time); + +#endif // VINE_GRAPH_NODE_H diff --git a/taskvine/src/manager/taskvine.h b/taskvine/src/manager/taskvine.h index 45625d3887..b8e4fd2e2e 100644 --- a/taskvine/src/manager/taskvine.h +++ b/taskvine/src/manager/taskvine.h @@ -1119,6 +1119,14 @@ int vine_enable_peer_transfers(struct vine_manager *m); /** Disable taskvine peer transfers to be scheduled by the manager **/ int vine_disable_peer_transfers(struct vine_manager *m); +/** Enable recovery tasks to be returned by vine_wait. +By default, recovery tasks are handled internally by the manager. **/ +int vine_enable_return_recovery_tasks(struct vine_manager *m); + +/** Disable recovery tasks from being returned by vine_wait. +Recovery tasks will be handled internally by the manager. **/ +int vine_disable_return_recovery_tasks(struct vine_manager *m); + /** When enabled, resources to tasks in are assigned in proportion to the size of the worker. If a resource is specified (e.g. with @ref vine_task_set_cores), proportional resources never go below explicit specifications. This mode is most @@ -1538,6 +1546,12 @@ void vine_counters_print(); */ char *vine_version_string(); +/** Absolute path of this run's workflow runtime directory. +@param m The manager. +@return Owned by the manager until @ref vine_delete. + */ +const char *vine_get_runtime_directory(struct vine_manager *m); + /** Returns path relative to the logs runtime directory @param m Reference to the current manager object. @param path Target filename. diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 9895a9ae26..98e3d655cd 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -4313,6 +4313,20 @@ int vine_disable_peer_transfers(struct vine_manager *q) return 1; } +int vine_enable_return_recovery_tasks(struct vine_manager *q) +{ + debug(D_VINE, "Return recovery tasks enabled"); + q->return_recovery_tasks = 1; + return 1; +} + +int vine_disable_return_recovery_tasks(struct vine_manager *q) +{ + debug(D_VINE, "Return recovery tasks disabled"); + q->return_recovery_tasks = 0; + return 1; +} + int vine_enable_proportional_resources(struct vine_manager *q) { debug(D_VINE, "Proportional resources enabled"); @@ -5231,7 +5245,11 @@ struct vine_task *find_task_to_return(struct vine_manager *q, const char *tag, i return t; break; case VINE_TASK_TYPE_RECOVERY: - /* do nothing and let vine_manager_consider_recovery_task do its job */ + /* if configured to return recovery tasks, return them to the user */ + if (q->return_recovery_tasks) { + return t; + } + /* otherwise, do nothing and let vine_manager_consider_recovery_task do its job */ break; case VINE_TASK_TYPE_LIBRARY_INSTANCE: /* silently delete the task, since it was created by the manager. @@ -5993,10 +6011,12 @@ int vine_tune(struct vine_manager *q, const char *name, double value) } else if (!strcmp(name, "max-library-retries")) { q->max_library_retries = MIN(1, value); + } else if (!strcmp(name, "disk-proportion-available-to-task")) { if (value < 1 && value > 0) { q->disk_proportion_available_to_task = value; } + } else if (!strcmp(name, "enforce-worker-eviction-interval")) { q->enforce_worker_eviction_interval = (timestamp_t)(MAX(0, (int)value) * ONE_SECOND); @@ -6006,6 +6026,12 @@ int vine_tune(struct vine_manager *q, const char *name, double value) } else if (!strcmp(name, "shift-disk-load")) { q->shift_disk_load = !!((int)value); + } else if (strcmp(name, "enable-debug-log") == 0) { + if ((int)value == 0) { + debug_flags_clear(); + debug_close(); + } + } else { debug(D_NOTICE | D_VINE, "Warning: tuning parameter \"%s\" not recognized\n", name); return -1; diff --git a/taskvine/src/manager/vine_manager.h b/taskvine/src/manager/vine_manager.h index bcc2d2afd1..46f371743a 100644 --- a/taskvine/src/manager/vine_manager.h +++ b/taskvine/src/manager/vine_manager.h @@ -222,6 +222,7 @@ struct vine_manager { int temp_replica_count; /* Number of replicas per temp file */ int clean_redundant_replicas; /* If true, remove redundant replicas of temp files to save disk space. */ int shift_disk_load; /* If true, shift storage burden to more available workers to minimize disk usage peaks. */ + int return_recovery_tasks; /* If true, recovery tasks are returned by vine_wait to the user. By default they are handled internally. */ double resource_submit_multiplier; /* Factor to permit overcommitment of resources at each worker. */ double bandwidth_limit; /* Artificial limit on bandwidth of manager<->worker transfers. */ diff --git a/taskvine/src/manager/vine_runtime_dir.c b/taskvine/src/manager/vine_runtime_dir.c index a2bc4194a0..421792ca26 100644 --- a/taskvine/src/manager/vine_runtime_dir.c +++ b/taskvine/src/manager/vine_runtime_dir.c @@ -134,6 +134,14 @@ char *vine_runtime_directory_create() return runtime_dir; } +const char *vine_get_runtime_directory(struct vine_manager *m) +{ + if (!m || !m->runtime_directory) { + return NULL; + } + return m->runtime_directory; +} + char *vine_get_path_log(struct vine_manager *m, const char *path) { return string_format("%s/vine-logs%s%s", m->runtime_directory, path ? "/" : "", path ? path : ""); diff --git a/taskvine/src/manager/vine_task.c b/taskvine/src/manager/vine_task.c index c401324b9f..ed4ccb2cf8 100644 --- a/taskvine/src/manager/vine_task.c +++ b/taskvine/src/manager/vine_task.c @@ -154,6 +154,9 @@ void vine_task_reset(struct vine_task *t) t->time_workers_execute_exhaustion = 0; t->time_workers_execute_failure = 0; + t->time_when_commit_start = 0; + t->time_when_commit_end = 0; + rmsummary_delete(t->resources_measured); rmsummary_delete(t->resources_allocated); t->resources_measured = rmsummary_create(-1); diff --git a/taskvine/src/worker/vine_cache.c b/taskvine/src/worker/vine_cache.c index 40b226c11e..fc97b46416 100644 --- a/taskvine/src/worker/vine_cache.c +++ b/taskvine/src/worker/vine_cache.c @@ -941,13 +941,22 @@ If any have definitively failed, they are removed from the cache. int vine_cache_check_files(struct vine_cache *c, struct link *manager) { - struct vine_cache_file *f; + if (hash_table_size(c->processing_transfers) < 1) { + return 1; + } + + char **cachenames = hash_table_keys_array(c->processing_transfers); + int i = 0; char *cachename; - HASH_TABLE_ITERATE(c->table, cachename, f) - { - vine_cache_check_file(c, f, cachename, manager); + while ((cachename = cachenames[i])) { + i++; + struct vine_cache_file *f = hash_table_lookup(c->table, cachename); + if (f) { + vine_cache_check_file(c, f, cachename, manager); + } - if (f->status == VINE_CACHE_STATUS_FAILED) { + f = hash_table_lookup(c->table, cachename); + if (f && f->status == VINE_CACHE_STATUS_FAILED) { /* if transfer failed, then we delete all of our records of the file. The manager * assumes that the file is not at the worker after the manager receives * the cache invalid message sent from vine_cache_check_outputs. */ @@ -956,5 +965,7 @@ int vine_cache_check_files(struct vine_cache *c, struct link *manager) /* Note that f may no longer be valid at this point */ } + + hash_table_free_keys_array(cachenames); return 1; }