diff --git a/parsl/dataflow/memoization.py b/parsl/dataflow/memoization.py index 0a5f541b9c..c23b297f39 100644 --- a/parsl/dataflow/memoization.py +++ b/parsl/dataflow/memoization.py @@ -167,7 +167,7 @@ def close(self) -> None: def update_memo(self, task: TaskRecord, r: Future[Any]) -> None: raise NotImplementedError - def checkpoint(self, tasks: Sequence[TaskRecord]) -> str: + def checkpoint(self, tasks: Sequence[TaskRecord]) -> None: raise NotImplementedError def check_memo(self, task: TaskRecord) -> Optional[Future[Any]]: @@ -388,7 +388,7 @@ def load_checkpoints(self, checkpointDirs: Optional[Sequence[str]]) -> Dict[str, else: return {} - def checkpoint(self, tasks: Sequence[TaskRecord]) -> str: + def checkpoint(self, tasks: Sequence[TaskRecord]) -> None: """Checkpoint the dfk incrementally to a checkpoint file. When called, every task that has been completed yet not @@ -457,8 +457,6 @@ def checkpoint(self, tasks: Sequence[TaskRecord]) -> str: else: logger.info("Done checkpointing {} tasks".format(count)) - return checkpoint_dir - def filter_for_checkpoint(self, app_fu: AppFuture) -> bool: """Overridable method to decide if an entry should be checkpointed""" return app_fu.exception() is None diff --git a/parsl/dataflow/memosql.py b/parsl/dataflow/memosql.py new file mode 100644 index 0000000000..fa9a00f668 --- /dev/null +++ b/parsl/dataflow/memosql.py @@ -0,0 +1,114 @@ +import logging +import pickle +import sqlite3 +from concurrent.futures import Future +from pathlib import Path +from typing import Any, Optional, Sequence + +from parsl.dataflow.dflow import DataFlowKernel +from parsl.dataflow.memoization import Memoizer, make_hash +from parsl.dataflow.taskrecord import TaskRecord + +logger = logging.getLogger(__name__) + + +class SQLiteMemoizer(Memoizer): + """Memoize out of memory into an sqlite3 database. + + TODO: probably going to need some kind of shutdown now, to close + the sqlite3 connection. + which might also be useful for driving final checkpoints in the + original impl? + """ + + def start(self, *, dfk: DataFlowKernel, memoize: bool = True, checkpoint_files: Sequence[str], run_dir: str) -> None: + """TODO: run_dir is the per-workflow run dir, but we need a broader checkpoint context... one level up + by default... get_all_checkpoints uses "runinfo/" as a relative path for that by default so replicating + that choice would do here. likewise I think for monitoring.""" + + self.db_path = Path(dfk.config.run_dir) / "checkpoint.sqlite3" + logger.debug("starting with db_path %r", self.db_path) + + # TODO: api wart... turning memoization on or off should not be part of the plugin API + self.memoize = memoize + + # TODO: this should not be an assert + assert sqlite3.threadsafety == 3, "need thread safe sqlite3 library to use check_same_thread=False" + connection = sqlite3.connect(self.db_path, check_same_thread=False) + cursor = connection.cursor() + + cursor.execute("CREATE TABLE IF NOT EXISTS checkpoints(key, result)") + # probably want some index on key because that's what we're doing all the access via. + + connection.commit() + connection.close() + logger.debug("checkpoint table created") + + def close(self): + pass + + def checkpoint(self, tasks: Sequence[TaskRecord]) -> None: + """All the behaviour for this memoizer is in check_memo and update_memo. + """ + logger.debug("Explicit checkpoint call is a no-op with this memoizer") + + def check_memo(self, task: TaskRecord) -> Optional[Future[Any]]: + """TODO: document this: check_memo is required to set the task hashsum, + if that's how we're going to key checkpoints in update_memo. (that's not + a requirement though: other equalities are available.""" + task_id = task['id'] + hashsum = make_hash(task) + logger.debug("Task {} has memoization hash {}".format(task_id, hashsum)) + task['hashsum'] = hashsum + + connection = sqlite3.connect(self.db_path, check_same_thread=False) + cursor = connection.cursor() + cursor.execute("SELECT result FROM checkpoints WHERE key = ?", (hashsum, )) + r = cursor.fetchone() + + if r is None: + connection.close() + return None + else: + data = pickle.loads(r[0]) + connection.close() + + memo_fu: Future = Future() + + if data['exception'] is None: + memo_fu.set_result(data['result']) + else: + assert data['result'] is None + memo_fu.set_exception(data['exception']) + + return memo_fu + + def update_memo(self, task: TaskRecord, r: Future[Any]) -> None: + logger.debug("updating memo") + + if not self.memoize or not task['memoize'] or 'hashsum' not in task: + logger.debug("preconditions for memo not satisfied") + return + + if not isinstance(task['hashsum'], str): + logger.error(f"Attempting to update app cache entry but hashsum is not a string key: {task['hashsum']}") + return + + app_fu = task['app_fu'] + hashsum = task['hashsum'] + + # this comes from the original concatenation-based checkpoint code: + if app_fu.exception() is None: + t = {'hash': hashsum, 'exception': None, 'result': app_fu.result()} + else: + t = {'hash': hashsum, 'exception': app_fu.exception(), 'result': None} + + value = pickle.dumps(t) + + connection = sqlite3.connect(self.db_path, check_same_thread=False) + cursor = connection.cursor() + + cursor.execute("INSERT INTO checkpoints VALUES(?, ?)", (hashsum, value)) + + connection.commit() + connection.close() diff --git a/parsl/executors/high_throughput/interchange.py b/parsl/executors/high_throughput/interchange.py index cd7d0596a9..ff2e048a7f 100644 --- a/parsl/executors/high_throughput/interchange.py +++ b/parsl/executors/high_throughput/interchange.py @@ -483,7 +483,7 @@ def process_tasks_to_send(self, interesting_managers: Set[bytes]) -> None: # Check if there are tasks that could be sent to managers logger.debug( - "Managers count (interesting/total): {}/{}", + "Managers count (interesting/total): %s/%s", len(interesting_managers), len(self._ready_managers) ) diff --git a/parsl/tests/configs/htex_local_alternate.py b/parsl/tests/configs/htex_local_alternate.py index 52124211bc..91ad5589d1 100644 --- a/parsl/tests/configs/htex_local_alternate.py +++ b/parsl/tests/configs/htex_local_alternate.py @@ -23,6 +23,7 @@ from parsl.data_provider.ftp import FTPInTaskStaging from parsl.data_provider.http import HTTPInTaskStaging from parsl.data_provider.zip import ZipFileStaging +from parsl.dataflow.memosql import SQLiteMemoizer from parsl.executors import HighThroughputExecutor from parsl.launchers import SingleNodeLauncher @@ -66,7 +67,8 @@ def fresh_config(): monitoring_debug=False, resource_monitoring_interval=1, ), - usage_tracking=True + usage_tracking=True, + memoizer=SQLiteMemoizer() ) diff --git a/parsl/tests/test_checkpointing/test_python_checkpoint_2_sqlite.py b/parsl/tests/test_checkpointing/test_python_checkpoint_2_sqlite.py new file mode 100644 index 0000000000..756dcad113 --- /dev/null +++ b/parsl/tests/test_checkpointing/test_python_checkpoint_2_sqlite.py @@ -0,0 +1,44 @@ +import contextlib +import os + +import pytest + +import parsl +from parsl import python_app +from parsl.dataflow.memosql import SQLiteMemoizer +from parsl.tests.configs.local_threads_checkpoint import fresh_config + + +@contextlib.contextmanager +def parsl_configured(run_dir, **kw): + c = fresh_config() + c.memoizer = SQLiteMemoizer() + c.run_dir = run_dir + for config_attr, config_val in kw.items(): + setattr(c, config_attr, config_val) + dfk = parsl.load(c) + for ex in dfk.executors.values(): + ex.working_dir = run_dir + yield dfk + + parsl.dfk().cleanup() + + +@python_app(cache=True) +def uuid_app(): + import uuid + return uuid.uuid4() + + +@pytest.mark.local +def test_loading_checkpoint(tmpd_cwd): + """Load memoization table from previous checkpoint + """ + with parsl_configured(tmpd_cwd, checkpoint_mode="task_exit"): + checkpoint_files = [os.path.join(parsl.dfk().run_dir, "checkpoint")] + result = uuid_app().result() + + with parsl_configured(tmpd_cwd, checkpoint_files=checkpoint_files): + relaunched = uuid_app().result() + + assert result == relaunched, "Expected following call to uuid_app to return cached uuid"